You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by ag...@apache.org on 2015/09/03 23:43:10 UTC

[01/39] ignite git commit: ignite-1334 Fixed concurrent destroyCache/node stop. Check initFut result in GridDhtPartitionsExchangeFuture.

Repository: ignite
Updated Branches:
  refs/heads/ignite-264 8af9e64c1 -> 637afb75b


ignite-1334 Fixed concurrent destroyCache/node stop. Check initFut result in GridDhtPartitionsExchangeFuture.


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

Branch: refs/heads/ignite-264
Commit: a61cf24b741daa679a66fc904960b74005d3e442
Parents: 612a6b8
Author: sboikov <sb...@gridgain.com>
Authored: Mon Aug 31 03:13:52 2015 -0700
Committer: Valentin Kulichenko <va...@gmail.com>
Committed: Mon Aug 31 13:01:42 2015 -0700

----------------------------------------------------------------------
 .../processors/cache/GridCacheProcessor.java    | 25 ++++--
 .../GridDhtPartitionsExchangeFuture.java        | 92 +++++++++++--------
 .../cache/IgniteDynamicCacheAndNodeStop.java    | 95 ++++++++++++++++++++
 .../testsuites/IgniteCacheTestSuite2.java       |  2 +
 4 files changed, 173 insertions(+), 41 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/a61cf24b/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 8e2b20e..f13af23 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
@@ -91,6 +91,9 @@ public class GridCacheProcessor extends GridProcessorAdapter {
     /** */
     private final Map<String, GridCacheAdapter<?, ?>> caches;
 
+    /** Caches stopped from onKernalStop callback. */
+    private final Map<String, GridCacheAdapter> stoppedCaches = new ConcurrentHashMap<>();
+
     /** Map of proxies. */
     private final Map<String, IgniteCacheProxy<?, ?>> jCacheProxies;
 
@@ -854,14 +857,16 @@ public class GridCacheProcessor extends GridProcessorAdapter {
     @SuppressWarnings("unchecked")
     @Override public void stop(boolean cancel) throws IgniteCheckedException {
         for (String cacheName : stopSeq) {
-            GridCacheAdapter<?, ?> cache = caches.remove(maskNull(cacheName));
+            GridCacheAdapter<?, ?> cache = stoppedCaches.remove(maskNull(cacheName));
 
             if (cache != null)
                 stopCache(cache, cancel);
         }
 
-        for (GridCacheAdapter<?, ?> cache : caches.values())
-            stopCache(cache, cancel);
+        for (GridCacheAdapter<?, ?> cache : stoppedCaches.values()) {
+            if (cache == stoppedCaches.remove(maskNull(cache.name())))
+                stopCache(cache, cancel);
+        }
 
         List<? extends GridCacheSharedManager<?, ?>> mgrs = sharedCtx.managers();
 
@@ -893,15 +898,23 @@ public class GridCacheProcessor extends GridProcessorAdapter {
         cacheStartedLatch.countDown();
 
         for (String cacheName : stopSeq) {
-            GridCacheAdapter<?, ?> cache = caches.get(maskNull(cacheName));
+            GridCacheAdapter<?, ?> cache = caches.remove(maskNull(cacheName));
+
+            if (cache != null) {
+                stoppedCaches.put(maskNull(cacheName), cache);
 
-            if (cache != null)
                 onKernalStop(cache, cancel);
+            }
         }
 
         for (Map.Entry<String, GridCacheAdapter<?, ?>> entry : caches.entrySet()) {
-            if (!stopSeq.contains(entry.getKey()))
+            GridCacheAdapter<?, ?> cache = entry.getValue();
+
+            if (cache == caches.remove(entry.getKey())) {
+                stoppedCaches.put(entry.getKey(), cache);
+
                 onKernalStop(entry.getValue(), cancel);
+            }
         }
 
         List<? extends GridCacheSharedManager<?, ?>> sharedMgrs = sharedCtx.managers();

http://git-wip-us.apache.org/repos/asf/ignite/blob/a61cf24b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsExchangeFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsExchangeFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsExchangeFuture.java
index 4971ca6..b70fbc4 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsExchangeFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsExchangeFuture.java
@@ -1136,52 +1136,54 @@ public class GridDhtPartitionsExchangeFuture extends GridFutureAdapter<AffinityT
         }
         else {
             initFut.listen(new CI1<IgniteInternalFuture<Boolean>>() {
-                @Override public void apply(IgniteInternalFuture<Boolean> t) {
+                @Override public void apply(IgniteInternalFuture<Boolean> f) {
                     try {
-                        if (!t.get()) // Just to check if there was an error.
+                        if (!f.get())
                             return;
+                    }
+                    catch (IgniteCheckedException e) {
+                        U.error(log, "Failed to initialize exchange future: " + this, e);
 
-                        ClusterNode loc = cctx.localNode();
+                        return;
+                    }
 
-                        singleMsgs.put(nodeId, msg);
+                    ClusterNode loc = cctx.localNode();
 
-                        boolean match = true;
+                    singleMsgs.put(nodeId, msg);
 
-                        // Check if oldest node has changed.
-                        if (!oldestNode.get().equals(loc)) {
-                            match = false;
+                    boolean match = true;
 
-                            synchronized (mux) {
-                                // Double check.
-                                if (oldestNode.get().equals(loc))
-                                    match = true;
-                            }
+                    // Check if oldest node has changed.
+                    if (!oldestNode.get().equals(loc)) {
+                        match = false;
+
+                        synchronized (mux) {
+                            // Double check.
+                            if (oldestNode.get().equals(loc))
+                                match = true;
                         }
+                    }
 
-                        if (match) {
-                            boolean allReceived;
+                    if (match) {
+                        boolean allReceived;
 
-                            synchronized (rcvdIds) {
-                                if (rcvdIds.add(nodeId))
-                                    updatePartitionSingleMap(msg);
+                        synchronized (rcvdIds) {
+                            if (rcvdIds.add(nodeId))
+                                updatePartitionSingleMap(msg);
 
-                                allReceived = allReceived();
-                            }
+                            allReceived = allReceived();
+                        }
 
-                            // If got all replies, and initialization finished, and reply has not been sent yet.
-                            if (allReceived && ready.get() && replied.compareAndSet(false, true)) {
-                                spreadPartitions();
+                        // If got all replies, and initialization finished, and reply has not been sent yet.
+                        if (allReceived && ready.get() && replied.compareAndSet(false, true)) {
+                            spreadPartitions();
 
-                                onDone(exchId.topologyVersion());
-                            }
-                            else if (log.isDebugEnabled())
-                                log.debug("Exchange future full map is not sent [allReceived=" + allReceived() +
-                                    ", ready=" + ready + ", replied=" + replied.get() + ", init=" + init.get() +
-                                    ", fut=" + this + ']');
+                            onDone(exchId.topologyVersion());
                         }
-                    }
-                    catch (IgniteCheckedException e) {
-                        U.error(log, "Failed to initialize exchange future: " + this, e);
+                        else if (log.isDebugEnabled())
+                            log.debug("Exchange future full map is not sent [allReceived=" + allReceived() +
+                                ", ready=" + ready + ", replied=" + replied.get() + ", init=" + init.get() +
+                                ", fut=" + this + ']');
                     }
                 }
             });
@@ -1244,7 +1246,17 @@ public class GridDhtPartitionsExchangeFuture extends GridFutureAdapter<AffinityT
         assert exchId.topologyVersion().equals(msg.topologyVersion());
 
         initFut.listen(new CI1<IgniteInternalFuture<Boolean>>() {
-            @Override public void apply(IgniteInternalFuture<Boolean> t) {
+            @Override public void apply(IgniteInternalFuture<Boolean> f) {
+                try {
+                    if (!f.get())
+                        return;
+                }
+                catch (IgniteCheckedException e) {
+                    U.error(log, "Failed to initialize exchange future: " + this, e);
+
+                    return;
+                }
+
                 ClusterNode curOldest = oldestNode.get();
 
                 if (!nodeId.equals(curOldest.id())) {
@@ -1333,8 +1345,18 @@ public class GridDhtPartitionsExchangeFuture extends GridFutureAdapter<AffinityT
 
         try {
             // Wait for initialization part of this future to complete.
-            initFut.listen(new CI1<IgniteInternalFuture<?>>() {
-                @Override public void apply(IgniteInternalFuture<?> f) {
+            initFut.listen(new CI1<IgniteInternalFuture<Boolean>>() {
+                @Override public void apply(IgniteInternalFuture<Boolean> f) {
+                    try {
+                        if (!f.get())
+                            return;
+                    }
+                    catch (IgniteCheckedException e) {
+                        U.error(log, "Failed to initialize exchange future: " + this, e);
+
+                        return;
+                    }
+
                     if (isDone())
                         return;
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/a61cf24b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteDynamicCacheAndNodeStop.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteDynamicCacheAndNodeStop.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteDynamicCacheAndNodeStop.java
new file mode 100644
index 0000000..a389e1f
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteDynamicCacheAndNodeStop.java
@@ -0,0 +1,95 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.cache;
+
+import org.apache.ignite.*;
+import org.apache.ignite.configuration.*;
+import org.apache.ignite.internal.*;
+import org.apache.ignite.spi.discovery.tcp.*;
+import org.apache.ignite.spi.discovery.tcp.ipfinder.*;
+import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.*;
+import org.apache.ignite.testframework.*;
+import org.apache.ignite.testframework.junits.common.*;
+
+import java.util.concurrent.*;
+
+/**
+ *
+ */
+public class IgniteDynamicCacheAndNodeStop extends GridCommonAbstractTest {
+    /** */
+    private static TcpDiscoveryIpFinder ipFinder = new TcpDiscoveryVmIpFinder(true);
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(gridName);
+
+        ((TcpDiscoverySpi)cfg.getDiscoverySpi()).setIpFinder(ipFinder);
+
+        return cfg;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTest() throws Exception {
+        super.afterTest();
+
+        stopAllGrids();
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testCacheAndNodeStop() throws Exception {
+        final Ignite ignite = startGrid(0);
+
+        for (int i = 0; i < 3; i++) {
+            log.info("Iteration: " + i);
+
+            startGrid(1);
+
+            final  CacheConfiguration ccfg = new CacheConfiguration();
+
+            ignite.createCache(ccfg);
+
+            final CyclicBarrier barrier = new CyclicBarrier(2);
+
+            IgniteInternalFuture<?> fut1 = GridTestUtils.runAsync(new Callable<Object>() {
+                @Override public Object call() throws Exception {
+                    barrier.await();
+
+                    ignite.destroyCache(null);
+
+                    return null;
+                }
+            });
+
+            IgniteInternalFuture<?> fut2 = GridTestUtils.runAsync(new Callable<Object>() {
+                @Override public Object call() throws Exception {
+                    barrier.await();
+
+                    stopGrid(1);
+
+                    return null;
+                }
+            });
+
+            fut1.get();
+            fut2.get();
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/a61cf24b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite2.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite2.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite2.java
index 495719f..4258733 100644
--- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite2.java
+++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite2.java
@@ -143,6 +143,8 @@ public class IgniteCacheTestSuite2 extends TestSuite {
         suite.addTest(new TestSuite(IgniteCacheEntryProcessorNodeJoinTest.class));
         suite.addTest(new TestSuite(IgniteAtomicCacheEntryProcessorNodeJoinTest.class));
         suite.addTest(new TestSuite(GridCacheNearTxForceKeyTest.class));
+        suite.addTest(new TestSuite(CrossCacheTxRandomOperationsTest.class));
+        suite.addTest(new TestSuite(IgniteDynamicCacheAndNodeStop.class));
 
         return suite;
     }


[28/39] ignite git commit: Merge remote-tracking branch 'remotes/upstream/master' into ignite-1.4

Posted by ag...@apache.org.
Merge remote-tracking branch 'remotes/upstream/master' into ignite-1.4


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

Branch: refs/heads/ignite-264
Commit: 27cd6154747155c77e29d194d98f9ba9bf6a1455
Parents: 662bc33 6ac9557
Author: ptupitsyn <pt...@gridgain.com>
Authored: Thu Sep 3 11:42:30 2015 +0300
Committer: ptupitsyn <pt...@gridgain.com>
Committed: Thu Sep 3 11:42:30 2015 +0300

----------------------------------------------------------------------
 .../configuration/PlatformConfiguration.java    |   2 +-
 .../apache/ignite/internal/IgniteKernal.java    |  14 +-
 .../cache/store/CacheOsStoreManager.java        |  32 +-
 .../cache/store/CacheStoreManager.java          |  10 +
 .../store/GridCacheStoreManagerAdapter.java     |   5 -
 .../platform/PlatformNoopProcessor.java         |  11 +-
 .../processors/platform/PlatformProcessor.java  |  10 +
 .../cache/store/PlatformCacheStore.java         |  25 +
 ...processors.platform.PlatformBootstrapFactory |   2 +
 .../PlatformAbstractConfigurationClosure.java   |  61 +++
 .../platform/PlatformConfigurationEx.java       |  48 ++
 .../platform/PlatformProcessorImpl.java         | 360 ++++++++++++++
 .../cache/store/PlatformCacheStore.java         |  25 -
 .../platform/cpp/PlatformCppBootstrap.java      |  31 ++
 .../cpp/PlatformCppBootstrapFactory.java        |  39 ++
 .../cpp/PlatformCppConfigurationClosure.java    |  99 ++++
 .../cpp/PlatformCppConfigurationEx.java         |  82 +++
 .../dotnet/PlatformDotNetBootstrap.java         |  31 ++
 .../dotnet/PlatformDotNetBootstrapFactory.java  |  39 ++
 .../dotnet/PlatformDotNetCacheStore.java        | 497 +++++++++++++++++++
 .../PlatformDotNetConfigurationClosure.java     | 255 ++++++++++
 .../dotnet/PlatformDotNetConfigurationEx.java   |  91 ++++
 .../platform/utils/PlatformUtils.java           |  14 +
 .../platform/cpp/PlatformCppConfiguration.java  |  47 ++
 .../dotnet/PlatformDotNetCacheStoreFactory.java | 139 ++++++
 .../dotnet/PlatformDotNetConfiguration.java     | 119 +++++
 .../dotnet/PlatformDotNetLifecycleBean.java     | 109 ++++
 .../PlatformDotNetPortableConfiguration.java    | 228 +++++++++
 ...PlatformDotNetPortableTypeConfiguration.java | 248 +++++++++
 parent/pom.xml                                  |   1 +
 30 files changed, 2635 insertions(+), 39 deletions(-)
----------------------------------------------------------------------


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


[39/39] ignite git commit: Merge remote-tracking branch 'upstream/ignite-1.4' into ignite-264

Posted by ag...@apache.org.
Merge remote-tracking branch 'upstream/ignite-1.4' into ignite-264


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

Branch: refs/heads/ignite-264
Commit: 637afb75b36aaf6cd449118cc15c8f897f17c00f
Parents: 8af9e64 77fc969
Author: Alexey Goncharuk <al...@gmail.com>
Authored: Thu Sep 3 14:41:16 2015 -0700
Committer: Alexey Goncharuk <al...@gmail.com>
Committed: Thu Sep 3 14:41:16 2015 -0700

----------------------------------------------------------------------
 examples/config/example-default.xml             |  76 ++
 examples/config/example-ignite.xml              |  56 +-
 .../config/portable/example-ignite-portable.xml |  44 +
 .../ignite/examples/portable/Address.java       |  72 ++
 .../ignite/examples/portable/Employee.java      |  93 +++
 .../ignite/examples/portable/EmployeeKey.java   |  90 +++
 .../portable/ExamplePortableNodeStartup.java    |  36 +
 .../ignite/examples/portable/Organization.java  |  93 +++
 .../examples/portable/OrganizationType.java     |  32 +
 ...mputeClientPortableTaskExecutionExample.java | 154 ++++
 .../portable/computegrid/ComputeClientTask.java | 116 +++
 .../portable/computegrid/package-info.java      |  21 +
 .../CacheClientPortablePutGetExample.java       | 226 ++++++
 .../CacheClientPortableQueryExample.java        | 323 ++++++++
 .../portable/datagrid/package-info.java         |  21 +
 .../ignite/examples/portable/package-info.java  |  21 +
 .../java8/datagrid/CacheAffinityExample.java    |  15 +
 .../java8/messaging/MessagingExample.java       |  11 +-
 .../CacheClientPortableExampleTest.java         |  46 ++
 .../ComputeClientPortableExampleTest.java       |  37 +
 .../testsuites/IgniteExamplesSelfTestSuite.java |   6 +
 .../java8/examples/BasicExamplesSelfTest.java   |  10 +-
 .../java8/examples/CacheExamplesSelfTest.java   |   8 +-
 .../examples/CheckpointExamplesSelfTest.java    |   8 +-
 .../examples/ClusterGroupExampleSelfTest.java   |   4 +-
 .../examples/ContinuationExamplesSelfTest.java  |   8 +-
 .../ContinuousMapperExamplesSelfTest.java       |   8 +-
 .../examples/DeploymentExamplesSelfTest.java    |   6 +-
 .../java8/examples/EventsExamplesSelfTest.java  |   5 +-
 .../HibernateL2CacheExampleSelfTest.java        |   8 +-
 .../java8/examples/IgfsExamplesSelfTest.java    |   6 +-
 .../examples/LifecycleExamplesSelfTest.java     |   8 +-
 .../examples/MemcacheRestExamplesSelfTest.java  |   4 +-
 .../examples/MessagingExamplesSelfTest.java     |   6 +-
 .../examples/MonteCarloExamplesSelfTest.java    |   8 +-
 .../examples/SpringBeanExamplesSelfTest.java    |   8 +-
 .../java8/examples/TaskExamplesSelfTest.java    |   4 +-
 .../IgniteExamplesJ8SelfTestSuite.java          |  12 +-
 .../configuration/CacheConfiguration.java       |  70 +-
 .../ignite/configuration/TopologyValidator.java |   4 +-
 .../apache/ignite/internal/IgniteKernal.java    |  97 ++-
 .../discovery/GridDiscoveryManager.java         |  52 +-
 .../portable/GridPortableMarshaller.java        |  44 +-
 .../portable/PortableAbstractLazyValue.java     |  57 --
 .../internal/portable/PortableBuilderEnum.java  | 114 ---
 .../internal/portable/PortableBuilderImpl.java  | 531 ------------
 .../portable/PortableBuilderReader.java         | 776 ------------------
 .../PortableBuilderSerializationAware.java      |  29 -
 .../portable/PortableBuilderSerializer.java     | 211 -----
 .../portable/PortableClassDescriptor.java       |  57 +-
 .../internal/portable/PortableContext.java      |   6 +-
 .../portable/PortableEnumArrayLazyValue.java    | 112 ---
 .../portable/PortableLazyArrayList.java         | 159 ----
 .../portable/PortableLazyLinkedList.java        | 215 -----
 .../internal/portable/PortableLazyMap.java      | 218 -----
 .../internal/portable/PortableLazyMapEntry.java |  66 --
 .../internal/portable/PortableLazySet.java      |  89 ---
 .../internal/portable/PortableLazyValue.java    |  28 -
 .../portable/PortableObjectArrayLazyValue.java  |  89 ---
 .../portable/PortablePlainLazyValue.java        |  47 --
 .../portable/PortablePlainPortableObject.java   |  50 --
 .../internal/portable/PortableReaderExImpl.java | 154 +++-
 .../ignite/internal/portable/PortableUtils.java |  11 +
 .../portable/PortableValueWithType.java         |  74 --
 .../internal/portable/PortableWriterExImpl.java | 159 +++-
 .../builder/PortableAbstractLazyValue.java      |  57 ++
 .../portable/builder/PortableBuilderEnum.java   | 116 +++
 .../portable/builder/PortableBuilderImpl.java   | 537 +++++++++++++
 .../portable/builder/PortableBuilderReader.java | 800 +++++++++++++++++++
 .../PortableBuilderSerializationAware.java      |  31 +
 .../builder/PortableBuilderSerializer.java      | 214 +++++
 .../builder/PortableEnumArrayLazyValue.java     | 114 +++
 .../portable/builder/PortableLazyArrayList.java | 166 ++++
 .../builder/PortableLazyLinkedList.java         | 217 +++++
 .../portable/builder/PortableLazyMap.java       | 220 +++++
 .../portable/builder/PortableLazyMapEntry.java  |  68 ++
 .../portable/builder/PortableLazySet.java       |  92 +++
 .../portable/builder/PortableLazyValue.java     |  28 +
 .../builder/PortableModifiableLazyValue.java    |  52 ++
 .../builder/PortableObjectArrayLazyValue.java   |  91 +++
 .../builder/PortablePlainLazyValue.java         |  49 ++
 .../builder/PortablePlainPortableObject.java    |  53 ++
 .../portable/builder/PortableValueWithType.java |  75 ++
 .../internal/portable/builder/package-info.java |  22 +
 .../cache/DynamicCacheChangeRequest.java        |  19 +-
 .../cache/DynamicCacheDescriptor.java           |  16 +-
 .../GridCacheLoaderWriterStoreFactory.java      |  20 +-
 .../GridCachePartitionExchangeManager.java      |   4 +-
 .../processors/cache/GridCacheProcessor.java    | 199 +++--
 .../distributed/dht/GridDhtTxPrepareFuture.java |  15 +-
 .../dht/GridPartitionedGetFuture.java           |  19 +-
 .../distributed/near/GridNearGetFuture.java     |  22 +-
 .../CacheObjectPortableProcessorImpl.java       |   2 +-
 .../ignite/internal/util/lang/GridFunc.java     |   1 +
 .../resources/META-INF/classnames.properties    |   2 +-
 .../store/StoreResourceInjectionSelfTest.java   | 104 +++
 .../GridPortableBuilderAdditionalSelfTest.java  | 232 +++++-
 .../portable/GridPortableBuilderSelfTest.java   |   1 +
 .../GridPortableMarshallerSelfTest.java         |  72 +-
 .../GridPortableMetaDataDisabledSelfTest.java   |  17 +
 .../portable/GridPortableMetaDataSelfTest.java  |  17 +
 .../mutabletest/GridPortableTestClasses.java    |  38 +-
 .../cache/CacheAffinityCallSelfTest.java        |  85 +-
 .../IgniteCacheP2pUnmarshallingErrorTest.java   |   5 +
 .../cache/IgniteDynamicCacheStartSelfTest.java  |   4 +-
 ...gniteTopologyValidatorAbstractCacheTest.java |  46 +-
 .../CachePutAllFailoverAbstractTest.java        | 126 ++-
 .../CachePutAllFailoverAtomicTest.java          |  30 +
 ...ridCachePartitionNotLoadedEventSelfTest.java |   6 +-
 ...gniteCachePutRetryTransactionalSelfTest.java |  42 -
 ...ClientNodePortableMetadataMultinodeTest.java |  11 +
 .../junits/common/GridCommonAbstractTest.java   |  41 +
 .../IgniteCacheFailoverTestSuite2.java          |   2 +
 .../ignite/testsuites/IgniteCacheTestSuite.java |   4 +-
 .../ignite/logger/log4j2/Log4J2Logger.java      |  20 +-
 .../ignite/platform/cpp/package-info.java       |  22 +
 .../ignite/platform/dotnet/package-info.java    |  22 +
 .../apache/ignite/platform/package-info.java    |  22 +
 parent/pom.xml                                  |   4 +
 119 files changed, 6141 insertions(+), 3382 deletions(-)
----------------------------------------------------------------------


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

http://git-wip-us.apache.org/repos/asf/ignite/blob/637afb75/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridPartitionedGetFuture.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/ignite/blob/637afb75/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/GridCachePartitionNotLoadedEventSelfTest.java
----------------------------------------------------------------------
diff --cc modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/GridCachePartitionNotLoadedEventSelfTest.java
index f0c0c20,e712cc9..5bc779c
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/GridCachePartitionNotLoadedEventSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/GridCachePartitionNotLoadedEventSelfTest.java
@@@ -142,7 -137,9 +142,9 @@@ public class GridCachePartitionNotLoade
          startGrid(0);
          startGrid(1);
  
+         awaitPartitionMapExchange();
+ 
 -        PartitionNotFullyLoadedListener lsnr = new PartitionNotFullyLoadedListener();
 +        final PartitionNotFullyLoadedListener lsnr = new PartitionNotFullyLoadedListener();
  
          ignite(1).events().localListen(lsnr, EventType.EVT_CACHE_REBALANCE_PART_DATA_LOST);
  


[20/39] ignite git commit: ignite-1273: added ability to modify arrays returned from PortableBuilder and fixed cyclic references processing for arrays and collections in PortableMarshaller

Posted by ag...@apache.org.
http://git-wip-us.apache.org/repos/asf/ignite/blob/9057a4c0/modules/core/src/test/java/org/apache/ignite/internal/portable/GridPortableBuilderAdditionalSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/portable/GridPortableBuilderAdditionalSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/portable/GridPortableBuilderAdditionalSelfTest.java
index 488361c..61ec714 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/portable/GridPortableBuilderAdditionalSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/portable/GridPortableBuilderAdditionalSelfTest.java
@@ -22,6 +22,7 @@ import com.google.common.collect.Lists;
 import com.google.common.collect.Maps;
 import com.google.common.collect.Sets;
 import java.lang.reflect.Field;
+import java.math.BigDecimal;
 import java.sql.Timestamp;
 import java.util.ArrayList;
 import java.util.Arrays;
@@ -38,6 +39,8 @@ import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.IgnitePortables;
 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.PortableBuilderImpl;
 import org.apache.ignite.internal.portable.mutabletest.GridPortableMarshalerAwareTestClass;
 import org.apache.ignite.internal.processors.cache.portable.CacheObjectPortableProcessorImpl;
 import org.apache.ignite.internal.processors.cache.portable.IgnitePortablesImpl;
@@ -237,10 +240,214 @@ public class GridPortableBuilderAdditionalSelfTest extends GridCommonAbstractTes
     /**
      *
      */
-    public void testSimpleArrayModification() {
+    public void testDateArrayModification() {
         TestObjectAllTypes obj = new TestObjectAllTypes();
 
-        obj.strArr = new String[]{"a", "a", "a"};
+        obj.dateArr =  new Date[] {new Date(11111), new Date(11111), new Date(11111)};
+
+        PortableBuilderImpl mutObj = wrap(obj);
+
+        Date[] arr = mutObj.getField("dateArr");
+        arr[0] = new Date(22222);
+
+        TestObjectAllTypes res = mutObj.build().deserialize();
+
+        Assert.assertArrayEquals(new Date[] {new Date(22222), new Date(11111), new Date(11111)}, res.dateArr);
+    }
+
+    /**
+     *
+     */
+    public void testUUIDArrayModification() {
+        TestObjectAllTypes obj = new TestObjectAllTypes();
+
+        obj.uuidArr = new UUID[] {new UUID(1, 1), new UUID(1, 1), new UUID(1, 1)};
+
+        PortableBuilderImpl mutObj = wrap(obj);
+
+        UUID[] arr = mutObj.getField("uuidArr");
+        arr[0] = new UUID(2, 2);
+
+        TestObjectAllTypes res = mutObj.build().deserialize();
+
+        Assert.assertArrayEquals(new UUID[] {new UUID(2, 2), new UUID(1, 1), new UUID(1, 1)}, res.uuidArr);
+    }
+
+    /**
+     *
+     */
+    public void testDecimalArrayModification() {
+        TestObjectAllTypes obj = new TestObjectAllTypes();
+
+        obj.bdArr = new BigDecimal[] {new BigDecimal(1000), new BigDecimal(1000), new BigDecimal(1000)};
+
+        PortableBuilderImpl mutObj = wrap(obj);
+
+        BigDecimal[] arr = mutObj.getField("bdArr");
+        arr[0] = new BigDecimal(2000);
+
+        TestObjectAllTypes res = mutObj.build().deserialize();
+
+        Assert.assertArrayEquals(new BigDecimal[] {new BigDecimal(1000), new BigDecimal(1000), new BigDecimal(1000)},
+            res.bdArr);
+    }
+
+    /**
+     *
+     */
+    public void testBooleanArrayModification() {
+        TestObjectAllTypes obj = new TestObjectAllTypes();
+
+        obj.zArr = new boolean[] {false, false, false};
+
+        PortableBuilderImpl mutObj = wrap(obj);
+
+        boolean[] arr = mutObj.getField("zArr");
+        arr[0] = true;
+
+        TestObjectAllTypes res = mutObj.build().deserialize();
+
+        boolean[] expected = new boolean[] {true, false, false};
+
+        assertEquals(expected.length, res.zArr.length);
+
+        for (int i = 0; i < expected.length; i++)
+            assertEquals(expected[i], res.zArr[i]);
+    }
+
+    /**
+     *
+     */
+    public void testCharArrayModification() {
+        TestObjectAllTypes obj = new TestObjectAllTypes();
+
+        obj.cArr = new char[] {'a', 'a', 'a'};
+
+        PortableBuilderImpl mutObj = wrap(obj);
+
+        char[] arr = mutObj.getField("cArr");
+        arr[0] = 'b';
+
+        TestObjectAllTypes res = mutObj.build().deserialize();
+
+        Assert.assertArrayEquals(new char[] {'b', 'a', 'a'}, res.cArr);
+    }
+
+    /**
+     *
+     */
+    public void testDoubleArrayModification() {
+        TestObjectAllTypes obj = new TestObjectAllTypes();
+
+        obj.dArr = new double[] {1.0, 1.0, 1.0};
+
+        PortableBuilderImpl mutObj = wrap(obj);
+
+        double[] arr = mutObj.getField("dArr");
+        arr[0] = 2.0;
+
+        TestObjectAllTypes res = mutObj.build().deserialize();
+
+        Assert.assertArrayEquals(new double[] {2.0, 1.0, 1.0}, res.dArr, 0);
+    }
+
+    /**
+     *
+     */
+    public void testFloatArrayModification() {
+        TestObjectAllTypes obj = new TestObjectAllTypes();
+
+        obj.fArr = new float[] {1.0f, 1.0f, 1.0f};
+
+        PortableBuilderImpl mutObj = wrap(obj);
+
+        float[] arr = mutObj.getField("fArr");
+        arr[0] = 2.0f;
+
+        TestObjectAllTypes res = mutObj.build().deserialize();
+
+        Assert.assertArrayEquals(new float[] {2.0f, 1.0f, 1.0f}, res.fArr, 0);
+    }
+
+    /**
+     *
+     */
+    public void testLongArrayModification() {
+        TestObjectAllTypes obj = new TestObjectAllTypes();
+
+        obj.lArr = new long[] {1, 1, 1};
+
+        PortableBuilderImpl mutObj = wrap(obj);
+
+        long[] arr = mutObj.getField("lArr");
+        arr[0] = 2;
+
+        TestObjectAllTypes res = mutObj.build().deserialize();
+
+        Assert.assertArrayEquals(new long[] {2, 1, 1}, res.lArr);
+    }
+
+    /**
+     *
+     */
+    public void testIntArrayModification() {
+        TestObjectAllTypes obj = new TestObjectAllTypes();
+
+        obj.iArr = new int[] {1, 1, 1};
+
+        PortableBuilderImpl mutObj = wrap(obj);
+
+        int[] arr = mutObj.getField("iArr");
+        arr[0] = 2;
+
+        TestObjectAllTypes res = mutObj.build().deserialize();
+
+        Assert.assertArrayEquals(new int[] {2, 1, 1}, res.iArr);
+    }
+
+    /**
+     *
+     */
+    public void testShortArrayModification() {
+        TestObjectAllTypes obj = new TestObjectAllTypes();
+
+        obj.sArr = new short[] {1, 1, 1};
+
+        PortableBuilderImpl mutObj = wrap(obj);
+
+        short[] arr = mutObj.getField("sArr");
+        arr[0] = 2;
+
+        TestObjectAllTypes res = mutObj.build().deserialize();
+
+        Assert.assertArrayEquals(new short[] {2, 1, 1}, res.sArr);
+    }
+
+    /**
+     *
+     */
+    public void testByteArrayModification() {
+        TestObjectAllTypes obj = new TestObjectAllTypes();
+
+        obj.bArr = new byte[] {1, 1, 1};
+
+        PortableBuilderImpl mutObj = wrap(obj);
+
+        byte[] arr = mutObj.getField("bArr");
+        arr[0] = 2;
+
+        TestObjectAllTypes res = mutObj.build().deserialize();
+
+        Assert.assertArrayEquals(new byte[] {2, 1, 1}, res.bArr);
+    }
+
+    /**
+     *
+     */
+    public void testStringArrayModification() {
+        TestObjectAllTypes obj = new TestObjectAllTypes();
+
+        obj.strArr = new String[] {"a", "a", "a"};
 
         PortableBuilderImpl mutObj = wrap(obj);
 
@@ -249,29 +456,27 @@ public class GridPortableBuilderAdditionalSelfTest extends GridCommonAbstractTes
 
         TestObjectAllTypes res = mutObj.build().deserialize();
 
-        Assert.assertArrayEquals(obj.strArr, res.strArr);
+        Assert.assertArrayEquals(new String[] {"b", "a", "a"}, res.strArr);
     }
 
     /**
      *
      */
     public void testModifyObjectArray() {
-        fail("https://issues.apache.org/jira/browse/IGNITE-1273");
-
         TestObjectContainer obj = new TestObjectContainer();
-        obj.foo = new Object[]{"a"};
+        obj.foo = new Object[] {"a"};
 
         PortableBuilderImpl mutObj = wrap(obj);
 
         Object[] arr = mutObj.getField("foo");
 
-        Assert.assertArrayEquals(new Object[]{"a"}, arr);
+        Assert.assertArrayEquals(new Object[] {"a"}, arr);
 
         arr[0] = "b";
 
         TestObjectContainer res = mutObj.build().deserialize();
 
-        Assert.assertArrayEquals(new Object[] {"a"}, (Object[])res.foo);
+        Assert.assertArrayEquals(new Object[] {"b"}, (Object[])res.foo);
     }
 
     /**
@@ -280,7 +485,7 @@ public class GridPortableBuilderAdditionalSelfTest extends GridCommonAbstractTes
     public void testOverrideObjectArrayField() {
         PortableBuilderImpl mutObj = wrap(new TestObjectContainer());
 
-        Object[] createdArr = {mutObj, "a", 1, new String[] {"s", "s"}, new byte[]{1, 2}, new UUID(3, 0)};
+        Object[] createdArr = {mutObj, "a", 1, new String[] {"s", "s"}, new byte[] {1, 2}, new UUID(3, 0)};
 
         mutObj.setField("foo", createdArr.clone());
 
@@ -296,7 +501,7 @@ public class GridPortableBuilderAdditionalSelfTest extends GridCommonAbstractTes
      */
     public void testDeepArray() {
         TestObjectContainer obj = new TestObjectContainer();
-        obj.foo = new Object[]{new Object[]{"a", obj}};
+        obj.foo = new Object[] {new Object[] {"a", obj}};
 
         PortableBuilderImpl mutObj = wrap(obj);
 
@@ -633,7 +838,7 @@ public class GridPortableBuilderAdditionalSelfTest extends GridCommonAbstractTes
     public void testEnumArrayModification() {
         TestObjectAllTypes obj = new TestObjectAllTypes();
 
-        obj.enumArr = new TestObjectEnum[]{TestObjectEnum.A, TestObjectEnum.B};
+        obj.enumArr = new TestObjectEnum[] {TestObjectEnum.A, TestObjectEnum.B};
 
         PortableBuilderImpl mutObj = wrap(obj);
 
@@ -953,12 +1158,10 @@ public class GridPortableBuilderAdditionalSelfTest extends GridCommonAbstractTes
      *
      */
     public void testCyclicArrays() {
-        fail("https://issues.apache.org/jira/browse/IGNITE-1273");
-
         TestObjectContainer obj = new TestObjectContainer();
 
         Object[] arr1 = new Object[1];
-        Object[] arr2 = new Object[]{arr1};
+        Object[] arr2 = new Object[] {arr1};
 
         arr1[0] = arr2;
 
@@ -976,7 +1179,6 @@ public class GridPortableBuilderAdditionalSelfTest extends GridCommonAbstractTes
      */
     @SuppressWarnings("TypeMayBeWeakened")
     public void testCyclicArrayList() {
-        fail("https://issues.apache.org/jira/browse/IGNITE-1273");
         TestObjectContainer obj = new TestObjectContainer();
 
         List<Object> arr1 = new ArrayList<>();

http://git-wip-us.apache.org/repos/asf/ignite/blob/9057a4c0/modules/core/src/test/java/org/apache/ignite/internal/portable/GridPortableBuilderSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/portable/GridPortableBuilderSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/portable/GridPortableBuilderSelfTest.java
index 3ba0a92..7f23c1f 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/portable/GridPortableBuilderSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/portable/GridPortableBuilderSelfTest.java
@@ -28,6 +28,7 @@ import java.util.UUID;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.IgnitePortables;
 import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.internal.portable.builder.PortableBuilderImpl;
 import org.apache.ignite.internal.portable.mutabletest.GridPortableTestClasses.TestObjectAllTypes;
 import org.apache.ignite.internal.portable.mutabletest.GridPortableTestClasses.TestObjectContainer;
 import org.apache.ignite.internal.portable.mutabletest.GridPortableTestClasses.TestObjectInner;

http://git-wip-us.apache.org/repos/asf/ignite/blob/9057a4c0/modules/core/src/test/java/org/apache/ignite/internal/portable/GridPortableMarshallerSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/portable/GridPortableMarshallerSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/portable/GridPortableMarshallerSelfTest.java
index 11e316a..c8287a0 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/portable/GridPortableMarshallerSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/portable/GridPortableMarshallerSelfTest.java
@@ -40,6 +40,7 @@ import java.util.concurrent.Callable;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.ConcurrentSkipListSet;
 import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.internal.portable.builder.PortableBuilderImpl;
 import org.apache.ignite.internal.util.GridUnsafe;
 import org.apache.ignite.internal.util.IgniteUtils;
 import org.apache.ignite.internal.util.lang.GridMapEntry;
@@ -68,6 +69,7 @@ import sun.misc.Unsafe;
 
 import static org.apache.ignite.internal.portable.PortableThreadLocalMemoryAllocator.THREAD_LOCAL_ALLOC;
 import static org.junit.Assert.assertArrayEquals;
+import static org.junit.Assert.assertTrue;
 
 /**
  * Portable marshaller tests.
@@ -1547,11 +1549,11 @@ public class GridPortableMarshallerSelfTest extends GridCommonAbstractTest {
 
         PortableObject copy = copy(po, F.<String, Object>asMap("bArr", new byte[]{1, 2, 3}));
 
-        assertArrayEquals(new byte[]{1, 2, 3}, copy.<byte[]>field("bArr"));
+        assertArrayEquals(new byte[] {1, 2, 3}, copy.<byte[]>field("bArr"));
 
         SimpleObject obj0 = copy.deserialize();
 
-        assertArrayEquals(new byte[]{1, 2, 3}, obj0.bArr);
+        assertArrayEquals(new byte[] {1, 2, 3}, obj0.bArr);
     }
 
     /**
@@ -1796,7 +1798,7 @@ public class GridPortableMarshallerSelfTest extends GridCommonAbstractTest {
 
         assertEquals("str555", obj0.str);
         assertEquals(newObj, obj0.inner);
-        assertArrayEquals(new byte[]{6, 7, 9}, obj0.bArr);
+        assertArrayEquals(new byte[] {6, 7, 9}, obj0.bArr);
     }
 
     /**
@@ -1832,7 +1834,7 @@ public class GridPortableMarshallerSelfTest extends GridCommonAbstractTest {
         assertEquals("str555", copy.<String>field("str"));
         assertEquals(newObj, copy.<PortableObject>field("inner").deserialize());
         assertEquals((short)2323, copy.<Short>field("s").shortValue());
-        assertArrayEquals(new byte[]{6, 7, 9}, copy.<byte[]>field("bArr"));
+        assertArrayEquals(new byte[] {6, 7, 9}, copy.<byte[]>field("bArr"));
         assertEquals((byte)111, copy.<Byte>field("b").byteValue());
 
         SimpleObject obj0 = copy.deserialize();
@@ -1841,7 +1843,7 @@ public class GridPortableMarshallerSelfTest extends GridCommonAbstractTest {
         assertEquals("str555", obj0.str);
         assertEquals(newObj, obj0.inner);
         assertEquals((short)2323, obj0.s);
-        assertArrayEquals(new byte[]{6, 7, 9}, obj0.bArr);
+        assertArrayEquals(new byte[] {6, 7, 9}, obj0.bArr);
         assertEquals((byte)111, obj0.b);
     }
 
@@ -2239,6 +2241,53 @@ public class GridPortableMarshallerSelfTest extends GridCommonAbstractTest {
     }
 
     /**
+     * @throws Exception If failed.
+     */
+    public void testCyclicReferencesMarshalling() throws Exception {
+        PortableMarshaller marsh = new PortableMarshaller();
+
+        SimpleObject obj = simpleObject();
+
+        obj.bArr = obj.inner.bArr;
+        obj.cArr = obj.inner.cArr;
+        obj.boolArr = obj.inner.boolArr;
+        obj.sArr = obj.inner.sArr;
+        obj.strArr = obj.inner.strArr;
+        obj.iArr = obj.inner.iArr;
+        obj.lArr = obj.inner.lArr;
+        obj.fArr = obj.inner.fArr;
+        obj.dArr = obj.inner.dArr;
+        obj.dateArr = obj.inner.dateArr;
+        obj.uuidArr = obj.inner.uuidArr;
+        obj.objArr = obj.inner.objArr;
+        obj.bdArr = obj.inner.bdArr;
+        obj.map = obj.inner.map;
+        obj.col = obj.inner.col;
+        obj.mEntry = obj.inner.mEntry;
+
+        SimpleObject res = (SimpleObject)marshalUnmarshal(obj, marsh);
+
+        assertEquals(obj, res);
+
+        assertTrue(res.bArr == res.inner.bArr);
+        assertTrue(res.cArr == res.inner.cArr);
+        assertTrue(res.boolArr == res.inner.boolArr);
+        assertTrue(res.sArr == res.inner.sArr);
+        assertTrue(res.strArr == res.inner.strArr);
+        assertTrue(res.iArr == res.inner.iArr);
+        assertTrue(res.lArr == res.inner.lArr);
+        assertTrue(res.fArr == res.inner.fArr);
+        assertTrue(res.dArr == res.inner.dArr);
+        assertTrue(res.dateArr == res.inner.dateArr);
+        assertTrue(res.uuidArr == res.inner.uuidArr);
+        assertTrue(res.objArr == res.inner.objArr);
+        assertTrue(res.bdArr == res.inner.bdArr);
+        assertTrue(res.map == res.inner.map);
+        assertTrue(res.col == res.inner.col);
+        assertTrue(res.mEntry == res.inner.mEntry);
+    }
+
+    /**
      *
      */
     private static class ObjectWithClassFields {
@@ -2424,6 +2473,7 @@ public class GridPortableMarshallerSelfTest extends GridCommonAbstractTest {
         inner.map = new HashMap<>();
         inner.enumVal = TestEnum.A;
         inner.enumArr = new TestEnum[] {TestEnum.A, TestEnum.B};
+        inner.bdArr = new BigDecimal[] {new BigDecimal(1000), BigDecimal.ONE};
 
         inner.col.add("str1");
         inner.col.add("str2");
@@ -2433,6 +2483,8 @@ public class GridPortableMarshallerSelfTest extends GridCommonAbstractTest {
         inner.map.put(2, "str2");
         inner.map.put(3, "str3");
 
+        inner.mEntry = inner.map.entrySet().iterator().next();
+
         SimpleObject outer = new SimpleObject();
 
         outer.b = 2;
@@ -2464,6 +2516,8 @@ public class GridPortableMarshallerSelfTest extends GridCommonAbstractTest {
         outer.enumVal = TestEnum.B;
         outer.enumArr = new TestEnum[] {TestEnum.B, TestEnum.C};
         outer.inner = inner;
+        outer.bdArr = new BigDecimal[] {new BigDecimal(5000), BigDecimal.TEN};
+
 
         outer.col.add("str4");
         outer.col.add("str5");
@@ -2473,6 +2527,8 @@ public class GridPortableMarshallerSelfTest extends GridCommonAbstractTest {
         outer.map.put(5, "str5");
         outer.map.put(6, "str6");
 
+        outer.mEntry = outer.map.entrySet().iterator().next();
+
         return outer;
     }
 
@@ -2757,6 +2813,9 @@ public class GridPortableMarshallerSelfTest extends GridCommonAbstractTest {
         private Object[] objArr;
 
         /** */
+        private BigDecimal[] bdArr;
+
+        /** */
         private Collection<String> col;
 
         /** */
@@ -2769,6 +2828,9 @@ public class GridPortableMarshallerSelfTest extends GridCommonAbstractTest {
         private TestEnum[] enumArr;
 
         /** */
+        private Map.Entry<Integer, String> mEntry;
+
+        /** */
         private SimpleObject inner;
 
         /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/9057a4c0/modules/core/src/test/java/org/apache/ignite/internal/portable/GridPortableMetaDataDisabledSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/portable/GridPortableMetaDataDisabledSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/portable/GridPortableMetaDataDisabledSelfTest.java
index d329818..05df23b 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/portable/GridPortableMetaDataDisabledSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/portable/GridPortableMetaDataDisabledSelfTest.java
@@ -20,6 +20,7 @@ import java.util.Arrays;
 import org.apache.ignite.IgnitePortables;
 import org.apache.ignite.configuration.IgniteConfiguration;
 import org.apache.ignite.marshaller.portable.PortableMarshaller;
+import org.apache.ignite.portable.PortableBuilder;
 import org.apache.ignite.portable.PortableException;
 import org.apache.ignite.portable.PortableMarshalAware;
 import org.apache.ignite.portable.PortableReader;
@@ -68,9 +69,17 @@ public class GridPortableMetaDataDisabledSelfTest extends GridCommonAbstractTest
 
             portables().toPortable(new TestObject1());
             portables().toPortable(new TestObject2());
+            portables().toPortable(new TestObject3());
 
             assertEquals(0, portables().metadata(TestObject1.class).fields().size());
             assertEquals(0, portables().metadata(TestObject2.class).fields().size());
+
+            PortableBuilder bldr = portables().builder("FakeType");
+
+            bldr.setField("field1", 0).setField("field2", "value").build();
+
+            assertNull(portables().metadata("FakeType"));
+            assertNull(portables().metadata(TestObject3.class));
         }
         finally {
             stopGrid();
@@ -218,4 +227,12 @@ public class GridPortableMetaDataDisabledSelfTest extends GridCommonAbstractTest
             // No-op.
         }
     }
+
+    /**
+     */
+    @SuppressWarnings("UnusedDeclaration")
+    private static class TestObject3 {
+        /** */
+        private int field;
+    }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/9057a4c0/modules/core/src/test/java/org/apache/ignite/internal/portable/GridPortableMetaDataSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/portable/GridPortableMetaDataSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/portable/GridPortableMetaDataSelfTest.java
index f6d7627..9054297 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/portable/GridPortableMetaDataSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/portable/GridPortableMetaDataSelfTest.java
@@ -147,6 +147,15 @@ public class GridPortableMetaDataSelfTest extends GridCommonAbstractTest {
     /**
      * @throws Exception If failed.
      */
+    public void testNoConfiguration() throws Exception {
+        portables().toPortable(new TestObject3());
+
+        assertNotNull(portables().metadata(TestObject3.class));
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
     public void testReflection() throws Exception {
         PortableMetadata meta = portables().metadata(TestObject1.class);
 
@@ -349,4 +358,12 @@ public class GridPortableMetaDataSelfTest extends GridCommonAbstractTest {
             // No-op.
         }
     }
+
+    /**
+     */
+    @SuppressWarnings("UnusedDeclaration")
+    private static class TestObject3 {
+        /** */
+        private int intVal;
+    }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/9057a4c0/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 ce97a8d..e49514b 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
@@ -23,6 +23,7 @@ import java.io.IOException;
 import java.io.ObjectOutput;
 import java.io.ObjectOutputStream;
 import java.io.Serializable;
+import java.math.BigDecimal;
 import java.util.ArrayList;
 import java.util.Date;
 import java.util.List;
@@ -182,7 +183,6 @@ public class GridPortableTestClasses {
         /** */
         public Date date;
 
-
         /** */
         public byte[] bArr;
 
@@ -208,12 +208,18 @@ public class GridPortableTestClasses {
         public boolean[] zArr;
 
         /** */
+        public BigDecimal[] bdArr;
+
+        /** */
         public String[] strArr;
 
         /** */
         public UUID[] uuidArr;
 
         /** */
+        public Date[] dateArr;
+
+        /** */
         public TestObjectEnum anEnum;
 
         /** */
@@ -222,8 +228,6 @@ public class GridPortableTestClasses {
         /** */
         public Map.Entry entry;
 
-        //public Date[] dateArr; // todo test date array.
-
         /**
          * @return Array.
          */
@@ -270,21 +274,23 @@ public class GridPortableTestClasses {
             uuid = new UUID(1, 1);
             date = new Date(1000000);
 
-            bArr = new byte[]{1, 2, 3};
-            sArr = new short[]{1, 2, 3};
-            iArr = new int[]{1, 2, 3};
-            lArr = new long[]{1, 2, 3};
-            fArr = new float[]{1, 2, 3};
-            dArr = new double[]{1, 2, 3};
-            cArr = new char[]{1, 2, 3};
-            zArr = new boolean[]{true, false};
+            bArr = new byte[] {1, 2, 3};
+            sArr = new short[] {1, 2, 3};
+            iArr = new int[] {1, 2, 3};
+            lArr = new long[] {1, 2, 3};
+            fArr = new float[] {1, 2, 3};
+            dArr = new double[] {1, 2, 3};
+            cArr = new char[] {1, 2, 3};
+            zArr = new boolean[] {true, false};
 
-            strArr = new String[]{"abc", "ab", "a"};
-            uuidArr = new UUID[]{new UUID(1, 1), new UUID(2, 2)};
+            strArr = new String[] {"abc", "ab", "a"};
+            uuidArr = new UUID[] {new UUID(1, 1), new UUID(2, 2)};
+            bdArr = new BigDecimal[] {new BigDecimal(1000), BigDecimal.TEN};
+            dateArr = new Date[] {new Date(1000000), new Date(200000)};
 
             anEnum = TestObjectEnum.A;
 
-            enumArr = new TestObjectEnum[]{TestObjectEnum.B};
+            enumArr = new TestObjectEnum[] {TestObjectEnum.B};
 
             entry = new GridMapEntry<>(1, "a");
         }
@@ -388,7 +394,6 @@ public class GridPortableTestClasses {
         private Map<String, List<Company>> companyByStreet = new TreeMap<>();
 
         /**
-         *
          * @param street Street.
          * @return Company.
          */
@@ -397,7 +402,6 @@ public class GridPortableTestClasses {
         }
 
         /**
-         *
          * @param company Company.
          */
         public void addCompany(Company company) {
@@ -413,7 +417,6 @@ public class GridPortableTestClasses {
         }
 
         /**
-         *
          * @return map
          */
         public Map<String, List<Company>> getCompanyByStreet() {
@@ -421,7 +424,6 @@ public class GridPortableTestClasses {
         }
 
         /**
-         *
          * @param companyByStreet map
          */
         public void setCompanyByStreet(Map<String, List<Company>> companyByStreet) {

http://git-wip-us.apache.org/repos/asf/ignite/blob/9057a4c0/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/GridCacheClientNodePortableMetadataMultinodeTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/GridCacheClientNodePortableMetadataMultinodeTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/GridCacheClientNodePortableMetadataMultinodeTest.java
index aa67574..1ba3d4d 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/GridCacheClientNodePortableMetadataMultinodeTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/GridCacheClientNodePortableMetadataMultinodeTest.java
@@ -31,6 +31,7 @@ import org.apache.ignite.IgnitePortables;
 import org.apache.ignite.configuration.CacheConfiguration;
 import org.apache.ignite.configuration.IgniteConfiguration;
 import org.apache.ignite.internal.IgniteInternalFuture;
+import org.apache.ignite.internal.util.lang.GridAbsPredicate;
 import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.marshaller.portable.PortableMarshaller;
 import org.apache.ignite.portable.PortableBuilder;
@@ -233,6 +234,16 @@ public class GridCacheClientNodePortableMetadataMultinodeTest extends GridCommon
 
             portables = ignite(i).portables();
 
+            final IgnitePortables p0 = portables;
+
+            GridTestUtils.waitForCondition(new GridAbsPredicate() {
+                @Override public boolean apply() {
+                    Collection<PortableMetadata> metaCol = p0.metadata();
+
+                    return metaCol.size() == 1000;
+                }
+            }, getTestTimeout());
+
             Collection<PortableMetadata> metaCol = portables.metadata();
 
             assertEquals(1000, metaCol.size());


[30/39] ignite git commit: Fixing platform JavaDocs.

Posted by ag...@apache.org.
Fixing platform JavaDocs.


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

Branch: refs/heads/ignite-264
Commit: 5463df29a5e7c5d2ad7e583fc77d5c1ed38cc5f2
Parents: 27cd615
Author: vozerov-gridgain <vo...@gridgain.com>
Authored: Thu Sep 3 14:28:07 2015 +0300
Committer: vozerov-gridgain <vo...@gridgain.com>
Committed: Thu Sep 3 14:28:07 2015 +0300

----------------------------------------------------------------------
 .../ignite/platform/cpp/package-info.java       | 22 ++++++++++++++++++++
 .../ignite/platform/dotnet/package-info.java    | 22 ++++++++++++++++++++
 .../apache/ignite/platform/package-info.java    | 22 ++++++++++++++++++++
 parent/pom.xml                                  |  4 ++++
 4 files changed, 70 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/5463df29/modules/platform/src/main/java/org/apache/ignite/platform/cpp/package-info.java
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/java/org/apache/ignite/platform/cpp/package-info.java b/modules/platform/src/main/java/org/apache/ignite/platform/cpp/package-info.java
new file mode 100644
index 0000000..6d745a7
--- /dev/null
+++ b/modules/platform/src/main/java/org/apache/ignite/platform/cpp/package-info.java
@@ -0,0 +1,22 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+/**
+ * <!-- Package description. -->
+ * Contains C++ platform-related classes.
+ */
+package org.apache.ignite.platform.cpp;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/5463df29/modules/platform/src/main/java/org/apache/ignite/platform/dotnet/package-info.java
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/java/org/apache/ignite/platform/dotnet/package-info.java b/modules/platform/src/main/java/org/apache/ignite/platform/dotnet/package-info.java
new file mode 100644
index 0000000..0765e01
--- /dev/null
+++ b/modules/platform/src/main/java/org/apache/ignite/platform/dotnet/package-info.java
@@ -0,0 +1,22 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+/**
+ * <!-- Package description. -->
+ * Contains .Net platform-related classes.
+ */
+package org.apache.ignite.platform.dotnet;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/5463df29/modules/platform/src/main/java/org/apache/ignite/platform/package-info.java
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/java/org/apache/ignite/platform/package-info.java b/modules/platform/src/main/java/org/apache/ignite/platform/package-info.java
new file mode 100644
index 0000000..ac89b58
--- /dev/null
+++ b/modules/platform/src/main/java/org/apache/ignite/platform/package-info.java
@@ -0,0 +1,22 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+/**
+ * <!-- Package description. -->
+ * Contains platform-related classes.
+ */
+package org.apache.ignite.platform;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/5463df29/parent/pom.xml
----------------------------------------------------------------------
diff --git a/parent/pom.xml b/parent/pom.xml
index ebe691f..eba7390 100644
--- a/parent/pom.xml
+++ b/parent/pom.xml
@@ -339,6 +339,10 @@
                                 <packages>org.apache.ignite.yarn*</packages>
                             </group>
                             <group>
+                                <title>Platforms</title>
+                                <packages>org.apache.ignite.platform*</packages>
+                            </group>
+                            <group>
                                 <title>Spark Integration</title>
                                 <packages>org.apache.ignite.spark.examples.java</packages>
                             </group>


[19/39] ignite git commit: ignite-1273: fixed cyclic references processing by PortableMarshaller and ability to modify array fields returned by PortableBuilder

Posted by ag...@apache.org.
ignite-1273: fixed cyclic references processing by PortableMarshaller and ability to modify array fields returned by PortableBuilder


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

Branch: refs/heads/ignite-264
Commit: 6ac9557ef71b7745eff90bcf15a4f7a958fcfa6b
Parents: 8bd8644
Author: Denis Magda <dm...@gridgain.com>
Authored: Thu Sep 3 08:30:36 2015 +0300
Committer: Denis Magda <dm...@gridgain.com>
Committed: Thu Sep 3 08:30:36 2015 +0300

----------------------------------------------------------------------
 .../portable/GridPortableMarshaller.java        |  44 +-
 .../portable/PortableAbstractLazyValue.java     |  57 --
 .../internal/portable/PortableBuilderEnum.java  | 114 ---
 .../internal/portable/PortableBuilderImpl.java  | 531 ------------
 .../portable/PortableBuilderReader.java         | 776 ------------------
 .../PortableBuilderSerializationAware.java      |  29 -
 .../portable/PortableBuilderSerializer.java     | 211 -----
 .../portable/PortableClassDescriptor.java       |  57 +-
 .../internal/portable/PortableContext.java      |   6 +-
 .../portable/PortableEnumArrayLazyValue.java    | 112 ---
 .../portable/PortableLazyArrayList.java         | 159 ----
 .../portable/PortableLazyLinkedList.java        | 215 -----
 .../internal/portable/PortableLazyMap.java      | 218 -----
 .../internal/portable/PortableLazyMapEntry.java |  66 --
 .../internal/portable/PortableLazySet.java      |  89 ---
 .../internal/portable/PortableLazyValue.java    |  28 -
 .../portable/PortableObjectArrayLazyValue.java  |  89 ---
 .../portable/PortablePlainLazyValue.java        |  47 --
 .../portable/PortablePlainPortableObject.java   |  50 --
 .../internal/portable/PortableReaderExImpl.java | 154 +++-
 .../ignite/internal/portable/PortableUtils.java |  11 +
 .../portable/PortableValueWithType.java         |  74 --
 .../internal/portable/PortableWriterExImpl.java | 159 +++-
 .../builder/PortableAbstractLazyValue.java      |  57 ++
 .../portable/builder/PortableBuilderEnum.java   | 116 +++
 .../portable/builder/PortableBuilderImpl.java   | 537 +++++++++++++
 .../portable/builder/PortableBuilderReader.java | 800 +++++++++++++++++++
 .../PortableBuilderSerializationAware.java      |  31 +
 .../builder/PortableBuilderSerializer.java      | 214 +++++
 .../builder/PortableEnumArrayLazyValue.java     | 114 +++
 .../portable/builder/PortableLazyArrayList.java | 166 ++++
 .../builder/PortableLazyLinkedList.java         | 217 +++++
 .../portable/builder/PortableLazyMap.java       | 220 +++++
 .../portable/builder/PortableLazyMapEntry.java  |  68 ++
 .../portable/builder/PortableLazySet.java       |  92 +++
 .../portable/builder/PortableLazyValue.java     |  28 +
 .../builder/PortableModifiableLazyValue.java    |  52 ++
 .../builder/PortableObjectArrayLazyValue.java   |  91 +++
 .../builder/PortablePlainLazyValue.java         |  49 ++
 .../builder/PortablePlainPortableObject.java    |  53 ++
 .../portable/builder/PortableValueWithType.java |  75 ++
 .../internal/portable/builder/package-info.java |  22 +
 .../CacheObjectPortableProcessorImpl.java       |   2 +-
 .../resources/META-INF/classnames.properties    |   2 +-
 .../GridPortableBuilderAdditionalSelfTest.java  | 232 +++++-
 .../portable/GridPortableBuilderSelfTest.java   |   1 +
 .../GridPortableMarshallerSelfTest.java         |  72 +-
 .../GridPortableMetaDataDisabledSelfTest.java   |  17 +
 .../portable/GridPortableMetaDataSelfTest.java  |  17 +
 .../mutabletest/GridPortableTestClasses.java    |  38 +-
 ...ClientNodePortableMetadataMultinodeTest.java |  11 +
 51 files changed, 3687 insertions(+), 3003 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/6ac9557e/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 3b2357e..c7a9e6f 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
@@ -142,67 +142,67 @@ public class GridPortableMarshaller {
     public static final byte OBJ = (byte)103;
 
     /** */
-    static final byte USER_SET = -1;
+    public static final byte USER_SET = -1;
 
     /** */
-    static final byte USER_COL = 0;
+    public static final byte USER_COL = 0;
 
     /** */
-    static final byte ARR_LIST = 1;
+    public static final byte ARR_LIST = 1;
 
     /** */
-    static final byte LINKED_LIST = 2;
+    public static final byte LINKED_LIST = 2;
 
     /** */
-    static final byte HASH_SET = 3;
+    public static final byte HASH_SET = 3;
 
     /** */
-    static final byte LINKED_HASH_SET = 4;
+    public static final byte LINKED_HASH_SET = 4;
 
     /** */
-    static final byte TREE_SET = 5;
+    public static final byte TREE_SET = 5;
 
     /** */
-    static final byte CONC_SKIP_LIST_SET = 6;
+    public static final byte CONC_SKIP_LIST_SET = 6;
 
     /** */
-    static final byte HASH_MAP = 1;
+    public static final byte HASH_MAP = 1;
 
     /** */
-    static final byte LINKED_HASH_MAP = 2;
+    public static final byte LINKED_HASH_MAP = 2;
 
     /** */
-    static final byte TREE_MAP = 3;
+    public static final byte TREE_MAP = 3;
 
     /** */
-    static final byte CONC_HASH_MAP = 4;
+    public static final byte CONC_HASH_MAP = 4;
 
     /** */
-    static final byte PROPERTIES_MAP = 5;
+    public static final byte PROPERTIES_MAP = 5;
 
     /** */
-    static final int OBJECT_TYPE_ID = -1;
+    public static final int OBJECT_TYPE_ID = -1;
 
     /** */
-    static final int UNREGISTERED_TYPE_ID = 0;
+    public static final int UNREGISTERED_TYPE_ID = 0;
 
     /** */
-    static final int TYPE_ID_POS = 2;
+    public static final int TYPE_ID_POS = 2;
 
     /** */
-    static final int HASH_CODE_POS = 6;
+    public static final int HASH_CODE_POS = 6;
 
     /** */
-    static final int TOTAL_LEN_POS = 10;
+    public static final int TOTAL_LEN_POS = 10;
 
     /** */
-    static final byte RAW_DATA_OFF_POS = 14;
+    public static final byte RAW_DATA_OFF_POS = 14;
 
     /** */
-    static final int CLS_NAME_POS = 18;
+    public static final int CLS_NAME_POS = 18;
 
     /** */
-    static final byte DFLT_HDR_LEN = 18;
+    public static final byte DFLT_HDR_LEN = 18;
 
     /** */
     private final PortableContext ctx;
@@ -301,4 +301,4 @@ public class GridPortableMarshaller {
     public PortableContext context() {
         return ctx;
     }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/6ac9557e/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableAbstractLazyValue.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableAbstractLazyValue.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableAbstractLazyValue.java
deleted file mode 100644
index 2b1c4b7..0000000
--- a/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableAbstractLazyValue.java
+++ /dev/null
@@ -1,57 +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;
-
-/**
- *
- */
-abstract class PortableAbstractLazyValue implements PortableLazyValue {
-    /** */
-    protected Object val;
-
-    /** */
-    protected final PortableBuilderReader reader;
-
-    /** */
-    protected final int valOff;
-
-    /**
-     * @param reader Reader.
-     * @param valOff Value.
-     */
-    protected PortableAbstractLazyValue(PortableBuilderReader reader, int valOff) {
-        this.reader = reader;
-        this.valOff = valOff;
-    }
-
-    /**
-     * @return Value.
-     */
-    protected abstract Object init();
-
-    /** {@inheritDoc} */
-    @Override public Object value() {
-        if (val == null) {
-            val = init();
-
-            assert val != null;
-        }
-
-        return val;
-    }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/6ac9557e/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableBuilderEnum.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableBuilderEnum.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableBuilderEnum.java
deleted file mode 100644
index b6ace99..0000000
--- a/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableBuilderEnum.java
+++ /dev/null
@@ -1,114 +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;
-
-import org.apache.ignite.internal.util.typedef.internal.U;
-import org.apache.ignite.portable.PortableInvalidClassException;
-
-/**
- *
- */
-public class PortableBuilderEnum implements PortableBuilderSerializationAware {
-    /** */
-    private final int ordinal;
-
-    /** */
-    private final int typeId;
-
-    /** */
-    private final String clsName;
-
-    /**
-     * @param typeId Type ID.
-     * @param anEnum Enum instance.
-     */
-    public PortableBuilderEnum(int typeId, Enum anEnum) {
-        ordinal = anEnum.ordinal();
-        this.typeId = typeId;
-        clsName = null;
-    }
-
-    /**
-     * @param reader PortableBuilderReader.
-     */
-    public PortableBuilderEnum(PortableBuilderReader reader) {
-        int typeId = reader.readInt();
-
-        if (typeId == GridPortableMarshaller.UNREGISTERED_TYPE_ID) {
-            clsName = reader.readString();
-
-            Class cls;
-
-            try {
-                // TODO: IGNITE-1272 - Is class loader needed here?
-                cls = U.forName(reader.readString(), null);
-            }
-            catch (ClassNotFoundException e) {
-                throw new PortableInvalidClassException("Failed to load the class: " + clsName, e);
-            }
-
-            this.typeId = reader.portableContext().descriptorForClass(cls).typeId();
-        }
-        else {
-            this.typeId = typeId;
-            this.clsName = null;
-        }
-
-        ordinal = reader.readInt();
-    }
-
-    /**
-     * @return Ordinal.
-     */
-    public int getOrdinal() {
-        return ordinal;
-    }
-
-    /** {@inheritDoc} */
-    @Override public void writeTo(PortableWriterExImpl writer, PortableBuilderSerializer ctx) {
-        writer.writeByte(GridPortableMarshaller.ENUM);
-
-        if (typeId == GridPortableMarshaller.UNREGISTERED_TYPE_ID) {
-            writer.writeInt(GridPortableMarshaller.UNREGISTERED_TYPE_ID);
-            writer.writeString(clsName);
-        }
-        else
-            writer.writeInt(typeId);
-
-        writer.writeInt(ordinal);
-    }
-
-    /** {@inheritDoc} */
-    @Override public boolean equals(Object o) {
-        if (o == null || getClass() != o.getClass())
-            return false;
-
-        PortableBuilderEnum that = (PortableBuilderEnum)o;
-
-        return ordinal == that.ordinal && typeId == that.typeId;
-    }
-
-    /** {@inheritDoc} */
-    @Override public int hashCode() {
-        int result = ordinal;
-
-        result = 31 * result + typeId;
-
-        return result;
-    }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/6ac9557e/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableBuilderImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableBuilderImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableBuilderImpl.java
deleted file mode 100644
index dac199a..0000000
--- a/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableBuilderImpl.java
+++ /dev/null
@@ -1,531 +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;
-
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.LinkedHashMap;
-import java.util.Map;
-import java.util.Set;
-import org.apache.ignite.internal.processors.cache.portable.CacheObjectPortableProcessorImpl;
-import org.apache.ignite.internal.util.GridArgumentCheck;
-import org.apache.ignite.internal.util.typedef.internal.U;
-import org.apache.ignite.portable.PortableBuilder;
-import org.apache.ignite.portable.PortableException;
-import org.apache.ignite.portable.PortableInvalidClassException;
-import org.apache.ignite.portable.PortableMetadata;
-import org.apache.ignite.portable.PortableObject;
-import org.jetbrains.annotations.Nullable;
-
-import static org.apache.ignite.internal.portable.GridPortableMarshaller.CLS_NAME_POS;
-import static org.apache.ignite.internal.portable.GridPortableMarshaller.DFLT_HDR_LEN;
-import static org.apache.ignite.internal.portable.GridPortableMarshaller.HASH_CODE_POS;
-import static org.apache.ignite.internal.portable.GridPortableMarshaller.RAW_DATA_OFF_POS;
-import static org.apache.ignite.internal.portable.GridPortableMarshaller.TOTAL_LEN_POS;
-import static org.apache.ignite.internal.portable.GridPortableMarshaller.TYPE_ID_POS;
-import static org.apache.ignite.internal.portable.GridPortableMarshaller.UNREGISTERED_TYPE_ID;
-
-/**
- *
- */
-public class PortableBuilderImpl implements PortableBuilder {
-    /** */
-    private static final Object REMOVED_FIELD_MARKER = new Object();
-
-    /** */
-    private final PortableContext ctx;
-
-    /** */
-    private final int typeId;
-
-    /** May be null. */
-    private String typeName;
-
-    /** May be null. */
-    private String clsNameToWrite;
-
-    /** */
-    private boolean registeredType = true;
-
-    /** */
-    private Map<String, Object> assignedVals;
-
-    /** */
-    private Map<Integer, Object> readCache;
-
-    /** Position of object in source array, or -1 if object is not created from PortableObject. */
-    private final int start;
-
-    /** Total header length */
-    private final int hdrLen;
-
-    /**
-     * Context of PortableObject reading process. Or {@code null} if object is not created from PortableObject.
-     */
-    private final PortableBuilderReader reader;
-
-    /** */
-    private int hashCode;
-
-    /**
-     * @param clsName Class name.
-     * @param ctx Portable context.
-     */
-    public PortableBuilderImpl(PortableContext ctx, String clsName) {
-        this(ctx, ctx.typeId(clsName), PortableContext.typeName(clsName));
-    }
-
-    /**
-     * @param typeId Type ID.
-     * @param ctx Portable context.
-     */
-    public PortableBuilderImpl(PortableContext ctx, int typeId) {
-        this(ctx, typeId, null);
-    }
-
-    /**
-     * @param typeName Type name.
-     * @param ctx Context.
-     * @param typeId Type id.
-     */
-    public PortableBuilderImpl(PortableContext ctx, int typeId, String typeName) {
-        this.typeId = typeId;
-        this.typeName = typeName;
-        this.ctx = ctx;
-
-        start = -1;
-        reader = null;
-        hdrLen = DFLT_HDR_LEN;
-
-        readCache = Collections.emptyMap();
-    }
-
-    /**
-     * @param obj Object to wrap.
-     */
-    public PortableBuilderImpl(PortableObjectImpl obj) {
-        this(new PortableBuilderReader(obj), obj.start());
-
-        reader.registerObject(this);
-    }
-
-    /**
-     * @param reader ctx
-     * @param start Start.
-     */
-    PortableBuilderImpl(PortableBuilderReader reader, int start) {
-        this.reader = reader;
-        this.start = start;
-
-        int typeId = reader.readIntAbsolute(start + TYPE_ID_POS);
-        ctx = reader.portableContext();
-        hashCode = reader.readIntAbsolute(start + HASH_CODE_POS);
-
-        if (typeId == UNREGISTERED_TYPE_ID) {
-            int mark = reader.position();
-
-            reader.position(start + CLS_NAME_POS);
-
-            clsNameToWrite = reader.readString();
-
-            Class cls;
-
-            try {
-                // TODO: IGNITE-1272 - Is class loader needed here?
-                cls = U.forName(clsNameToWrite, null);
-            }
-            catch (ClassNotFoundException e) {
-                throw new PortableInvalidClassException("Failed to load the class: " + clsNameToWrite, e);
-            }
-
-            this.typeId = ctx.descriptorForClass(cls).typeId();
-
-            registeredType = false;
-
-            hdrLen = reader.position() - mark;
-
-            reader.position(mark);
-        }
-        else {
-            this.typeId = typeId;
-            hdrLen = DFLT_HDR_LEN;
-        }
-    }
-
-    /** {@inheritDoc} */
-    @Override public PortableObject build() {
-        try (PortableWriterExImpl writer = new PortableWriterExImpl(ctx, 0, typeId, false)) {
-
-            PortableBuilderSerializer serializationCtx = new PortableBuilderSerializer();
-
-            serializationCtx.registerObjectWriting(this, 0);
-
-            serializeTo(writer, serializationCtx);
-
-            byte[] arr = writer.array();
-
-            return new PortableObjectImpl(ctx, arr, 0);
-        }
-    }
-
-    /**
-     * @param writer Writer.
-     * @param serializer Serializer.
-     */
-    void serializeTo(PortableWriterExImpl writer, PortableBuilderSerializer serializer) {
-        writer.doWriteByte(GridPortableMarshaller.OBJ);
-        writer.doWriteBoolean(true);
-        writer.doWriteInt(registeredType ? typeId : UNREGISTERED_TYPE_ID);
-        writer.doWriteInt(hashCode);
-
-        // Length and raw offset.
-        writer.reserve(8);
-
-        if (!registeredType)
-            writer.writeString(clsNameToWrite);
-
-
-        Set<Integer> remainsFlds = null;
-
-        if (reader != null) {
-            Map<Integer, Object> assignedFldsById;
-
-            if (assignedVals != null) {
-                assignedFldsById = U.newHashMap(assignedVals.size());
-
-                for (Map.Entry<String, Object> entry : assignedVals.entrySet()) {
-                    int fldId = ctx.fieldId(typeId, entry.getKey());
-
-                    assignedFldsById.put(fldId, entry.getValue());
-                }
-
-                remainsFlds = assignedFldsById.keySet();
-            }
-            else
-                assignedFldsById = Collections.emptyMap();
-
-            int rawOff = start + reader.readIntAbsolute(start + RAW_DATA_OFF_POS);
-
-            reader.position(start + hdrLen);
-
-            int cpStart = -1;
-
-            while (reader.position() < rawOff) {
-                int fldId = reader.readInt();
-
-                int len = reader.readInt();
-
-                if (assignedFldsById.containsKey(fldId)) {
-                    if (cpStart >= 0) {
-                        writer.write(reader.array(), cpStart, reader.position() - 4 - 4 - cpStart);
-
-                        cpStart = -1;
-                    }
-
-                    Object assignedVal = assignedFldsById.remove(fldId);
-
-                    reader.skip(len);
-
-                    if (assignedVal != REMOVED_FIELD_MARKER) {
-                        writer.writeInt(fldId);
-
-                        int lenPos = writer.reserveAndMark(4);
-
-                        serializer.writeValue(writer, assignedVal);
-
-                        writer.writeDelta(lenPos);
-                    }
-                }
-                else {
-                    if (len != 0 && PortableUtils.isPlainType(reader.readByte(0))) {
-                        if (cpStart < 0)
-                            cpStart = reader.position() - 4 - 4;
-
-                        reader.skip(len);
-                    }
-                    else {
-                        if (cpStart >= 0) {
-                            writer.write(reader.array(), cpStart, reader.position() - 4 - cpStart);
-
-                            cpStart = -1;
-                        }
-                        else
-                            writer.writeInt(fldId);
-
-                        Object val;
-
-                        if (len == 0)
-                            val = null;
-                        else if (readCache == null) {
-                            int savedPos = reader.position();
-
-                            val = reader.parseValue();
-
-                            assert reader.position() == savedPos + len;
-                        }
-                        else {
-                            val = readCache.get(fldId);
-
-                            reader.skip(len);
-                        }
-
-                        int lenPos = writer.reserveAndMark(4);
-
-                        serializer.writeValue(writer, val);
-
-                        writer.writeDelta(lenPos);
-                    }
-                }
-            }
-
-            if (cpStart >= 0)
-                writer.write(reader.array(), cpStart, reader.position() - cpStart);
-        }
-
-        if (assignedVals != null && (remainsFlds == null || !remainsFlds.isEmpty())) {
-            boolean metadataEnabled = ctx.isMetaDataEnabled(typeId);
-
-            PortableMetadata metadata = null;
-
-            if (metadataEnabled)
-                metadata = ctx.metaData(typeId);
-
-            Map<String, String> newFldsMetadata = null;
-
-            for (Map.Entry<String, Object> entry : assignedVals.entrySet()) {
-                Object val = entry.getValue();
-
-                if (val == REMOVED_FIELD_MARKER)
-                    continue;
-
-                String name = entry.getKey();
-
-                int fldId = ctx.fieldId(typeId, name);
-
-                if (remainsFlds != null && !remainsFlds.contains(fldId))
-                    continue;
-
-                writer.writeInt(fldId);
-
-                int lenPos = writer.reserveAndMark(4);
-
-                serializer.writeValue(writer, val);
-
-                writer.writeDelta(lenPos);
-
-                if (metadataEnabled) {
-                    String oldFldTypeName = metadata == null ? null : metadata.fieldTypeName(name);
-
-                    String newFldTypeName;
-
-                    if (val instanceof PortableValueWithType)
-                        newFldTypeName = ((PortableValueWithType)val).typeName();
-                    else {
-                        byte type = PortableUtils.typeByClass(val.getClass());
-
-                        newFldTypeName = CacheObjectPortableProcessorImpl.fieldTypeName(type);
-                    }
-
-                    if (oldFldTypeName == null) {
-                        // It's a new field, we have to add it to metadata.
-
-                        if (newFldsMetadata == null)
-                            newFldsMetadata = new HashMap<>();
-
-                        newFldsMetadata.put(name, newFldTypeName);
-                    }
-                    else {
-                        if (!"Object".equals(oldFldTypeName) && !oldFldTypeName.equals(newFldTypeName)) {
-                            throw new PortableException(
-                                "Wrong value has been set [" +
-                                    "typeName=" + (typeName == null ? metadata.typeName() : typeName) +
-                                    ", fieldName=" + name +
-                                    ", fieldType=" + oldFldTypeName +
-                                    ", assignedValueType=" + newFldTypeName +
-                                    ", assignedValue=" + (((PortableValueWithType)val).value()) + ']'
-                            );
-                        }
-                    }
-                }
-            }
-
-            if (newFldsMetadata != null) {
-                String typeName = this.typeName;
-
-                if (typeName == null)
-                    typeName = metadata.typeName();
-
-                ctx.updateMetaData(typeId, typeName, newFldsMetadata);
-            }
-        }
-
-        writer.writeRawOffsetIfNeeded();
-
-        if (reader != null) {
-            int rawOff = reader.readIntAbsolute(start + RAW_DATA_OFF_POS);
-            int len = reader.readIntAbsolute(start + TOTAL_LEN_POS);
-
-            if (rawOff < len)
-                writer.write(reader.array(), rawOff, len - rawOff);
-        }
-
-        writer.writeLength();
-    }
-
-    /** {@inheritDoc} */
-    @Override public PortableBuilderImpl hashCode(int hashCode) {
-        this.hashCode = hashCode;
-
-        return this;
-    }
-
-    /**
-     *
-     */
-    private void ensureReadCacheInit() {
-        if (readCache == null) {
-            Map<Integer, Object> readCache = new HashMap<>();
-
-            int pos = start + hdrLen;
-            int end = start + reader.readIntAbsolute(start + RAW_DATA_OFF_POS);
-
-            while (pos < end) {
-                int fieldId = reader.readIntAbsolute(pos);
-
-                pos += 4;
-
-                int len = reader.readIntAbsolute(pos);
-
-                pos += 4;
-
-                Object val = reader.getValueQuickly(pos, len);
-
-                readCache.put(fieldId, val);
-
-                pos += len;
-            }
-
-            this.readCache = readCache;
-        }
-    }
-
-    /** {@inheritDoc} */
-    @Override public <F> F getField(String name) {
-        Object val;
-
-        if (assignedVals != null && assignedVals.containsKey(name)) {
-            val = assignedVals.get(name);
-
-            if (val == REMOVED_FIELD_MARKER)
-                return null;
-        }
-        else {
-            ensureReadCacheInit();
-
-            int fldId = ctx.fieldId(typeId, name);
-
-            val = readCache.get(fldId);
-        }
-
-        return (F)PortableUtils.unwrapLazy(val);
-    }
-
-    /** {@inheritDoc} */
-    @Override public PortableBuilder setField(String name, Object val) {
-        GridArgumentCheck.notNull(val, "val");
-
-        if (assignedVals == null)
-            assignedVals = new LinkedHashMap<>();
-
-        Object oldVal = assignedVals.put(name, val);
-
-        if (oldVal instanceof PortableValueWithType) {
-            ((PortableValueWithType)oldVal).value(val);
-
-            assignedVals.put(name, oldVal);
-        }
-
-        return this;
-    }
-
-    /** {@inheritDoc} */
-    @Override public <T> PortableBuilder setField(String name, @Nullable T val, Class<? super T> type) {
-        if (assignedVals == null)
-            assignedVals = new LinkedHashMap<>();
-
-        //int fldId = ctx.fieldId(typeId, fldName);
-
-        assignedVals.put(name, new PortableValueWithType(PortableUtils.typeByClass(type), val));
-
-        return this;
-    }
-
-    /** {@inheritDoc} */
-    @Override public PortableBuilder setField(String name, @Nullable PortableBuilder builder) {
-        if (builder == null)
-            return setField(name, null, Object.class);
-        else
-            return setField(name, (Object)builder);
-    }
-
-    /**
-     * Removes field from portable object.
-     *
-     * @param name Field name.
-     * @return {@code this} instance for chaining.
-     */
-    @Override public PortableBuilderImpl removeField(String name) {
-        if (assignedVals == null)
-            assignedVals = new LinkedHashMap<>();
-
-        assignedVals.put(name, REMOVED_FIELD_MARKER);
-
-        return this;
-    }
-
-    /**
-     * Creates builder initialized by specified portable object.
-     *
-     * @param obj Portable object to initialize builder.
-     * @return New builder.
-     */
-    public static PortableBuilderImpl wrap(PortableObject obj) {
-        PortableObjectImpl heapObj;
-
-        if (obj instanceof PortableObjectOffheapImpl)
-            heapObj = (PortableObjectImpl)((PortableObjectOffheapImpl)obj).heapCopy();
-        else
-            heapObj = (PortableObjectImpl)obj;
-
-        return new PortableBuilderImpl(heapObj);
-    }
-
-    /**
-     * @return Object start position in source array.
-     */
-    int start() {
-        return start;
-    }
-
-    /**
-     * @return Object type id.
-     */
-    int typeId() {
-        return typeId;
-    }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/6ac9557e/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableBuilderReader.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableBuilderReader.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableBuilderReader.java
deleted file mode 100644
index 30b31f0..0000000
--- a/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableBuilderReader.java
+++ /dev/null
@@ -1,776 +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;
-
-import java.sql.Timestamp;
-import java.util.Date;
-import java.util.HashMap;
-import java.util.Map;
-import org.apache.ignite.portable.PortableException;
-
-import static java.nio.charset.StandardCharsets.UTF_8;
-import static org.apache.ignite.internal.portable.GridPortableMarshaller.NULL;
-import static org.apache.ignite.internal.portable.GridPortableMarshaller.STRING;
-
-/**
- *
- */
-class PortableBuilderReader {
-    /** */
-    private static final PortablePrimitives PRIM = PortablePrimitives.get();
-
-    /** */
-    private final Map<Integer, PortableBuilderImpl> objMap = new HashMap<>();
-
-    /** */
-    private final PortableContext ctx;
-
-    /** */
-    private final PortableReaderExImpl reader;
-
-    /** */
-    private byte[] arr;
-
-    /** */
-    private int pos;
-
-    /**
-     * @param objImpl Portable object
-     */
-    PortableBuilderReader(PortableObjectImpl objImpl) {
-        ctx = objImpl.context();
-        arr = objImpl.array();
-        pos = objImpl.start();
-
-        // TODO: IGNITE-1272 - Is class loader needed here?
-        reader = new PortableReaderExImpl(portableContext(), arr, pos, null);
-    }
-
-    /**
-     * @return Portable context.
-     */
-    public PortableContext portableContext() {
-        return ctx;
-    }
-
-    /**
-     * @param obj Mutable portable object.
-     */
-    public void registerObject(PortableBuilderImpl obj) {
-        objMap.put(obj.start(), obj);
-    }
-
-    /**
-     * @return Read int value.
-     */
-    public int readInt() {
-        int res = readInt(0);
-
-        pos += 4;
-
-        return res;
-    }
-
-    /**
-     * @return Read int value.
-     */
-    public byte readByte() {
-        return arr[pos++];
-    }
-
-    /**
-     * @return Read boolean value.
-     */
-    public boolean readBoolean() {
-        return readByte() == 1;
-    }
-
-    /**
-     * @return Read int value.
-     */
-    public byte readByte(int off) {
-        return arr[pos + off];
-    }
-
-    /**
-     * @param off Offset related to {@link #pos}
-     * @return Read int value.
-     */
-    public int readInt(int off) {
-        return PRIM.readInt(arr, pos + off);
-    }
-
-    /**
-     * @param pos Position in the source array.
-     * @return Read int value.
-     */
-    public int readIntAbsolute(int pos) {
-        return PRIM.readInt(arr, pos);
-    }
-
-    /**
-     * @return Read length of array.
-     */
-    public int readLength() {
-        return PRIM.readInt(arr, pos);
-    }
-
-    /**
-     * Read string length.
-     *
-     * @return String length.
-     */
-    public int readStringLength() {
-        boolean utf = PRIM.readBoolean(arr, pos);
-
-        int arrLen = PRIM.readInt(arr, pos + 1);
-
-        return 1 + (utf ? arrLen : arrLen << 1);
-    }
-
-    /**
-     * Reads string.
-     *
-     * @return String.
-     */
-    public String readString() {
-        byte flag = readByte();
-
-        if (flag == NULL)
-            return null;
-
-        if (flag != STRING)
-            throw new PortableException("Failed to deserialize String.");
-
-        boolean convert = readBoolean();
-        int len = readInt();
-
-        String str;
-
-        if (convert) {
-            str = new String(arr, pos, len, UTF_8);
-
-            pos += len;
-        }
-        else {
-            str = String.valueOf(PRIM.readCharArray(arr, pos, len));
-
-            pos += len << 1;
-        }
-
-        return str;
-    }
-
-    /**
-     *
-     */
-    public void skipValue() {
-        byte type = arr[pos++];
-
-        int len;
-
-        switch (type) {
-            case GridPortableMarshaller.NULL:
-                return;
-
-            case GridPortableMarshaller.OBJ:
-                pos += readInt(GridPortableMarshaller.TOTAL_LEN_POS - 1) - 1;
-
-                return;
-
-            case GridPortableMarshaller.BOOLEAN:
-            case GridPortableMarshaller.BYTE:
-                len = 1;
-                break;
-
-            case GridPortableMarshaller.CHAR:
-            case GridPortableMarshaller.SHORT:
-                len = 2;
-
-                break;
-
-            case GridPortableMarshaller.HANDLE:
-            case GridPortableMarshaller.FLOAT:
-            case GridPortableMarshaller.INT:
-                len = 4;
-
-                break;
-
-            case GridPortableMarshaller.ENUM:
-                //skipping type id and ordinal value
-                len = 8;
-
-                break;
-
-            case GridPortableMarshaller.LONG:
-            case GridPortableMarshaller.DOUBLE:
-                len = 8;
-
-                break;
-
-            case GridPortableMarshaller.BYTE_ARR:
-            case GridPortableMarshaller.BOOLEAN_ARR:
-                len = 4 + readLength();
-
-                break;
-
-            case GridPortableMarshaller.STRING:
-                len = 4 + readStringLength();
-
-                break;
-
-            case GridPortableMarshaller.DECIMAL:
-                len = /** scale */ 4  + /** mag len */ 4  + /** mag bytes count */ readInt(4);
-
-                break;
-
-            case GridPortableMarshaller.UUID:
-                len = 8 + 8;
-
-                break;
-
-            case GridPortableMarshaller.DATE:
-                len = 8 + 4;
-
-                break;
-
-            case GridPortableMarshaller.CHAR_ARR:
-            case GridPortableMarshaller.SHORT_ARR:
-                len = 4 + readLength() * 2;
-
-                break;
-
-            case GridPortableMarshaller.INT_ARR:
-            case GridPortableMarshaller.FLOAT_ARR:
-                len = 4 + readLength() * 4;
-
-                break;
-
-            case GridPortableMarshaller.LONG_ARR:
-            case GridPortableMarshaller.DOUBLE_ARR:
-                len = 4 + readLength() * 8;
-
-                break;
-
-            case GridPortableMarshaller.DECIMAL_ARR:
-            case GridPortableMarshaller.DATE_ARR:
-            case GridPortableMarshaller.OBJ_ARR:
-            case GridPortableMarshaller.ENUM_ARR:
-            case GridPortableMarshaller.UUID_ARR:
-            case GridPortableMarshaller.STRING_ARR: {
-                int size = readInt();
-
-                for (int i = 0; i < size; i++)
-                    skipValue();
-
-                return;
-            }
-
-            case GridPortableMarshaller.COL: {
-                int size = readInt();
-
-                pos++; // skip collection type
-
-                for (int i = 0; i < size; i++)
-                    skipValue();
-
-                return;
-            }
-
-            case GridPortableMarshaller.MAP: {
-                int size = readInt();
-
-                pos++; // skip collection type
-
-                for (int i = 0; i < size; i++) {
-                    skipValue(); // skip key.
-                    skipValue(); // skip value.
-                }
-
-                return;
-            }
-
-            case GridPortableMarshaller.MAP_ENTRY:
-                skipValue();
-                skipValue();
-
-                return;
-
-            case GridPortableMarshaller.PORTABLE_OBJ:
-                len = readInt() + 4;
-
-                break;
-
-            default:
-                throw new PortableException("Invalid flag value: " + type);
-        }
-
-        pos += len;
-    }
-
-    /**
-     * @param pos Position.
-     * @param len Length.
-     * @return Object.
-     */
-    public Object getValueQuickly(int pos, int len) {
-        byte type = arr[pos];
-
-        switch (type) {
-            case GridPortableMarshaller.NULL:
-                return null;
-
-            case GridPortableMarshaller.HANDLE: {
-                int objStart = pos - readIntAbsolute(pos + 1);
-
-                PortableBuilderImpl res = objMap.get(objStart);
-
-                if (res == null) {
-                    res = new PortableBuilderImpl(this, objStart);
-
-                    objMap.put(objStart, res);
-                }
-
-                return res;
-            }
-
-            case GridPortableMarshaller.OBJ: {
-                PortableBuilderImpl res = objMap.get(pos);
-
-                if (res == null) {
-                    res = new PortableBuilderImpl(this, pos);
-
-                    objMap.put(pos, res);
-                }
-
-                return res;
-            }
-
-            case GridPortableMarshaller.BYTE:
-                return arr[pos + 1];
-
-            case GridPortableMarshaller.SHORT:
-                return PRIM.readShort(arr, pos + 1);
-
-            case GridPortableMarshaller.INT:
-                return PRIM.readInt(arr, pos + 1);
-
-            case GridPortableMarshaller.LONG:
-                return PRIM.readLong(arr, pos + 1);
-
-            case GridPortableMarshaller.FLOAT:
-                return PRIM.readFloat(arr, pos + 1);
-
-            case GridPortableMarshaller.DOUBLE:
-                return PRIM.readDouble(arr, pos + 1);
-
-            case GridPortableMarshaller.CHAR:
-                return PRIM.readChar(arr, pos + 1);
-
-            case GridPortableMarshaller.BOOLEAN:
-                return arr[pos + 1] != 0;
-
-            case GridPortableMarshaller.DECIMAL:
-            case GridPortableMarshaller.STRING:
-            case GridPortableMarshaller.UUID:
-            case GridPortableMarshaller.DATE:
-            case GridPortableMarshaller.BYTE_ARR:
-            case GridPortableMarshaller.SHORT_ARR:
-            case GridPortableMarshaller.INT_ARR:
-            case GridPortableMarshaller.LONG_ARR:
-            case GridPortableMarshaller.FLOAT_ARR:
-            case GridPortableMarshaller.DOUBLE_ARR:
-            case GridPortableMarshaller.CHAR_ARR:
-            case GridPortableMarshaller.BOOLEAN_ARR:
-            case GridPortableMarshaller.DECIMAL_ARR:
-            case GridPortableMarshaller.DATE_ARR:
-            case GridPortableMarshaller.UUID_ARR:
-            case GridPortableMarshaller.STRING_ARR:
-                return new PortablePlainLazyValue(this, pos, len);
-
-            case GridPortableMarshaller.COL:
-            case GridPortableMarshaller.OBJ_ARR:
-            case GridPortableMarshaller.MAP:
-            case GridPortableMarshaller.ENUM_ARR:
-            case GridPortableMarshaller.MAP_ENTRY:
-                return new LazyCollection(pos);
-
-            case GridPortableMarshaller.ENUM: {
-                if (len == 1) {
-                    assert readByte(pos) == GridPortableMarshaller.NULL;
-
-                    return null;
-                }
-
-                int mark = position();
-                position(pos + 1);
-
-                PortableBuilderEnum builderEnum = new PortableBuilderEnum(this);
-
-                position(mark);
-
-                return builderEnum;
-            }
-
-            case GridPortableMarshaller.PORTABLE_OBJ: {
-                int size = readIntAbsolute(pos + 1);
-
-                int start = readIntAbsolute(pos + 4 + size);
-
-                PortableObjectImpl portableObj = new PortableObjectImpl(ctx, arr, pos + 4 + start);
-
-                return new PortablePlainPortableObject(portableObj);
-            }
-
-            default:
-                throw new PortableException("Invalid flag value: " + type);
-        }
-    }
-
-    /**
-     * @return Parsed value.
-     */
-    public Object parseValue() {
-        int valPos = pos;
-
-        byte type = arr[pos++];
-
-        int plainLazyValLen;
-
-        switch (type) {
-            case GridPortableMarshaller.NULL:
-                return null;
-
-            case GridPortableMarshaller.HANDLE: {
-                int objStart = pos - 1 - readInt();
-
-                PortableBuilderImpl res = objMap.get(objStart);
-
-                if (res == null) {
-                    res = new PortableBuilderImpl(this, objStart);
-
-                    objMap.put(objStart, res);
-                }
-
-                return res;
-            }
-
-            case GridPortableMarshaller.OBJ: {
-                pos--;
-
-                PortableBuilderImpl res = objMap.get(pos);
-
-                if (res == null) {
-                    res = new PortableBuilderImpl(this, pos);
-
-                    objMap.put(pos, res);
-                }
-
-                pos += readInt(GridPortableMarshaller.TOTAL_LEN_POS);
-
-                return res;
-            }
-
-            case GridPortableMarshaller.BYTE:
-                return arr[pos++];
-
-            case GridPortableMarshaller.SHORT: {
-                Object res = PRIM.readShort(arr, pos);
-                pos += 2;
-                return res;
-            }
-
-            case GridPortableMarshaller.INT:
-                return readInt();
-
-            case GridPortableMarshaller.LONG:
-                plainLazyValLen = 8;
-
-                break;
-
-            case GridPortableMarshaller.FLOAT:
-                plainLazyValLen = 4;
-
-                break;
-
-            case GridPortableMarshaller.DOUBLE:
-                plainLazyValLen = 8;
-
-                break;
-
-            case GridPortableMarshaller.CHAR:
-                plainLazyValLen = 2;
-
-                break;
-
-            case GridPortableMarshaller.BOOLEAN:
-                return arr[pos++] != 0;
-
-            case GridPortableMarshaller.DECIMAL:
-                plainLazyValLen = /** scale */ 4  + /** mag len */ 4  + /** mag bytes count */ readInt(4);
-
-                break;
-
-            case GridPortableMarshaller.STRING:
-                plainLazyValLen = 4 + readStringLength();
-
-                break;
-
-            case GridPortableMarshaller.UUID:
-                plainLazyValLen = 8 + 8;
-
-                break;
-
-            case GridPortableMarshaller.DATE:
-                plainLazyValLen = 8 + 4;
-
-                break;
-
-            case GridPortableMarshaller.BYTE_ARR:
-                plainLazyValLen = 4 + readLength();
-
-                break;
-
-            case GridPortableMarshaller.SHORT_ARR:
-                plainLazyValLen = 4 + readLength() * 2;
-
-                break;
-
-            case GridPortableMarshaller.INT_ARR:
-                plainLazyValLen = 4 + readLength() * 4;
-
-                break;
-
-            case GridPortableMarshaller.LONG_ARR:
-                plainLazyValLen = 4 + readLength() * 8;
-
-                break;
-
-            case GridPortableMarshaller.FLOAT_ARR:
-                plainLazyValLen = 4 + readLength() * 4;
-
-                break;
-
-            case GridPortableMarshaller.DOUBLE_ARR:
-                plainLazyValLen = 4 + readLength() * 8;
-
-                break;
-
-            case GridPortableMarshaller.CHAR_ARR:
-                plainLazyValLen = 4 + readLength() * 2;
-
-                break;
-
-            case GridPortableMarshaller.BOOLEAN_ARR:
-                plainLazyValLen = 4 + readLength();
-
-                break;
-
-            case GridPortableMarshaller.OBJ_ARR:
-                return new PortableObjectArrayLazyValue(this);
-
-            case GridPortableMarshaller.DATE_ARR: {
-                int size = readInt();
-
-                Date[] res = new Date[size];
-
-                for (int i = 0; i < res.length; i++) {
-                    byte flag = arr[pos++];
-
-                    if (flag == GridPortableMarshaller.NULL) continue;
-
-                    if (flag != GridPortableMarshaller.DATE)
-                        throw new PortableException("Invalid flag value: " + flag);
-
-                    long time = PRIM.readLong(arr, pos);
-
-                    pos += 8;
-
-                    if (ctx.isUseTimestamp()) {
-                        Timestamp ts = new Timestamp(time);
-
-                        ts.setNanos(ts.getNanos() + readInt());
-
-                        res[i] = ts;
-                    }
-                    else {
-                        res[i] = new Date(time);
-
-                        pos += 4;
-                    }
-                }
-
-                return res;
-            }
-
-            case GridPortableMarshaller.UUID_ARR:
-            case GridPortableMarshaller.STRING_ARR:
-            case GridPortableMarshaller.DECIMAL_ARR: {
-                int size = readInt();
-
-                for (int i = 0; i < size; i++) {
-                    byte flag = arr[pos++];
-
-                    if (flag == GridPortableMarshaller.UUID)
-                        pos += 8 + 8;
-                    else if (flag == GridPortableMarshaller.STRING)
-                        pos += 4 + readStringLength();
-                    else if (flag == GridPortableMarshaller.DECIMAL)
-                        pos += 4 + readLength();
-                    else
-                        assert flag == GridPortableMarshaller.NULL;
-                }
-
-                return new PortablePlainLazyValue(this, valPos, pos - valPos);
-            }
-
-            case GridPortableMarshaller.COL: {
-                int size = readInt();
-                byte colType = arr[pos++];
-
-                switch (colType) {
-                    case GridPortableMarshaller.USER_COL:
-                    case GridPortableMarshaller.ARR_LIST:
-                        return new PortableLazyArrayList(this, size);
-
-                    case GridPortableMarshaller.LINKED_LIST:
-                        return new PortableLazyLinkedList(this, size);
-
-                    case GridPortableMarshaller.HASH_SET:
-                    case GridPortableMarshaller.LINKED_HASH_SET:
-                    case GridPortableMarshaller.TREE_SET:
-                    case GridPortableMarshaller.CONC_SKIP_LIST_SET:
-                        return new PortableLazySet(this, size);
-                }
-
-                throw new PortableException("Unknown collection type: " + colType);
-            }
-
-            case GridPortableMarshaller.MAP:
-                return PortableLazyMap.parseMap(this);
-
-            case GridPortableMarshaller.ENUM:
-                return new PortableBuilderEnum(this);
-
-            case GridPortableMarshaller.ENUM_ARR:
-                return new PortableEnumArrayLazyValue(this);
-
-            case GridPortableMarshaller.MAP_ENTRY:
-                return new PortableLazyMapEntry(this);
-
-            case GridPortableMarshaller.PORTABLE_OBJ: {
-                int size = readInt();
-
-                pos += size;
-
-                int start = readInt();
-
-                PortableObjectImpl portableObj = new PortableObjectImpl(ctx, arr,
-                    pos - 4 - size + start);
-
-                return new PortablePlainPortableObject(portableObj);
-            }
-
-
-            default:
-                throw new PortableException("Invalid flag value: " + type);
-        }
-
-        PortablePlainLazyValue res = new PortablePlainLazyValue(this, valPos, 1 + plainLazyValLen);
-
-        pos += plainLazyValLen;
-
-        return res;
-    }
-
-    /**
-     * @return Array.
-     */
-    public byte[] array() {
-        return arr;
-    }
-
-    /**
-     * @return Position of reader.
-     */
-    public int position() {
-        return pos;
-    }
-
-    /**
-     * @param pos New pos.
-     */
-    public void position(int pos) {
-        this.pos = pos;
-    }
-
-    /**
-     * @param n Number of bytes to skip.
-     */
-    public void skip(int n) {
-        pos += n;
-    }
-
-    /**
-     * @return Reader.
-     */
-    PortableReaderExImpl reader() {
-        return reader;
-    }
-
-    /**
-     *
-     */
-    private class LazyCollection implements PortableLazyValue {
-        /** */
-        private final int valOff;
-
-        /** */
-        private Object col;
-
-        /**
-         * @param valOff Value.
-         */
-        protected LazyCollection(int valOff) {
-            this.valOff = valOff;
-        }
-
-        /**
-         * @return Object.
-         */
-        private Object wrappedCollection() {
-            if (col == null) {
-                position(valOff);
-
-                col = parseValue();
-            }
-
-            return col;
-        }
-
-        /** {@inheritDoc} */
-        @Override public void writeTo(PortableWriterExImpl writer, PortableBuilderSerializer ctx) {
-            ctx.writeValue(writer, wrappedCollection());
-        }
-
-        /** {@inheritDoc} */
-        @Override public Object value() {
-            return PortableUtils.unwrapLazy(wrappedCollection());
-        }
-    }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/6ac9557e/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableBuilderSerializationAware.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableBuilderSerializationAware.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableBuilderSerializationAware.java
deleted file mode 100644
index d75cd7b..0000000
--- a/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableBuilderSerializationAware.java
+++ /dev/null
@@ -1,29 +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;
-
-/**
- *
- */
-interface PortableBuilderSerializationAware {
-    /**
-     * @param writer Writer.
-     * @param ctx Context.
-     */
-    public void writeTo(PortableWriterExImpl writer, PortableBuilderSerializer ctx);
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/6ac9557e/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableBuilderSerializer.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableBuilderSerializer.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableBuilderSerializer.java
deleted file mode 100644
index 6ce9f4c..0000000
--- a/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableBuilderSerializer.java
+++ /dev/null
@@ -1,211 +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;
-
-import java.util.Collection;
-import java.util.IdentityHashMap;
-import java.util.Map;
-import org.apache.ignite.internal.util.GridConcurrentSkipListSet;
-import org.apache.ignite.portable.PortableObject;
-
-/**
- *
- */
-class PortableBuilderSerializer {
-    /** */
-    private final Map<PortableBuilderImpl, Integer> objToPos = new IdentityHashMap<>();
-
-    /** */
-    private Map<PortableObject, PortableBuilderImpl> portableObjToWrapper;
-
-    /**
-     * @param obj Mutable object.
-     * @param posInResArr Object position in the array.
-     */
-    public void registerObjectWriting(PortableBuilderImpl obj, int posInResArr) {
-        objToPos.put(obj, posInResArr);
-    }
-
-    /**
-     * @param writer Writer.
-     * @param val Value.
-     */
-    public void writeValue(PortableWriterExImpl writer, Object val) {
-        if (val == null) {
-            writer.writeByte(GridPortableMarshaller.NULL);
-
-            return;
-        }
-
-        if (val instanceof PortableBuilderSerializationAware) {
-            ((PortableBuilderSerializationAware)val).writeTo(writer, this);
-
-            return;
-        }
-
-        if (val instanceof PortableObjectEx) {
-            if (portableObjToWrapper == null)
-                portableObjToWrapper = new IdentityHashMap<>();
-
-            PortableBuilderImpl wrapper = portableObjToWrapper.get(val);
-
-            if (wrapper == null) {
-                wrapper = PortableBuilderImpl.wrap((PortableObject)val);
-
-                portableObjToWrapper.put((PortableObject)val, wrapper);
-            }
-
-            val = wrapper;
-        }
-
-        if (val instanceof PortableBuilderImpl) {
-            PortableBuilderImpl obj = (PortableBuilderImpl)val;
-
-            Integer posInResArr = objToPos.get(obj);
-
-            if (posInResArr == null) {
-                objToPos.put(obj, writer.outputStream().position());
-
-                obj.serializeTo(writer.newWriter(obj.typeId()), this);
-            }
-            else {
-                int handle = writer.outputStream().position() - posInResArr;
-
-                writer.writeByte(GridPortableMarshaller.HANDLE);
-                writer.writeInt(handle);
-            }
-
-            return;
-        }
-
-        if (val.getClass().isEnum()) {
-            writer.writeByte(GridPortableMarshaller.ENUM);
-            writer.writeInt(writer.context().typeId(val.getClass().getName()));
-            writer.writeInt(((Enum)val).ordinal());
-
-            return;
-        }
-
-        if (val instanceof Collection) {
-            Collection<?> c = (Collection<?>)val;
-
-            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());
-
-
-            writer.writeByte(colType);
-
-            for (Object obj : c)
-                writeValue(writer, obj);
-
-            return;
-        }
-
-        if (val instanceof Map) {
-            Map<?, ?> map = (Map<?, ?>)val;
-
-            writer.writeByte(GridPortableMarshaller.MAP);
-            writer.writeInt(map.size());
-
-            writer.writeByte(writer.context().mapType(map.getClass()));
-
-            for (Map.Entry<?, ?> entry : map.entrySet()) {
-                writeValue(writer, entry.getKey());
-                writeValue(writer, entry.getValue());
-            }
-
-            return;
-        }
-
-        Byte flag = PortableUtils.PLAIN_CLASS_TO_FLAG.get(val.getClass());
-
-        if (flag != null) {
-            PortableUtils.writePlainObject(writer, val);
-
-            return;
-        }
-
-        if (val instanceof Object[]) {
-            int compTypeId = writer.context().typeId(((Object[])val).getClass().getComponentType().getName());
-
-            if (val instanceof PortableBuilderEnum[]) {
-                writeArray(writer, GridPortableMarshaller.ENUM_ARR, (Object[])val, compTypeId);
-
-                return;
-            }
-
-            if (((Object[])val).getClass().getComponentType().isEnum()) {
-                Enum[] enumArr = (Enum[])val;
-
-                writer.writeByte(GridPortableMarshaller.ENUM_ARR);
-                writer.writeInt(compTypeId);
-                writer.writeInt(enumArr.length);
-
-                for (Enum anEnum : enumArr)
-                    writeValue(writer, anEnum);
-
-                return;
-            }
-
-            writeArray(writer, GridPortableMarshaller.OBJ_ARR, (Object[])val, compTypeId);
-
-            return;
-        }
-
-        writer.doWriteObject(val, false);
-    }
-
-    /**
-     * @param writer Writer.
-     * @param elementType Element type.
-     * @param arr The array.
-     * @param compTypeId Component type ID.
-     */
-    public void writeArray(PortableWriterExImpl writer, byte elementType, Object[] arr, int compTypeId) {
-        writer.writeByte(elementType);
-        writer.writeInt(compTypeId);
-        writer.writeInt(arr.length);
-
-        for (Object obj : arr)
-            writeValue(writer, obj);
-    }
-
-    /**
-     * @param writer Writer.
-     * @param elementType Element type.
-     * @param arr The array.
-     * @param clsName Component class name.
-     */
-    public void writeArray(PortableWriterExImpl writer, byte elementType, Object[] arr, String clsName) {
-        writer.writeByte(elementType);
-        writer.writeInt(GridPortableMarshaller.UNREGISTERED_TYPE_ID);
-        writer.writeString(clsName);
-        writer.writeInt(arr.length);
-
-        for (Object obj : arr)
-            writeValue(writer, obj);
-    }
-
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/6ac9557e/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 7e4266c..24ad5ce 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
@@ -48,7 +48,7 @@ import static java.lang.reflect.Modifier.isTransient;
 /**
  * Portable class descriptor.
  */
-class PortableClassDescriptor {
+public class PortableClassDescriptor {
     /** */
     private final PortableContext ctx;
 
@@ -123,7 +123,7 @@ class PortableClassDescriptor {
         boolean keepDeserialized
     ) throws PortableException {
         this(ctx, cls, userType, typeId, typeName, idMapper, serializer, useTs, metaDataEnabled, keepDeserialized,
-             true);
+            true);
     }
 
     /**
@@ -285,7 +285,7 @@ class PortableClassDescriptor {
     /**
      * @return Type ID.
      */
-    int typeId() {
+    public int typeId() {
         return typeId;
     }
 
@@ -399,7 +399,7 @@ class PortableClassDescriptor {
                 break;
 
             case DECIMAL:
-                writer.doWriteDecimal((BigDecimal) obj);
+                writer.doWriteDecimal((BigDecimal)obj);
 
                 break;
 
@@ -656,39 +656,32 @@ class PortableClassDescriptor {
      * @return Whether further write is needed.
      */
     private boolean writeHeader(Object obj, PortableWriterExImpl writer) {
-        int handle = writer.handle(obj);
-
-        if (handle >= 0) {
-            writer.doWriteByte(GridPortableMarshaller.HANDLE);
-            writer.doWriteInt(handle);
-
+        if (writer.tryWriteAsHandle(obj))
             return false;
-        }
-        else {
-            int pos = writer.position();
 
-            writer.doWriteByte(GridPortableMarshaller.OBJ);
-            writer.doWriteBoolean(userType);
-            writer.doWriteInt(registered ? typeId : GridPortableMarshaller.UNREGISTERED_TYPE_ID);
-            writer.doWriteInt(obj instanceof CacheObjectImpl ? 0 : obj.hashCode());
+        int pos = writer.position();
 
-            // For length and raw offset.
-            int reserved = writer.reserve(8);
+        writer.doWriteByte(GridPortableMarshaller.OBJ);
+        writer.doWriteBoolean(userType);
+        writer.doWriteInt(registered ? typeId : GridPortableMarshaller.UNREGISTERED_TYPE_ID);
+        writer.doWriteInt(obj instanceof CacheObjectImpl ? 0 : obj.hashCode());
 
-            // Class name in case if typeId registration is failed.
-            if (!registered)
-                writer.doWriteString(cls.getName());
+        // For length and raw offset.
+        int reserved = writer.reserve(8);
 
-            int current = writer.position();
-            int len = current - pos;
+        // Class name in case if typeId registration is failed.
+        if (!registered)
+            writer.doWriteString(cls.getName());
 
-            // Default raw offset (equal to header length).
-            writer.position(reserved + 4);
-            writer.doWriteInt(len);
-            writer.position(current);
+        int current = writer.position();
+        int len = current - pos;
 
-            return true;
-        }
+        // Default raw offset (equal to header length).
+        writer.position(reserved + 4);
+        writer.doWriteInt(len);
+        writer.position(current);
+
+        return true;
     }
 
     /**
@@ -787,7 +780,7 @@ class PortableClassDescriptor {
         else if (cls == PortableObjectImpl.class)
             return Mode.PORTABLE_OBJ;
         else if (PortableMarshalAware.class.isAssignableFrom(cls))
-           return Mode.PORTABLE;
+            return Mode.PORTABLE;
         else if (Externalizable.class.isAssignableFrom(cls))
             return Mode.EXTERNALIZABLE;
         else if (Map.Entry.class.isAssignableFrom(cls))
@@ -1352,4 +1345,4 @@ class PortableClassDescriptor {
             return typeName;
         }
     }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/6ac9557e/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 15b6433..db7e41e 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
@@ -794,10 +794,10 @@ public class PortableContext implements Externalizable {
      * @param typeId Type ID.
      * @return Whether meta data is enabled.
      */
-    boolean isMetaDataEnabled(int typeId) {
+    public boolean isMetaDataEnabled(int typeId) {
         Boolean enabled = metaEnabled.get(typeId);
 
-        return enabled != null ? enabled : true;
+        return enabled != null ? enabled : metaDataEnabled;
     }
 
     /**
@@ -827,7 +827,7 @@ public class PortableContext implements Externalizable {
      * @param fields Fields map.
      * @throws PortableException In case of error.
      */
-    void updateMetaData(int typeId, String typeName, Map<String, String> fields) throws PortableException {
+    public void updateMetaData(int typeId, String typeName, Map<String, String> fields) throws PortableException {
         updateMetaData(typeId, new PortableMetaDataImpl(typeName, fields, null));
     }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/6ac9557e/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableEnumArrayLazyValue.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableEnumArrayLazyValue.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableEnumArrayLazyValue.java
deleted file mode 100644
index c953bb3..0000000
--- a/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableEnumArrayLazyValue.java
+++ /dev/null
@@ -1,112 +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;
-
-import org.apache.ignite.internal.util.typedef.internal.U;
-import org.apache.ignite.portable.PortableException;
-import org.apache.ignite.portable.PortableInvalidClassException;
-
-/**
- *
- */
-class PortableEnumArrayLazyValue extends PortableAbstractLazyValue {
-    /** */
-    private final int len;
-
-    /** */
-    private final int compTypeId;
-
-    /** */
-    private final String clsName;
-
-    /**
-     * @param reader Reader.
-     */
-    protected PortableEnumArrayLazyValue(PortableBuilderReader reader) {
-        super(reader, reader.position() - 1);
-
-        int typeId = reader.readInt();
-
-        if (typeId == GridPortableMarshaller.UNREGISTERED_TYPE_ID) {
-            clsName = reader.readString();
-
-            Class cls;
-
-            try {
-                // TODO: IGNITE-1272 - Is class loader needed here?
-                cls = U.forName(reader.readString(), null);
-            }
-            catch (ClassNotFoundException e) {
-                throw new PortableInvalidClassException("Failed to load the class: " + clsName, e);
-            }
-
-            compTypeId = reader.portableContext().descriptorForClass(cls).typeId();
-        }
-        else {
-            compTypeId = typeId;
-            clsName = null;
-        }
-
-        int size = reader.readInt();
-
-        for (int i = 0; i < size; i++)
-            reader.skipValue();
-
-        len = reader.position() - valOff;
-    }
-
-    /** {@inheritDoc} */
-    @Override protected Object init() {
-        reader.position(valOff + 1);
-
-        //skipping component type id
-        reader.readInt();
-
-        int size = reader.readInt();
-
-        PortableBuilderEnum[] res = new PortableBuilderEnum[size];
-
-        for (int i = 0; i < size; i++) {
-            byte flag = reader.readByte();
-
-            if (flag == GridPortableMarshaller.NULL)
-                continue;
-
-            if (flag != GridPortableMarshaller.ENUM)
-                throw new PortableException("Invalid flag value: " + flag);
-
-            res[i] = new PortableBuilderEnum(reader);
-        }
-
-        return res;
-    }
-
-    /** {@inheritDoc} */
-    @Override public void writeTo(PortableWriterExImpl writer, PortableBuilderSerializer ctx) {
-        if (val != null) {
-            if (clsName != null)
-                ctx.writeArray(writer, GridPortableMarshaller.ENUM_ARR, (Object[])val, clsName);
-            else
-                ctx.writeArray(writer, GridPortableMarshaller.ENUM_ARR, (Object[])val, compTypeId);
-
-            return;
-        }
-
-        writer.write(reader.array(), valOff, len);
-    }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/6ac9557e/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableLazyArrayList.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableLazyArrayList.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableLazyArrayList.java
deleted file mode 100644
index 84a0c22..0000000
--- a/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableLazyArrayList.java
+++ /dev/null
@@ -1,159 +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;
-
-import java.util.AbstractList;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.List;
-
-/**
- *
- */
-class PortableLazyArrayList extends AbstractList<Object> implements PortableBuilderSerializationAware {
-    /** */
-    private final PortableBuilderReader reader;
-
-    /** */
-    private final int off;
-
-    /** */
-    private List<Object> delegate;
-
-    /**
-     * @param reader Reader.
-     * @param size Size,
-     */
-    PortableLazyArrayList(PortableBuilderReader reader, int size) {
-        this.reader = reader;
-        off = reader.position() - 1/* flag */ - 4/* size */ - 1/* col type */;
-
-        assert size >= 0;
-
-        for (int i = 0; i < size; i++)
-            reader.skipValue();
-    }
-
-    /**
-     *
-     */
-    private void ensureDelegateInit() {
-        if (delegate == null) {
-            int size = reader.readIntAbsolute(off + 1);
-
-            reader.position(off + 1/* flag */ + 4/* size */ + 1/* col type */);
-
-            delegate = new ArrayList<>(size);
-
-            for (int i = 0; i < size; i++)
-                delegate.add(reader.parseValue());
-        }
-    }
-
-    /** {@inheritDoc} */
-    @Override public Object get(int idx) {
-        ensureDelegateInit();
-
-        return PortableUtils.unwrapLazy(delegate.get(idx));
-    }
-
-    /** {@inheritDoc} */
-    @Override public boolean add(Object o) {
-        ensureDelegateInit();
-
-        return delegate.add(o);
-    }
-
-    /** {@inheritDoc} */
-    @Override public void add(int idx, Object element) {
-        ensureDelegateInit();
-
-        delegate.add(idx, element);
-    }
-
-    /** {@inheritDoc} */
-    @Override public Object set(int idx, Object element) {
-        ensureDelegateInit();
-
-        return PortableUtils.unwrapLazy(delegate.set(idx, element));
-    }
-
-    /** {@inheritDoc} */
-    @Override public Object remove(int idx) {
-        ensureDelegateInit();
-
-        return PortableUtils.unwrapLazy(delegate.remove(idx));
-    }
-
-    /** {@inheritDoc} */
-    @Override public void clear() {
-        if (delegate == null)
-            delegate = new ArrayList<>();
-        else
-            delegate.clear();
-    }
-
-    /** {@inheritDoc} */
-    @Override public boolean addAll(int idx, Collection<?> c) {
-        return delegate.addAll(idx, c);
-    }
-
-    /** {@inheritDoc} */
-    @Override protected void removeRange(int fromIdx, int toIdx) {
-        ensureDelegateInit();
-
-        delegate.subList(fromIdx, toIdx).clear();
-    }
-
-    /** {@inheritDoc} */
-    @Override public int size() {
-        if (delegate == null)
-            return reader.readIntAbsolute(off + 1);
-
-        return delegate.size();
-    }
-
-    /** {@inheritDoc} */
-    @Override public void writeTo(PortableWriterExImpl writer, PortableBuilderSerializer ctx) {
-        if (delegate == null) {
-            int size = reader.readIntAbsolute(off + 1);
-
-            int hdrSize = 1 /* flag */ + 4 /* size */ + 1 /* col type */;
-
-            writer.write(reader.array(), off, hdrSize);
-
-            reader.position(off + hdrSize);
-
-            for (int i = 0; i < size; i++) {
-                Object o = reader.parseValue();
-
-                ctx.writeValue(writer, o);
-            }
-        }
-        else {
-            writer.writeByte(GridPortableMarshaller.COL);
-            writer.writeInt(delegate.size());
-
-            byte colType = reader.array()[off + 1 /* flag */ + 4 /* size */];
-            writer.writeByte(colType);
-
-            for (Object o : delegate)
-                ctx.writeValue(writer, o);
-        }
-    }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/6ac9557e/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableLazyLinkedList.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableLazyLinkedList.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableLazyLinkedList.java
deleted file mode 100644
index 7b2e15a..0000000
--- a/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableLazyLinkedList.java
+++ /dev/null
@@ -1,215 +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;
-
-import java.util.AbstractList;
-import java.util.Collection;
-import java.util.Iterator;
-import java.util.LinkedList;
-import java.util.List;
-import java.util.ListIterator;
-
-/**
- *
- */
-class PortableLazyLinkedList extends AbstractList<Object> implements PortableBuilderSerializationAware {
-    /** */
-    private final PortableBuilderReader reader;
-
-    /** */
-    private final int off;
-
-    /** */
-    private List<Object> delegate;
-
-    /**
-     * @param reader Reader.
-     * @param size Size,
-     */
-    PortableLazyLinkedList(PortableBuilderReader reader, int size) {
-        this.reader = reader;
-        off = reader.position() - 1/* flag */ - 4/* size */ - 1/* col type */;
-
-        assert size >= 0;
-
-        for (int i = 0; i < size; i++)
-            reader.skipValue();
-    }
-
-    /**
-     *
-     */
-    private void ensureDelegateInit() {
-        if (delegate == null) {
-            int size = reader.readIntAbsolute(off + 1);
-
-            reader.position(off + 1/* flag */ + 4/* size */ + 1/* col type */);
-
-            delegate = new LinkedList<>();
-
-            for (int i = 0; i < size; i++)
-                delegate.add(reader.parseValue());
-        }
-    }
-
-    /** {@inheritDoc} */
-    @Override public Object get(int idx) {
-        ensureDelegateInit();
-
-        return PortableUtils.unwrapLazy(delegate.get(idx));
-    }
-
-    /** {@inheritDoc} */
-    @Override public boolean add(Object o) {
-        ensureDelegateInit();
-
-        return delegate.add(o);
-    }
-
-    /** {@inheritDoc} */
-    @Override public void add(int idx, Object element) {
-        ensureDelegateInit();
-
-        delegate.add(idx, element);
-    }
-
-    /** {@inheritDoc} */
-    @Override public Object set(int idx, Object element) {
-        ensureDelegateInit();
-
-        return PortableUtils.unwrapLazy(delegate.set(idx, element));
-    }
-
-    /** {@inheritDoc} */
-    @Override public Object remove(int idx) {
-        ensureDelegateInit();
-
-        return PortableUtils.unwrapLazy(delegate.remove(idx));
-    }
-
-    /** {@inheritDoc} */
-    @Override public void clear() {
-        if (delegate == null)
-            delegate = new LinkedList<>();
-        else
-            delegate.clear();
-    }
-
-    /** {@inheritDoc} */
-    @Override public boolean addAll(int idx, Collection<?> c) {
-        ensureDelegateInit();
-
-        return delegate.addAll(idx, c);
-    }
-
-    /** {@inheritDoc} */
-    @Override protected void removeRange(int fromIdx, int toIdx) {
-        ensureDelegateInit();
-
-        delegate.subList(fromIdx, toIdx).clear();
-    }
-
-    /** {@inheritDoc} */
-    @Override public int size() {
-        if (delegate == null)
-            return reader.readIntAbsolute(off + 1);
-
-        return delegate.size();
-    }
-
-    /** {@inheritDoc} */
-    @Override public ListIterator<Object> listIterator(final int idx) {
-        ensureDelegateInit();
-
-        return new ListIterator<Object>() {
-            /** */
-            private final ListIterator<Object> delegate = PortableLazyLinkedList.super.listIterator(idx);
-
-            @Override public boolean hasNext() {
-                return delegate.hasNext();
-            }
-
-            @Override public Object next() {
-                return PortableUtils.unwrapLazy(delegate.next());
-            }
-
-            @Override public boolean hasPrevious() {
-                return delegate.hasPrevious();
-            }
-
-            @Override public Object previous() {
-                return PortableUtils.unwrapLazy(delegate.previous());
-            }
-
-            @Override public int nextIndex() {
-                return delegate.nextIndex();
-            }
-
-            @Override public int previousIndex() {
-                return delegate.previousIndex();
-            }
-
-            @Override public void remove() {
-                delegate.remove();
-            }
-
-            @Override public void set(Object o) {
-                delegate.set(o);
-            }
-
-            @Override public void add(Object o) {
-                delegate.add(o);
-            }
-        };
-    }
-
-    /** {@inheritDoc} */
-    @Override public Iterator<Object> iterator() {
-        ensureDelegateInit();
-
-        return PortableUtils.unwrapLazyIterator(super.iterator());
-    }
-
-    /** {@inheritDoc} */
-    @Override public void writeTo(PortableWriterExImpl writer, PortableBuilderSerializer ctx) {
-        if (delegate == null) {
-            int size = reader.readIntAbsolute(off + 1);
-
-            int hdrSize = 1 /* flag */ + 4 /* size */ + 1 /* col type */;
-            writer.write(reader.array(), off, hdrSize);
-
-            reader.position(off + hdrSize);
-
-            for (int i = 0; i < size; i++) {
-                Object o = reader.parseValue();
-
-                ctx.writeValue(writer, o);
-            }
-        }
-        else {
-            writer.writeByte(GridPortableMarshaller.COL);
-            writer.writeInt(delegate.size());
-
-            byte colType = reader.array()[off + 1 /* flag */ + 4 /* size */];
-            writer.writeByte(colType);
-
-            for (Object o : delegate)
-                ctx.writeValue(writer, o);
-        }
-    }
-}
\ No newline at end of file


[26/39] ignite git commit: More debug info for 'get' futures.

Posted by ag...@apache.org.
More debug info for 'get' futures.


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

Branch: refs/heads/ignite-264
Commit: 3de5f98cf91f17a2848b0ffae8326011e88e7e7e
Parents: 3a280a0
Author: sboikov <sb...@gridgain.com>
Authored: Thu Sep 3 11:39:33 2015 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Thu Sep 3 11:39:33 2015 +0300

----------------------------------------------------------------------
 .../dht/GridPartitionedGetFuture.java           | 19 +++++++++++++++--
 .../distributed/near/GridNearGetFuture.java     | 22 +++++++++++++++++++-
 .../ignite/internal/util/lang/GridFunc.java     |  1 +
 3 files changed, 39 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/3de5f98c/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridPartitionedGetFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridPartitionedGetFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridPartitionedGetFuture.java
index 82ff69f..2f0de86 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridPartitionedGetFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridPartitionedGetFuture.java
@@ -290,7 +290,7 @@ public class GridPartitionedGetFuture<K, V> extends GridCompoundIdentityFuture<M
      * @param f Future.
      * @return {@code True} if mini-future.
      */
-    private boolean isMini(IgniteInternalFuture<Map<K, V>> f) {
+    private boolean isMini(IgniteInternalFuture<?> f) {
         return f.getClass().equals(MiniFuture.class);
     }
 
@@ -598,7 +598,22 @@ public class GridPartitionedGetFuture<K, V> extends GridCompoundIdentityFuture<M
 
     /** {@inheritDoc} */
     @Override public String toString() {
-        return S.toString(GridPartitionedGetFuture.class, this, super.toString());
+        Collection<String> futs = F.viewReadOnly(futures(), new C1<IgniteInternalFuture<?>, String>() {
+            @SuppressWarnings("unchecked")
+            @Override public String apply(IgniteInternalFuture<?> f) {
+                if (isMini(f)) {
+                    return "[node=" + ((MiniFuture)f).node().id() +
+                        ", loc=" + ((MiniFuture)f).node().isLocal() +
+                        ", done=" + f.isDone() + "]";
+                }
+                else
+                    return "[loc=true, done=" + f.isDone() + "]";
+            }
+        });
+
+        return S.toString(GridPartitionedGetFuture.class, this,
+            "innerFuts", futs,
+            "super", super.toString());
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/3de5f98c/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearGetFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearGetFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearGetFuture.java
index 4de8294..9d2113e 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearGetFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearGetFuture.java
@@ -291,7 +291,7 @@ public final class GridNearGetFuture<K, V> extends GridCompoundIdentityFuture<Ma
      * @param f Future.
      * @return {@code True} if mini-future.
      */
-    private boolean isMini(IgniteInternalFuture<Map<K, V>> f) {
+    private boolean isMini(IgniteInternalFuture<?> f) {
         return f.getClass().equals(MiniFuture.class);
     }
 
@@ -721,6 +721,26 @@ public final class GridNearGetFuture<K, V> extends GridCompoundIdentityFuture<Ma
         return map;
     }
 
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        Collection<String> futs = F.viewReadOnly(futures(), new C1<IgniteInternalFuture<?>, String>() {
+            @SuppressWarnings("unchecked")
+            @Override public String apply(IgniteInternalFuture<?> f) {
+                if (isMini(f)) {
+                    return "[node=" + ((MiniFuture)f).node().id() +
+                        ", loc=" + ((MiniFuture)f).node().isLocal() +
+                        ", done=" + f.isDone() + "]";
+                }
+                else
+                    return "[loc=true, done=" + f.isDone() + "]";
+            }
+        });
+
+        return S.toString(GridNearGetFuture.class, this,
+            "innerFuts", futs,
+            "super", super.toString());
+    }
+
     /**
      * Mini-future for get operations. Mini-futures are only waiting on a single
      * node as opposed to multiple nodes.

http://git-wip-us.apache.org/repos/asf/ignite/blob/3de5f98c/modules/core/src/main/java/org/apache/ignite/internal/util/lang/GridFunc.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/lang/GridFunc.java b/modules/core/src/main/java/org/apache/ignite/internal/util/lang/GridFunc.java
index a9e6cd0..ffeeca0 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/util/lang/GridFunc.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/util/lang/GridFunc.java
@@ -1658,6 +1658,7 @@ public class GridFunc {
      * @return Light-weight view on given collection with provided predicate.
      */
     @SuppressWarnings("RedundantTypeArguments")
+    @SafeVarargs
     public static <T1, T2> Collection<T2> viewReadOnly(@Nullable final Collection<? extends T1> c,
         final IgniteClosure<? super T1, T2> trans, @Nullable final IgnitePredicate<? super T1>... p) {
         A.notNull(trans, "trans");


[16/39] ignite git commit: ignite-1273: fixed cyclic references processing by PortableMarshaller and ability to modify array fields returned by PortableBuilder

Posted by ag...@apache.org.
http://git-wip-us.apache.org/repos/asf/ignite/blob/6ac9557e/modules/core/src/test/java/org/apache/ignite/internal/portable/GridPortableBuilderAdditionalSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/portable/GridPortableBuilderAdditionalSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/portable/GridPortableBuilderAdditionalSelfTest.java
index 488361c..61ec714 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/portable/GridPortableBuilderAdditionalSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/portable/GridPortableBuilderAdditionalSelfTest.java
@@ -22,6 +22,7 @@ import com.google.common.collect.Lists;
 import com.google.common.collect.Maps;
 import com.google.common.collect.Sets;
 import java.lang.reflect.Field;
+import java.math.BigDecimal;
 import java.sql.Timestamp;
 import java.util.ArrayList;
 import java.util.Arrays;
@@ -38,6 +39,8 @@ import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.IgnitePortables;
 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.PortableBuilderImpl;
 import org.apache.ignite.internal.portable.mutabletest.GridPortableMarshalerAwareTestClass;
 import org.apache.ignite.internal.processors.cache.portable.CacheObjectPortableProcessorImpl;
 import org.apache.ignite.internal.processors.cache.portable.IgnitePortablesImpl;
@@ -237,10 +240,214 @@ public class GridPortableBuilderAdditionalSelfTest extends GridCommonAbstractTes
     /**
      *
      */
-    public void testSimpleArrayModification() {
+    public void testDateArrayModification() {
         TestObjectAllTypes obj = new TestObjectAllTypes();
 
-        obj.strArr = new String[]{"a", "a", "a"};
+        obj.dateArr =  new Date[] {new Date(11111), new Date(11111), new Date(11111)};
+
+        PortableBuilderImpl mutObj = wrap(obj);
+
+        Date[] arr = mutObj.getField("dateArr");
+        arr[0] = new Date(22222);
+
+        TestObjectAllTypes res = mutObj.build().deserialize();
+
+        Assert.assertArrayEquals(new Date[] {new Date(22222), new Date(11111), new Date(11111)}, res.dateArr);
+    }
+
+    /**
+     *
+     */
+    public void testUUIDArrayModification() {
+        TestObjectAllTypes obj = new TestObjectAllTypes();
+
+        obj.uuidArr = new UUID[] {new UUID(1, 1), new UUID(1, 1), new UUID(1, 1)};
+
+        PortableBuilderImpl mutObj = wrap(obj);
+
+        UUID[] arr = mutObj.getField("uuidArr");
+        arr[0] = new UUID(2, 2);
+
+        TestObjectAllTypes res = mutObj.build().deserialize();
+
+        Assert.assertArrayEquals(new UUID[] {new UUID(2, 2), new UUID(1, 1), new UUID(1, 1)}, res.uuidArr);
+    }
+
+    /**
+     *
+     */
+    public void testDecimalArrayModification() {
+        TestObjectAllTypes obj = new TestObjectAllTypes();
+
+        obj.bdArr = new BigDecimal[] {new BigDecimal(1000), new BigDecimal(1000), new BigDecimal(1000)};
+
+        PortableBuilderImpl mutObj = wrap(obj);
+
+        BigDecimal[] arr = mutObj.getField("bdArr");
+        arr[0] = new BigDecimal(2000);
+
+        TestObjectAllTypes res = mutObj.build().deserialize();
+
+        Assert.assertArrayEquals(new BigDecimal[] {new BigDecimal(1000), new BigDecimal(1000), new BigDecimal(1000)},
+            res.bdArr);
+    }
+
+    /**
+     *
+     */
+    public void testBooleanArrayModification() {
+        TestObjectAllTypes obj = new TestObjectAllTypes();
+
+        obj.zArr = new boolean[] {false, false, false};
+
+        PortableBuilderImpl mutObj = wrap(obj);
+
+        boolean[] arr = mutObj.getField("zArr");
+        arr[0] = true;
+
+        TestObjectAllTypes res = mutObj.build().deserialize();
+
+        boolean[] expected = new boolean[] {true, false, false};
+
+        assertEquals(expected.length, res.zArr.length);
+
+        for (int i = 0; i < expected.length; i++)
+            assertEquals(expected[i], res.zArr[i]);
+    }
+
+    /**
+     *
+     */
+    public void testCharArrayModification() {
+        TestObjectAllTypes obj = new TestObjectAllTypes();
+
+        obj.cArr = new char[] {'a', 'a', 'a'};
+
+        PortableBuilderImpl mutObj = wrap(obj);
+
+        char[] arr = mutObj.getField("cArr");
+        arr[0] = 'b';
+
+        TestObjectAllTypes res = mutObj.build().deserialize();
+
+        Assert.assertArrayEquals(new char[] {'b', 'a', 'a'}, res.cArr);
+    }
+
+    /**
+     *
+     */
+    public void testDoubleArrayModification() {
+        TestObjectAllTypes obj = new TestObjectAllTypes();
+
+        obj.dArr = new double[] {1.0, 1.0, 1.0};
+
+        PortableBuilderImpl mutObj = wrap(obj);
+
+        double[] arr = mutObj.getField("dArr");
+        arr[0] = 2.0;
+
+        TestObjectAllTypes res = mutObj.build().deserialize();
+
+        Assert.assertArrayEquals(new double[] {2.0, 1.0, 1.0}, res.dArr, 0);
+    }
+
+    /**
+     *
+     */
+    public void testFloatArrayModification() {
+        TestObjectAllTypes obj = new TestObjectAllTypes();
+
+        obj.fArr = new float[] {1.0f, 1.0f, 1.0f};
+
+        PortableBuilderImpl mutObj = wrap(obj);
+
+        float[] arr = mutObj.getField("fArr");
+        arr[0] = 2.0f;
+
+        TestObjectAllTypes res = mutObj.build().deserialize();
+
+        Assert.assertArrayEquals(new float[] {2.0f, 1.0f, 1.0f}, res.fArr, 0);
+    }
+
+    /**
+     *
+     */
+    public void testLongArrayModification() {
+        TestObjectAllTypes obj = new TestObjectAllTypes();
+
+        obj.lArr = new long[] {1, 1, 1};
+
+        PortableBuilderImpl mutObj = wrap(obj);
+
+        long[] arr = mutObj.getField("lArr");
+        arr[0] = 2;
+
+        TestObjectAllTypes res = mutObj.build().deserialize();
+
+        Assert.assertArrayEquals(new long[] {2, 1, 1}, res.lArr);
+    }
+
+    /**
+     *
+     */
+    public void testIntArrayModification() {
+        TestObjectAllTypes obj = new TestObjectAllTypes();
+
+        obj.iArr = new int[] {1, 1, 1};
+
+        PortableBuilderImpl mutObj = wrap(obj);
+
+        int[] arr = mutObj.getField("iArr");
+        arr[0] = 2;
+
+        TestObjectAllTypes res = mutObj.build().deserialize();
+
+        Assert.assertArrayEquals(new int[] {2, 1, 1}, res.iArr);
+    }
+
+    /**
+     *
+     */
+    public void testShortArrayModification() {
+        TestObjectAllTypes obj = new TestObjectAllTypes();
+
+        obj.sArr = new short[] {1, 1, 1};
+
+        PortableBuilderImpl mutObj = wrap(obj);
+
+        short[] arr = mutObj.getField("sArr");
+        arr[0] = 2;
+
+        TestObjectAllTypes res = mutObj.build().deserialize();
+
+        Assert.assertArrayEquals(new short[] {2, 1, 1}, res.sArr);
+    }
+
+    /**
+     *
+     */
+    public void testByteArrayModification() {
+        TestObjectAllTypes obj = new TestObjectAllTypes();
+
+        obj.bArr = new byte[] {1, 1, 1};
+
+        PortableBuilderImpl mutObj = wrap(obj);
+
+        byte[] arr = mutObj.getField("bArr");
+        arr[0] = 2;
+
+        TestObjectAllTypes res = mutObj.build().deserialize();
+
+        Assert.assertArrayEquals(new byte[] {2, 1, 1}, res.bArr);
+    }
+
+    /**
+     *
+     */
+    public void testStringArrayModification() {
+        TestObjectAllTypes obj = new TestObjectAllTypes();
+
+        obj.strArr = new String[] {"a", "a", "a"};
 
         PortableBuilderImpl mutObj = wrap(obj);
 
@@ -249,29 +456,27 @@ public class GridPortableBuilderAdditionalSelfTest extends GridCommonAbstractTes
 
         TestObjectAllTypes res = mutObj.build().deserialize();
 
-        Assert.assertArrayEquals(obj.strArr, res.strArr);
+        Assert.assertArrayEquals(new String[] {"b", "a", "a"}, res.strArr);
     }
 
     /**
      *
      */
     public void testModifyObjectArray() {
-        fail("https://issues.apache.org/jira/browse/IGNITE-1273");
-
         TestObjectContainer obj = new TestObjectContainer();
-        obj.foo = new Object[]{"a"};
+        obj.foo = new Object[] {"a"};
 
         PortableBuilderImpl mutObj = wrap(obj);
 
         Object[] arr = mutObj.getField("foo");
 
-        Assert.assertArrayEquals(new Object[]{"a"}, arr);
+        Assert.assertArrayEquals(new Object[] {"a"}, arr);
 
         arr[0] = "b";
 
         TestObjectContainer res = mutObj.build().deserialize();
 
-        Assert.assertArrayEquals(new Object[] {"a"}, (Object[])res.foo);
+        Assert.assertArrayEquals(new Object[] {"b"}, (Object[])res.foo);
     }
 
     /**
@@ -280,7 +485,7 @@ public class GridPortableBuilderAdditionalSelfTest extends GridCommonAbstractTes
     public void testOverrideObjectArrayField() {
         PortableBuilderImpl mutObj = wrap(new TestObjectContainer());
 
-        Object[] createdArr = {mutObj, "a", 1, new String[] {"s", "s"}, new byte[]{1, 2}, new UUID(3, 0)};
+        Object[] createdArr = {mutObj, "a", 1, new String[] {"s", "s"}, new byte[] {1, 2}, new UUID(3, 0)};
 
         mutObj.setField("foo", createdArr.clone());
 
@@ -296,7 +501,7 @@ public class GridPortableBuilderAdditionalSelfTest extends GridCommonAbstractTes
      */
     public void testDeepArray() {
         TestObjectContainer obj = new TestObjectContainer();
-        obj.foo = new Object[]{new Object[]{"a", obj}};
+        obj.foo = new Object[] {new Object[] {"a", obj}};
 
         PortableBuilderImpl mutObj = wrap(obj);
 
@@ -633,7 +838,7 @@ public class GridPortableBuilderAdditionalSelfTest extends GridCommonAbstractTes
     public void testEnumArrayModification() {
         TestObjectAllTypes obj = new TestObjectAllTypes();
 
-        obj.enumArr = new TestObjectEnum[]{TestObjectEnum.A, TestObjectEnum.B};
+        obj.enumArr = new TestObjectEnum[] {TestObjectEnum.A, TestObjectEnum.B};
 
         PortableBuilderImpl mutObj = wrap(obj);
 
@@ -953,12 +1158,10 @@ public class GridPortableBuilderAdditionalSelfTest extends GridCommonAbstractTes
      *
      */
     public void testCyclicArrays() {
-        fail("https://issues.apache.org/jira/browse/IGNITE-1273");
-
         TestObjectContainer obj = new TestObjectContainer();
 
         Object[] arr1 = new Object[1];
-        Object[] arr2 = new Object[]{arr1};
+        Object[] arr2 = new Object[] {arr1};
 
         arr1[0] = arr2;
 
@@ -976,7 +1179,6 @@ public class GridPortableBuilderAdditionalSelfTest extends GridCommonAbstractTes
      */
     @SuppressWarnings("TypeMayBeWeakened")
     public void testCyclicArrayList() {
-        fail("https://issues.apache.org/jira/browse/IGNITE-1273");
         TestObjectContainer obj = new TestObjectContainer();
 
         List<Object> arr1 = new ArrayList<>();

http://git-wip-us.apache.org/repos/asf/ignite/blob/6ac9557e/modules/core/src/test/java/org/apache/ignite/internal/portable/GridPortableBuilderSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/portable/GridPortableBuilderSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/portable/GridPortableBuilderSelfTest.java
index 3ba0a92..7f23c1f 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/portable/GridPortableBuilderSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/portable/GridPortableBuilderSelfTest.java
@@ -28,6 +28,7 @@ import java.util.UUID;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.IgnitePortables;
 import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.internal.portable.builder.PortableBuilderImpl;
 import org.apache.ignite.internal.portable.mutabletest.GridPortableTestClasses.TestObjectAllTypes;
 import org.apache.ignite.internal.portable.mutabletest.GridPortableTestClasses.TestObjectContainer;
 import org.apache.ignite.internal.portable.mutabletest.GridPortableTestClasses.TestObjectInner;

http://git-wip-us.apache.org/repos/asf/ignite/blob/6ac9557e/modules/core/src/test/java/org/apache/ignite/internal/portable/GridPortableMarshallerSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/portable/GridPortableMarshallerSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/portable/GridPortableMarshallerSelfTest.java
index 11e316a..c8287a0 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/portable/GridPortableMarshallerSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/portable/GridPortableMarshallerSelfTest.java
@@ -40,6 +40,7 @@ import java.util.concurrent.Callable;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.ConcurrentSkipListSet;
 import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.internal.portable.builder.PortableBuilderImpl;
 import org.apache.ignite.internal.util.GridUnsafe;
 import org.apache.ignite.internal.util.IgniteUtils;
 import org.apache.ignite.internal.util.lang.GridMapEntry;
@@ -68,6 +69,7 @@ import sun.misc.Unsafe;
 
 import static org.apache.ignite.internal.portable.PortableThreadLocalMemoryAllocator.THREAD_LOCAL_ALLOC;
 import static org.junit.Assert.assertArrayEquals;
+import static org.junit.Assert.assertTrue;
 
 /**
  * Portable marshaller tests.
@@ -1547,11 +1549,11 @@ public class GridPortableMarshallerSelfTest extends GridCommonAbstractTest {
 
         PortableObject copy = copy(po, F.<String, Object>asMap("bArr", new byte[]{1, 2, 3}));
 
-        assertArrayEquals(new byte[]{1, 2, 3}, copy.<byte[]>field("bArr"));
+        assertArrayEquals(new byte[] {1, 2, 3}, copy.<byte[]>field("bArr"));
 
         SimpleObject obj0 = copy.deserialize();
 
-        assertArrayEquals(new byte[]{1, 2, 3}, obj0.bArr);
+        assertArrayEquals(new byte[] {1, 2, 3}, obj0.bArr);
     }
 
     /**
@@ -1796,7 +1798,7 @@ public class GridPortableMarshallerSelfTest extends GridCommonAbstractTest {
 
         assertEquals("str555", obj0.str);
         assertEquals(newObj, obj0.inner);
-        assertArrayEquals(new byte[]{6, 7, 9}, obj0.bArr);
+        assertArrayEquals(new byte[] {6, 7, 9}, obj0.bArr);
     }
 
     /**
@@ -1832,7 +1834,7 @@ public class GridPortableMarshallerSelfTest extends GridCommonAbstractTest {
         assertEquals("str555", copy.<String>field("str"));
         assertEquals(newObj, copy.<PortableObject>field("inner").deserialize());
         assertEquals((short)2323, copy.<Short>field("s").shortValue());
-        assertArrayEquals(new byte[]{6, 7, 9}, copy.<byte[]>field("bArr"));
+        assertArrayEquals(new byte[] {6, 7, 9}, copy.<byte[]>field("bArr"));
         assertEquals((byte)111, copy.<Byte>field("b").byteValue());
 
         SimpleObject obj0 = copy.deserialize();
@@ -1841,7 +1843,7 @@ public class GridPortableMarshallerSelfTest extends GridCommonAbstractTest {
         assertEquals("str555", obj0.str);
         assertEquals(newObj, obj0.inner);
         assertEquals((short)2323, obj0.s);
-        assertArrayEquals(new byte[]{6, 7, 9}, obj0.bArr);
+        assertArrayEquals(new byte[] {6, 7, 9}, obj0.bArr);
         assertEquals((byte)111, obj0.b);
     }
 
@@ -2239,6 +2241,53 @@ public class GridPortableMarshallerSelfTest extends GridCommonAbstractTest {
     }
 
     /**
+     * @throws Exception If failed.
+     */
+    public void testCyclicReferencesMarshalling() throws Exception {
+        PortableMarshaller marsh = new PortableMarshaller();
+
+        SimpleObject obj = simpleObject();
+
+        obj.bArr = obj.inner.bArr;
+        obj.cArr = obj.inner.cArr;
+        obj.boolArr = obj.inner.boolArr;
+        obj.sArr = obj.inner.sArr;
+        obj.strArr = obj.inner.strArr;
+        obj.iArr = obj.inner.iArr;
+        obj.lArr = obj.inner.lArr;
+        obj.fArr = obj.inner.fArr;
+        obj.dArr = obj.inner.dArr;
+        obj.dateArr = obj.inner.dateArr;
+        obj.uuidArr = obj.inner.uuidArr;
+        obj.objArr = obj.inner.objArr;
+        obj.bdArr = obj.inner.bdArr;
+        obj.map = obj.inner.map;
+        obj.col = obj.inner.col;
+        obj.mEntry = obj.inner.mEntry;
+
+        SimpleObject res = (SimpleObject)marshalUnmarshal(obj, marsh);
+
+        assertEquals(obj, res);
+
+        assertTrue(res.bArr == res.inner.bArr);
+        assertTrue(res.cArr == res.inner.cArr);
+        assertTrue(res.boolArr == res.inner.boolArr);
+        assertTrue(res.sArr == res.inner.sArr);
+        assertTrue(res.strArr == res.inner.strArr);
+        assertTrue(res.iArr == res.inner.iArr);
+        assertTrue(res.lArr == res.inner.lArr);
+        assertTrue(res.fArr == res.inner.fArr);
+        assertTrue(res.dArr == res.inner.dArr);
+        assertTrue(res.dateArr == res.inner.dateArr);
+        assertTrue(res.uuidArr == res.inner.uuidArr);
+        assertTrue(res.objArr == res.inner.objArr);
+        assertTrue(res.bdArr == res.inner.bdArr);
+        assertTrue(res.map == res.inner.map);
+        assertTrue(res.col == res.inner.col);
+        assertTrue(res.mEntry == res.inner.mEntry);
+    }
+
+    /**
      *
      */
     private static class ObjectWithClassFields {
@@ -2424,6 +2473,7 @@ public class GridPortableMarshallerSelfTest extends GridCommonAbstractTest {
         inner.map = new HashMap<>();
         inner.enumVal = TestEnum.A;
         inner.enumArr = new TestEnum[] {TestEnum.A, TestEnum.B};
+        inner.bdArr = new BigDecimal[] {new BigDecimal(1000), BigDecimal.ONE};
 
         inner.col.add("str1");
         inner.col.add("str2");
@@ -2433,6 +2483,8 @@ public class GridPortableMarshallerSelfTest extends GridCommonAbstractTest {
         inner.map.put(2, "str2");
         inner.map.put(3, "str3");
 
+        inner.mEntry = inner.map.entrySet().iterator().next();
+
         SimpleObject outer = new SimpleObject();
 
         outer.b = 2;
@@ -2464,6 +2516,8 @@ public class GridPortableMarshallerSelfTest extends GridCommonAbstractTest {
         outer.enumVal = TestEnum.B;
         outer.enumArr = new TestEnum[] {TestEnum.B, TestEnum.C};
         outer.inner = inner;
+        outer.bdArr = new BigDecimal[] {new BigDecimal(5000), BigDecimal.TEN};
+
 
         outer.col.add("str4");
         outer.col.add("str5");
@@ -2473,6 +2527,8 @@ public class GridPortableMarshallerSelfTest extends GridCommonAbstractTest {
         outer.map.put(5, "str5");
         outer.map.put(6, "str6");
 
+        outer.mEntry = outer.map.entrySet().iterator().next();
+
         return outer;
     }
 
@@ -2757,6 +2813,9 @@ public class GridPortableMarshallerSelfTest extends GridCommonAbstractTest {
         private Object[] objArr;
 
         /** */
+        private BigDecimal[] bdArr;
+
+        /** */
         private Collection<String> col;
 
         /** */
@@ -2769,6 +2828,9 @@ public class GridPortableMarshallerSelfTest extends GridCommonAbstractTest {
         private TestEnum[] enumArr;
 
         /** */
+        private Map.Entry<Integer, String> mEntry;
+
+        /** */
         private SimpleObject inner;
 
         /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/6ac9557e/modules/core/src/test/java/org/apache/ignite/internal/portable/GridPortableMetaDataDisabledSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/portable/GridPortableMetaDataDisabledSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/portable/GridPortableMetaDataDisabledSelfTest.java
index d329818..05df23b 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/portable/GridPortableMetaDataDisabledSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/portable/GridPortableMetaDataDisabledSelfTest.java
@@ -20,6 +20,7 @@ import java.util.Arrays;
 import org.apache.ignite.IgnitePortables;
 import org.apache.ignite.configuration.IgniteConfiguration;
 import org.apache.ignite.marshaller.portable.PortableMarshaller;
+import org.apache.ignite.portable.PortableBuilder;
 import org.apache.ignite.portable.PortableException;
 import org.apache.ignite.portable.PortableMarshalAware;
 import org.apache.ignite.portable.PortableReader;
@@ -68,9 +69,17 @@ public class GridPortableMetaDataDisabledSelfTest extends GridCommonAbstractTest
 
             portables().toPortable(new TestObject1());
             portables().toPortable(new TestObject2());
+            portables().toPortable(new TestObject3());
 
             assertEquals(0, portables().metadata(TestObject1.class).fields().size());
             assertEquals(0, portables().metadata(TestObject2.class).fields().size());
+
+            PortableBuilder bldr = portables().builder("FakeType");
+
+            bldr.setField("field1", 0).setField("field2", "value").build();
+
+            assertNull(portables().metadata("FakeType"));
+            assertNull(portables().metadata(TestObject3.class));
         }
         finally {
             stopGrid();
@@ -218,4 +227,12 @@ public class GridPortableMetaDataDisabledSelfTest extends GridCommonAbstractTest
             // No-op.
         }
     }
+
+    /**
+     */
+    @SuppressWarnings("UnusedDeclaration")
+    private static class TestObject3 {
+        /** */
+        private int field;
+    }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/6ac9557e/modules/core/src/test/java/org/apache/ignite/internal/portable/GridPortableMetaDataSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/portable/GridPortableMetaDataSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/portable/GridPortableMetaDataSelfTest.java
index f6d7627..9054297 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/portable/GridPortableMetaDataSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/portable/GridPortableMetaDataSelfTest.java
@@ -147,6 +147,15 @@ public class GridPortableMetaDataSelfTest extends GridCommonAbstractTest {
     /**
      * @throws Exception If failed.
      */
+    public void testNoConfiguration() throws Exception {
+        portables().toPortable(new TestObject3());
+
+        assertNotNull(portables().metadata(TestObject3.class));
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
     public void testReflection() throws Exception {
         PortableMetadata meta = portables().metadata(TestObject1.class);
 
@@ -349,4 +358,12 @@ public class GridPortableMetaDataSelfTest extends GridCommonAbstractTest {
             // No-op.
         }
     }
+
+    /**
+     */
+    @SuppressWarnings("UnusedDeclaration")
+    private static class TestObject3 {
+        /** */
+        private int intVal;
+    }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/6ac9557e/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 ce97a8d..e49514b 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
@@ -23,6 +23,7 @@ import java.io.IOException;
 import java.io.ObjectOutput;
 import java.io.ObjectOutputStream;
 import java.io.Serializable;
+import java.math.BigDecimal;
 import java.util.ArrayList;
 import java.util.Date;
 import java.util.List;
@@ -182,7 +183,6 @@ public class GridPortableTestClasses {
         /** */
         public Date date;
 
-
         /** */
         public byte[] bArr;
 
@@ -208,12 +208,18 @@ public class GridPortableTestClasses {
         public boolean[] zArr;
 
         /** */
+        public BigDecimal[] bdArr;
+
+        /** */
         public String[] strArr;
 
         /** */
         public UUID[] uuidArr;
 
         /** */
+        public Date[] dateArr;
+
+        /** */
         public TestObjectEnum anEnum;
 
         /** */
@@ -222,8 +228,6 @@ public class GridPortableTestClasses {
         /** */
         public Map.Entry entry;
 
-        //public Date[] dateArr; // todo test date array.
-
         /**
          * @return Array.
          */
@@ -270,21 +274,23 @@ public class GridPortableTestClasses {
             uuid = new UUID(1, 1);
             date = new Date(1000000);
 
-            bArr = new byte[]{1, 2, 3};
-            sArr = new short[]{1, 2, 3};
-            iArr = new int[]{1, 2, 3};
-            lArr = new long[]{1, 2, 3};
-            fArr = new float[]{1, 2, 3};
-            dArr = new double[]{1, 2, 3};
-            cArr = new char[]{1, 2, 3};
-            zArr = new boolean[]{true, false};
+            bArr = new byte[] {1, 2, 3};
+            sArr = new short[] {1, 2, 3};
+            iArr = new int[] {1, 2, 3};
+            lArr = new long[] {1, 2, 3};
+            fArr = new float[] {1, 2, 3};
+            dArr = new double[] {1, 2, 3};
+            cArr = new char[] {1, 2, 3};
+            zArr = new boolean[] {true, false};
 
-            strArr = new String[]{"abc", "ab", "a"};
-            uuidArr = new UUID[]{new UUID(1, 1), new UUID(2, 2)};
+            strArr = new String[] {"abc", "ab", "a"};
+            uuidArr = new UUID[] {new UUID(1, 1), new UUID(2, 2)};
+            bdArr = new BigDecimal[] {new BigDecimal(1000), BigDecimal.TEN};
+            dateArr = new Date[] {new Date(1000000), new Date(200000)};
 
             anEnum = TestObjectEnum.A;
 
-            enumArr = new TestObjectEnum[]{TestObjectEnum.B};
+            enumArr = new TestObjectEnum[] {TestObjectEnum.B};
 
             entry = new GridMapEntry<>(1, "a");
         }
@@ -388,7 +394,6 @@ public class GridPortableTestClasses {
         private Map<String, List<Company>> companyByStreet = new TreeMap<>();
 
         /**
-         *
          * @param street Street.
          * @return Company.
          */
@@ -397,7 +402,6 @@ public class GridPortableTestClasses {
         }
 
         /**
-         *
          * @param company Company.
          */
         public void addCompany(Company company) {
@@ -413,7 +417,6 @@ public class GridPortableTestClasses {
         }
 
         /**
-         *
          * @return map
          */
         public Map<String, List<Company>> getCompanyByStreet() {
@@ -421,7 +424,6 @@ public class GridPortableTestClasses {
         }
 
         /**
-         *
          * @param companyByStreet map
          */
         public void setCompanyByStreet(Map<String, List<Company>> companyByStreet) {

http://git-wip-us.apache.org/repos/asf/ignite/blob/6ac9557e/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/GridCacheClientNodePortableMetadataMultinodeTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/GridCacheClientNodePortableMetadataMultinodeTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/GridCacheClientNodePortableMetadataMultinodeTest.java
index aa67574..1ba3d4d 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/GridCacheClientNodePortableMetadataMultinodeTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/GridCacheClientNodePortableMetadataMultinodeTest.java
@@ -31,6 +31,7 @@ import org.apache.ignite.IgnitePortables;
 import org.apache.ignite.configuration.CacheConfiguration;
 import org.apache.ignite.configuration.IgniteConfiguration;
 import org.apache.ignite.internal.IgniteInternalFuture;
+import org.apache.ignite.internal.util.lang.GridAbsPredicate;
 import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.marshaller.portable.PortableMarshaller;
 import org.apache.ignite.portable.PortableBuilder;
@@ -233,6 +234,16 @@ public class GridCacheClientNodePortableMetadataMultinodeTest extends GridCommon
 
             portables = ignite(i).portables();
 
+            final IgnitePortables p0 = portables;
+
+            GridTestUtils.waitForCondition(new GridAbsPredicate() {
+                @Override public boolean apply() {
+                    Collection<PortableMetadata> metaCol = p0.metadata();
+
+                    return metaCol.size() == 1000;
+                }
+            }, getTestTimeout());
+
             Collection<PortableMetadata> metaCol = portables.metadata();
 
             assertEquals(1000, metaCol.size());


[08/39] ignite git commit: Fixed test.

Posted by ag...@apache.org.
Fixed test.


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

Branch: refs/heads/ignite-264
Commit: ba756cdd64bb6e9b1e678765245beea4e1181a98
Parents: 2c236ac
Author: sboikov <sb...@gridgain.com>
Authored: Wed Sep 2 13:40:53 2015 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Wed Sep 2 13:46:06 2015 +0300

----------------------------------------------------------------------
 .../cache/CacheAffinityCallSelfTest.java        | 85 ++++++++++++++------
 1 file changed, 62 insertions(+), 23 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/ba756cdd/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheAffinityCallSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheAffinityCallSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheAffinityCallSelfTest.java
index 78d16f8..c25bc7c 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheAffinityCallSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheAffinityCallSelfTest.java
@@ -21,12 +21,15 @@ import java.util.concurrent.Callable;
 import org.apache.ignite.Ignite;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.IgniteCompute;
-import org.apache.ignite.IgniteException;
 import org.apache.ignite.cluster.ClusterGroupEmptyException;
+import org.apache.ignite.cluster.ClusterNode;
 import org.apache.ignite.compute.ComputeTaskCancelledException;
 import org.apache.ignite.configuration.CacheConfiguration;
 import org.apache.ignite.configuration.IgniteConfiguration;
 import org.apache.ignite.internal.IgniteInternalFuture;
+import org.apache.ignite.internal.IgniteKernal;
+import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
+import org.apache.ignite.internal.processors.affinity.GridAffinityProcessor;
 import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.lang.IgniteCallable;
 import org.apache.ignite.lang.IgniteRunnable;
@@ -78,9 +81,12 @@ public class CacheAffinityCallSelfTest extends GridCommonAbstractTest {
 
         cfg.setCacheConfiguration(ccfg);
 
-        if (gridName.equals(getTestGridName(SERVERS_COUNT)))
+        if (gridName.equals(getTestGridName(SERVERS_COUNT))) {
             cfg.setClientMode(true);
 
+            spi.setForceServerMode(true);
+        }
+
         return cfg;
     }
 
@@ -93,21 +99,35 @@ public class CacheAffinityCallSelfTest extends GridCommonAbstractTest {
      * @throws Exception If failed.
      */
     public void testAffinityCallRestartNode() throws Exception {
-        startGrids(4);
+        startGridsMultiThreaded(SERVERS_COUNT);
 
-        Integer key = primaryKey(grid(0).cache(CACHE_NAME));
+        final int ITERS = 5;
 
-        IgniteInternalFuture<Object> fut = GridTestUtils.runAsync(new Callable<Object>() {
-            @Override public Object call() throws Exception {
-                U.sleep(500);
-                stopGrid(0);
+        for (int i = 0; i < ITERS; i++) {
+            log.info("Iteration: " + i);
 
-                return null;
-            }
-        });
+            Integer key = primaryKey(grid(0).cache(CACHE_NAME));
+
+            long topVer = grid(0).cluster().topologyVersion();
+
+            IgniteInternalFuture<Object> fut = GridTestUtils.runAsync(new Callable<Object>() {
+                @Override public Object call() throws Exception {
+                    U.sleep(500);
+
+                    stopGrid(0);
+
+                    return null;
+                }
+            });
+
+            while (!fut.isDone())
+                grid(1).compute().affinityCall(CACHE_NAME, key, new CheckCallable(key, topVer, topVer + 1));
 
-        while (!fut.isDone())
-            grid(1).compute().affinityCall(CACHE_NAME, key, new CheckCallable(key));
+            fut.get();
+
+            if (i < ITERS - 1)
+                startGrid(0);
+        }
 
         stopAllGrids();
     }
@@ -116,12 +136,14 @@ public class CacheAffinityCallSelfTest extends GridCommonAbstractTest {
      * @throws Exception If failed.
      */
     public void testAffinityCallNoServerNode() throws Exception {
-        startGrids(SERVERS_COUNT + 1);
+        startGridsMultiThreaded(SERVERS_COUNT + 1);
 
         final Integer key = 1;
 
         final Ignite client = grid(SERVERS_COUNT);
 
+        assertTrue(client.configuration().isClientMode());
+
         final IgniteInternalFuture<Object> fut = GridTestUtils.runAsync(new Callable<Object>() {
             @Override public Object call() throws Exception {
                 for (int i = 0; i < SERVERS_COUNT; ++i)
@@ -136,15 +158,10 @@ public class CacheAffinityCallSelfTest extends GridCommonAbstractTest {
                 client.compute().affinityCall(CACHE_NAME, key, new CheckCallable(key));
         }
         catch (ComputeTaskCancelledException e) {
-            assertTrue(e.getMessage().contains("stopping"));
+            assertTrue("Unexpected error: " + e, e.getMessage().contains("stopping"));
         }
         catch(ClusterGroupEmptyException e) {
-            assertTrue(e.getMessage().contains("Topology projection is empty"));
-        }
-        catch(IgniteException e) {
-            assertTrue(e.getMessage().contains("Client node disconnected") ||
-                e.getMessage().contains("Failed to reconnect to cluster") ||
-                e.getMessage().contains("Failed to execute task, client node disconnected."));
+            assertTrue("Unexpected error: " + e, e.getMessage().contains("Topology projection is empty"));
         }
 
         stopAllGrids();
@@ -161,16 +178,38 @@ public class CacheAffinityCallSelfTest extends GridCommonAbstractTest {
         @IgniteInstanceResource
         private Ignite ignite;
 
+        /** */
+        private long[] topVers;
+
         /**
          * @param key Key.
+         * @param topVers Topology versions to check.
          */
-        public CheckCallable(Object key) {
+        public CheckCallable(Object key, long... topVers) {
             this.key = key;
+            this.topVers = topVers;
         }
 
         /** {@inheritDoc} */
         @Override public Object call() throws IgniteCheckedException {
-            assert ignite.cluster().localNode().id().equals(ignite.cluster().mapKeyToNode(CACHE_NAME, key).id());
+            if (topVers.length > 0) {
+                boolean pass = false;
+
+                GridCacheAffinityManager aff =
+                    ((IgniteKernal)ignite).context().cache().internalCache(CACHE_NAME).context().affinity();
+
+                ClusterNode loc = ignite.cluster().localNode();
+
+                for (long topVer : topVers) {
+                    if (loc.equals(aff.primary(key, new AffinityTopologyVersion(topVer, 0)))) {
+                        pass = true;
+
+                        break;
+                    }
+                }
+
+                assertTrue(pass);
+            }
 
             return null;
         }


[37/39] ignite git commit: Log4j2LoggerSelfTest.testLogFilesTwoNodes test case for Windows.

Posted by ag...@apache.org.
Log4j2LoggerSelfTest.testLogFilesTwoNodes test case for Windows.


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

Branch: refs/heads/ignite-264
Commit: 8901575e14f7334874d630ae685db624d00dcc7f
Parents: 154f185
Author: Artem Shutak <ar...@gmail.com>
Authored: Thu Sep 3 18:57:47 2015 +0300
Committer: nikolay_tikhonov <nt...@gridgain.com>
Committed: Thu Sep 3 18:57:47 2015 +0300

----------------------------------------------------------------------
 .../ignite/logger/log4j2/Log4J2Logger.java      | 20 +++++++++++++++++---
 1 file changed, 17 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/8901575e/modules/log4j2/src/main/java/org/apache/ignite/logger/log4j2/Log4J2Logger.java
----------------------------------------------------------------------
diff --git a/modules/log4j2/src/main/java/org/apache/ignite/logger/log4j2/Log4J2Logger.java b/modules/log4j2/src/main/java/org/apache/ignite/logger/log4j2/Log4J2Logger.java
index 5e6ab34..ffe8e1b 100644
--- a/modules/log4j2/src/main/java/org/apache/ignite/logger/log4j2/Log4J2Logger.java
+++ b/modules/log4j2/src/main/java/org/apache/ignite/logger/log4j2/Log4J2Logger.java
@@ -242,10 +242,10 @@ public class Log4J2Logger implements IgniteLogger, LoggerNodeIdAware {
                             Appender innerApp = control.getAppender();
 
                             if (innerApp instanceof FileAppender)
-                                return ((FileAppender)innerApp).getFileName();
+                                return normilize(((FileAppender)innerApp).getFileName());
 
                             if (innerApp instanceof RollingFileAppender)
-                                return ((RollingFileAppender)innerApp).getFileName();
+                                return normilize(((RollingFileAppender)innerApp).getFileName());
                         }
                     }
                     catch (IllegalAccessException | NoSuchFieldException e) {
@@ -259,6 +259,20 @@ public class Log4J2Logger implements IgniteLogger, LoggerNodeIdAware {
     }
 
     /**
+     * Normalizes given path for windows.
+     * Log4j2 doesn't replace unix directory delimiters which used at 'fileName' to windows.
+     *
+     * @param path Path.
+     * @return Normalized path.
+     */
+    private String normilize(String path) {
+        if (!U.isWindows())
+            return path;
+
+        return path.replace('/', File.separatorChar);
+    }
+
+    /**
      * Adds console appender when needed with some default logging settings.
      *
      * @param initLogClo Optional log implementation init closure.
@@ -495,4 +509,4 @@ public class Log4J2Logger implements IgniteLogger, LoggerNodeIdAware {
     @Override public String toString() {
         return S.toString(Log4J2Logger.class, this);
     }
-}
\ No newline at end of file
+}


[03/39] ignite git commit: ignite-993 Update registeredCaches map only from discovery thread

Posted by ag...@apache.org.
ignite-993 Update registeredCaches map only from discovery thread


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

Branch: refs/heads/ignite-264
Commit: 3321645392e25a2d0ed5e469de57d6d1a2dc173d
Parents: c1b537f
Author: sboikov <sb...@gridgain.com>
Authored: Tue Sep 1 15:02:33 2015 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Tue Sep 1 15:21:54 2015 +0300

----------------------------------------------------------------------
 .../discovery/GridDiscoveryManager.java         |  50 +++--
 .../cache/DynamicCacheChangeRequest.java        |  17 ++
 .../cache/DynamicCacheDescriptor.java           |  14 --
 .../GridCachePartitionExchangeManager.java      |   2 +-
 .../processors/cache/GridCacheProcessor.java    | 187 +++++++++----------
 .../cache/IgniteDynamicCacheStartSelfTest.java  |   2 -
 6 files changed, 137 insertions(+), 135 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/33216453/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManager.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManager.java
index fac6f6d..ce5aca9 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManager.java
@@ -245,7 +245,9 @@ public class GridDiscoveryManager extends GridManagerAdapter<DiscoverySpi> {
      * @param cacheName Cache name.
      */
     public void removeCacheFilter(String cacheName) {
-        registeredCaches.remove(cacheName);
+        CachePredicate p = registeredCaches.remove(cacheName);
+
+        assert p != null : cacheName;
     }
 
     /**
@@ -254,12 +256,14 @@ public class GridDiscoveryManager extends GridManagerAdapter<DiscoverySpi> {
      * @param cacheName Cache name.
      * @param clientNodeId Near node ID.
      * @param nearEnabled Near enabled flag.
+     * @return {@code True} if new node ID was added.
      */
-    public void addClientNode(String cacheName, UUID clientNodeId, boolean nearEnabled) {
-        CachePredicate pred = registeredCaches.get(cacheName);
+    public boolean addClientNode(String cacheName, UUID clientNodeId, boolean nearEnabled) {
+        CachePredicate p = registeredCaches.get(cacheName);
 
-        if (pred != null)
-            pred.addClientNode(clientNodeId, nearEnabled);
+        assert p != null : cacheName;
+
+        return p.addClientNode(clientNodeId, nearEnabled);
     }
 
     /**
@@ -267,12 +271,14 @@ public class GridDiscoveryManager extends GridManagerAdapter<DiscoverySpi> {
      *
      * @param cacheName Cache name.
      * @param clientNodeId Near node ID.
+     * @return {@code True} if existing node ID was removed.
      */
-    public void onClientCacheClose(String cacheName, UUID clientNodeId) {
-        CachePredicate predicate = registeredCaches.get(cacheName);
+    public boolean onClientCacheClose(String cacheName, UUID clientNodeId) {
+        CachePredicate p = registeredCaches.get(cacheName);
+
+        assert p != null : cacheName;
 
-        if (predicate != null)
-            predicate.onNodeLeft(clientNodeId);
+        return p.onNodeLeft(clientNodeId);
     }
 
     /**
@@ -2621,16 +2627,16 @@ public class GridDiscoveryManager extends GridManagerAdapter<DiscoverySpi> {
      */
     private static class CachePredicate {
         /** Cache filter. */
-        private IgnitePredicate<ClusterNode> cacheFilter;
+        private final IgnitePredicate<ClusterNode> cacheFilter;
 
         /** If near cache is enabled on data nodes. */
-        private boolean nearEnabled;
+        private final boolean nearEnabled;
 
         /** Flag indicating if cache is local. */
-        private boolean loc;
+        private final boolean loc;
 
         /** Collection of client near nodes. */
-        private ConcurrentHashMap<UUID, Boolean> clientNodes;
+        private final ConcurrentHashMap<UUID, Boolean> clientNodes;
 
         /**
          * @param cacheFilter Cache filter.
@@ -2650,16 +2656,26 @@ public class GridDiscoveryManager extends GridManagerAdapter<DiscoverySpi> {
         /**
          * @param nodeId Near node ID to add.
          * @param nearEnabled Near enabled flag.
+         * @return {@code True} if new node ID was added.
          */
-        public void addClientNode(UUID nodeId, boolean nearEnabled) {
-            clientNodes.putIfAbsent(nodeId, nearEnabled);
+        public boolean addClientNode(UUID nodeId, boolean nearEnabled) {
+            assert nodeId != null;
+
+            Boolean old = clientNodes.putIfAbsent(nodeId, nearEnabled);
+
+            return old == null;
         }
 
         /**
          * @param leftNodeId Left node ID.
+         * @return {@code True} if existing node ID was removed.
          */
-        public void onNodeLeft(UUID leftNodeId) {
-            clientNodes.remove(leftNodeId);
+        public boolean onNodeLeft(UUID leftNodeId) {
+            assert leftNodeId != null;
+
+            Boolean old = clientNodes.remove(leftNodeId);
+
+            return old != null;
         }
 
         /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/33216453/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/DynamicCacheChangeRequest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/DynamicCacheChangeRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/DynamicCacheChangeRequest.java
index 7af1572..2029a95 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/DynamicCacheChangeRequest.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/DynamicCacheChangeRequest.java
@@ -66,6 +66,9 @@ public class DynamicCacheChangeRequest implements Serializable {
     /** Template configuration flag. */
     private boolean template;
 
+    /** */
+    private transient boolean exchangeNeeded;
+
     /**
      * Constructor creates cache stop request.
      *
@@ -78,6 +81,20 @@ public class DynamicCacheChangeRequest implements Serializable {
     }
 
     /**
+     * @return {@code True} if request should trigger partition exchange.
+     */
+    public boolean exchangeNeeded() {
+        return exchangeNeeded;
+    }
+
+    /**
+     * @param exchangeNeeded {@code True} if request should trigger partition exchange.
+     */
+    public void exchangeNeeded(boolean exchangeNeeded) {
+        this.exchangeNeeded = exchangeNeeded;
+    }
+
+    /**
      * @param template {@code True} if this is request for adding template configuration.
      */
     public void template(boolean template) {

http://git-wip-us.apache.org/repos/asf/ignite/blob/33216453/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/DynamicCacheDescriptor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/DynamicCacheDescriptor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/DynamicCacheDescriptor.java
index 9c6cc43..f68e920 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/DynamicCacheDescriptor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/DynamicCacheDescriptor.java
@@ -168,20 +168,6 @@ public class DynamicCacheDescriptor {
     }
 
     /**
-     * Sets cancelled flag.
-     */
-    public void onCancelled() {
-        cancelled = true;
-    }
-
-    /**
-     * @return Cancelled flag.
-     */
-    public boolean cancelled() {
-        return cancelled;
-    }
-
-    /**
      * @param nodeId Remote node ID.
      * @return Configuration.
      */

http://git-wip-us.apache.org/repos/asf/ignite/blob/33216453/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePartitionExchangeManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePartitionExchangeManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePartitionExchangeManager.java
index 4398b4c..5b6f750 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePartitionExchangeManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePartitionExchangeManager.java
@@ -156,7 +156,7 @@ public class GridCachePartitionExchangeManager<K, V> extends GridCacheSharedMana
 
                         // Validate requests to check if event should trigger partition exchange.
                         for (DynamicCacheChangeRequest req : batch.requests()) {
-                            if (cctx.cache().exchangeNeeded(req))
+                            if (req.exchangeNeeded())
                                 valid.add(req);
                             else
                                 cctx.cache().completeStartFuture(req);

http://git-wip-us.apache.org/repos/asf/ignite/blob/33216453/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 f13af23..c1d0d17 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
@@ -1405,36 +1405,6 @@ public class GridCacheProcessor extends GridProcessorAdapter {
     }
 
     /**
-     * @param req Request to check.
-     * @return {@code True} if change request was registered to apply.
-     */
-    @SuppressWarnings("IfMayBeConditional")
-    public boolean exchangeNeeded(DynamicCacheChangeRequest req) {
-        DynamicCacheDescriptor desc = registeredCaches.get(maskNull(req.cacheName()));
-
-        if (desc != null) {
-            if (req.close()) {
-                assert req.initiatingNodeId() != null : req;
-
-                return true;
-            }
-
-            if (desc.deploymentId().equals(req.deploymentId())) {
-                if (req.start())
-                    return !desc.cancelled();
-                else
-                    return desc.cancelled();
-            }
-
-            // If client requested cache start
-            if (req.initiatingNodeId() != null)
-                return true;
-        }
-
-        return false;
-    }
-
-    /**
      * @param reqs Requests to start.
      * @param topVer Topology version.
      * @throws IgniteCheckedException If failed to start cache.
@@ -1622,11 +1592,6 @@ public class GridCacheProcessor extends GridProcessorAdapter {
                     stopGateway(req);
 
                     prepareCacheStop(req);
-
-                    DynamicCacheDescriptor desc = registeredCaches.get(masked);
-
-                    if (desc != null && desc.cancelled() && desc.deploymentId().equals(req.deploymentId()))
-                        registeredCaches.remove(masked, desc);
                 }
                 else if (req.close() && req.initiatingNodeId().equals(ctx.localNodeId())) {
                     IgniteCacheProxy<?, ?> proxy = jCacheProxies.remove(masked);
@@ -1709,17 +1674,15 @@ public class GridCacheProcessor extends GridProcessorAdapter {
             new ArrayList<>(registeredCaches.size() + registeredTemplates.size());
 
         for (DynamicCacheDescriptor desc : registeredCaches.values()) {
-            if (!desc.cancelled()) {
-                DynamicCacheChangeRequest req = new DynamicCacheChangeRequest(desc.cacheConfiguration().getName(), null);
+            DynamicCacheChangeRequest req = new DynamicCacheChangeRequest(desc.cacheConfiguration().getName(), null);
 
-                req.startCacheConfiguration(desc.cacheConfiguration());
+            req.startCacheConfiguration(desc.cacheConfiguration());
 
-                req.cacheType(desc.cacheType());
+            req.cacheType(desc.cacheType());
 
-                req.deploymentId(desc.deploymentId());
+            req.deploymentId(desc.deploymentId());
 
-                reqs.add(req);
-            }
+            reqs.add(req);
         }
 
         for (DynamicCacheDescriptor desc : registeredTemplates.values()) {
@@ -1980,10 +1943,11 @@ public class GridCacheProcessor extends GridProcessorAdapter {
                 return new GridFinishedFuture<>(e);
             }
 
-            if (desc != null && !desc.cancelled()) {
-                if (failIfExists)
+            if (desc != null) {
+                if (failIfExists) {
                     return new GridFinishedFuture<>(new CacheExistsException("Failed to start cache " +
                         "(a cache with the same name is already started): " + cacheName));
+                }
                 else {
                     CacheConfiguration descCfg = desc.cacheConfiguration();
 
@@ -2029,7 +1993,7 @@ public class GridCacheProcessor extends GridProcessorAdapter {
         else {
             req.clientStartOnly(true);
 
-            if (desc != null && !desc.cancelled())
+            if (desc != null)
                 ccfg = desc.cacheConfiguration();
 
             if (ccfg == null) {
@@ -2212,83 +2176,104 @@ public class GridCacheProcessor extends GridProcessorAdapter {
 
             DynamicCacheDescriptor desc = registeredCaches.get(maskNull(req.cacheName()));
 
-            if (req.start()) {
-                CacheConfiguration ccfg = req.startCacheConfiguration();
+            boolean needExchange = false;
 
-                DynamicCacheStartFuture startFut = (DynamicCacheStartFuture)pendingFuts.get(
-                    maskNull(ccfg.getName()));
+            DynamicCacheStartFuture fut = null;
 
-                // Check if cache with the same name was concurrently started form different node.
-                if (desc != null) {
-                    if (!req.clientStartOnly() && req.failIfExists()) {
-                        // If local node initiated start, fail the start future.
-                        if (startFut != null && startFut.deploymentId().equals(req.deploymentId())) {
-                            startFut.onDone(new CacheExistsException("Failed to start cache " +
-                                "(a cache with the same name is already started): " + U.maskName(ccfg.getName())));
-                        }
+            if (ctx.localNodeId().equals(req.initiatingNodeId())) {
+                fut = (DynamicCacheStartFuture)pendingFuts.get(maskNull(req.cacheName()));
 
-                        return;
+                if (!req.deploymentId().equals(fut.deploymentId()))
+                    fut = null;
+            }
+
+            if (req.start()) {
+                if (desc == null) {
+                    if (req.clientStartOnly()) {
+                        if (fut != null)
+                            fut.onDone(new IgniteCheckedException("Failed to start client cache " +
+                                "(a cache with the given name is not started): " + U.maskName(req.cacheName())));
                     }
+                    else {
+                        CacheConfiguration ccfg = req.startCacheConfiguration();
+
+                        assert req.cacheType() != null : req;
+                        assert F.eq(ccfg.getName(), req.cacheName()) : req;
 
-                    req.clientStartOnly(true);
+                        DynamicCacheDescriptor startDesc =
+                            new DynamicCacheDescriptor(ctx, ccfg, req.cacheType(), false, req.deploymentId());
+
+                        DynamicCacheDescriptor old = registeredCaches.put(maskNull(ccfg.getName()), startDesc);
+
+                        assert old == null :
+                            "Dynamic cache map was concurrently modified [new=" + startDesc + ", old=" + old + ']';
+
+                        ctx.discovery().setCacheFilter(
+                            ccfg.getName(),
+                            ccfg.getNodeFilter(),
+                            ccfg.getNearConfiguration() != null,
+                            ccfg.getCacheMode() == LOCAL);
+
+                        ctx.discovery().addClientNode(req.cacheName(),
+                            req.initiatingNodeId(),
+                            req.nearCacheConfiguration() != null);
+
+                        needExchange = true;
+                    }
                 }
                 else {
                     if (req.clientStartOnly()) {
-                        if (startFut != null && startFut.deploymentId().equals(req.deploymentId())) {
-                            startFut.onDone(new IgniteCheckedException("Failed to start client cache " +
-                                "(a cache with the given name is not started): " + U.maskName(ccfg.getName())));
-                        }
+                        assert req.initiatingNodeId() != null : req;
 
-                        return;
+                        needExchange = ctx.discovery().addClientNode(req.cacheName(),
+                            req.initiatingNodeId(),
+                            req.nearCacheConfiguration() != null);
                     }
-                }
-
-                if (!req.clientStartOnly() && desc == null) {
-                    assert req.cacheType() != null : req;
-
-                    DynamicCacheDescriptor startDesc =
-                        new DynamicCacheDescriptor(ctx, ccfg, req.cacheType(), false, req.deploymentId());
+                    else {
+                        if (req.failIfExists() ) {
+                            if (fut != null)
+                                fut.onDone(new CacheExistsException("Failed to start cache " +
+                                    "(a cache with the same name is already started): " + U.maskName(req.cacheName())));
+                        }
+                        else {
+                            // Cache already exists, exchange is needed only if client cache should be created.
+                            ClusterNode node = ctx.discovery().node(req.initiatingNodeId());
 
-                    DynamicCacheDescriptor old = registeredCaches.put(maskNull(ccfg.getName()), startDesc);
+                            boolean clientReq = node != null &&
+                                !ctx.discovery().cacheAffinityNode(node, req.cacheName());
 
-                    assert old == null :
-                        "Dynamic cache map was concurrently modified [new=" + startDesc + ", old=" + old + ']';
+                            needExchange = clientReq && ctx.discovery().addClientNode(req.cacheName(),
+                                req.initiatingNodeId(),
+                                req.nearCacheConfiguration() != null);
 
-                    ctx.discovery().setCacheFilter(
-                        ccfg.getName(),
-                        ccfg.getNodeFilter(),
-                        ccfg.getNearConfiguration() != null,
-                        ccfg.getCacheMode() == LOCAL);
+                            if (needExchange)
+                                req.clientStartOnly(true);
+                        }
+                    }
                 }
-
-                ctx.discovery().addClientNode(req.cacheName(),
-                    req.initiatingNodeId(),
-                    req.nearCacheConfiguration() != null);
             }
             else {
-                assert req.stop() || req.close() : req;
+                assert req.stop() ^ req.close() : req;
 
-                if (desc == null) {
-                    // If local node initiated start, finish future.
-                    DynamicCacheStartFuture changeFut =
-                        (DynamicCacheStartFuture)pendingFuts.get(maskNull(req.cacheName()));
+                if (desc != null) {
+                    if (req.stop()) {
+                        DynamicCacheDescriptor old = registeredCaches.remove(maskNull(req.cacheName()));
 
-                    if (changeFut != null && changeFut.deploymentId().equals(req.deploymentId())) {
-                        // No-op.
-                        changeFut.onDone();
-                    }
+                        assert old != null : "Dynamic cache map was concurrently modified [req=" + req + ']';
 
-                    return;
-                }
+                        ctx.discovery().removeCacheFilter(req.cacheName());
 
-                if (req.stop()) {
-                    desc.onCancelled();
+                        needExchange = true;
+                    }
+                    else {
+                        assert req.close() : req;
 
-                    ctx.discovery().removeCacheFilter(req.cacheName());
+                        needExchange = ctx.discovery().onClientCacheClose(req.cacheName(), req.initiatingNodeId());
+                    }
                 }
-                else
-                    ctx.discovery().onClientCacheClose(req.cacheName(), req.initiatingNodeId());
             }
+
+            req.exchangeNeeded(needExchange);
         }
     }
 
@@ -2711,7 +2696,7 @@ public class GridCacheProcessor extends GridProcessorAdapter {
 
         DynamicCacheDescriptor desc = registeredCaches.get(maskNull(name));
 
-        if (desc == null || desc.cancelled())
+        if (desc == null)
             throw new IllegalArgumentException("Cache is not started: " + name);
 
         if (!desc.cacheType().userCache())
@@ -2779,7 +2764,7 @@ public class GridCacheProcessor extends GridProcessorAdapter {
     public CacheConfiguration cacheConfiguration(String name) {
         DynamicCacheDescriptor desc = registeredCaches.get(maskNull(name));
 
-        if (desc == null || desc.cancelled())
+        if (desc == null)
             throw new IllegalStateException("Cache doesn't exist: " + name);
         else
             return desc.cacheConfiguration();

http://git-wip-us.apache.org/repos/asf/ignite/blob/33216453/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteDynamicCacheStartSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteDynamicCacheStartSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteDynamicCacheStartSelfTest.java
index d1f8016..95f7701 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteDynamicCacheStartSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteDynamicCacheStartSelfTest.java
@@ -1118,8 +1118,6 @@ public class IgniteDynamicCacheStartSelfTest extends GridCommonAbstractTest {
      * @throws Exception If failed.
      */
     public void testStartStopSameCacheMultinode() throws Exception {
-        fail("https://issues.apache.org/jira/browse/IGNITE-993");
-
         final AtomicInteger idx = new AtomicInteger();
 
         IgniteInternalFuture<?> fut = GridTestUtils.runMultiThreadedAsync(new Callable<Object>() {


[12/39] ignite git commit: Added test.

Posted by ag...@apache.org.
Added test.


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

Branch: refs/heads/ignite-264
Commit: 67699564d594647236552450d362be1e04c3d476
Parents: 3aa9ea3
Author: sboikov <sb...@gridgain.com>
Authored: Wed Sep 2 15:25:15 2015 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Wed Sep 2 15:34:57 2015 +0300

----------------------------------------------------------------------
 .../CachePutAllFailoverAbstractTest.java        | 110 ++++++++++++++-----
 ...gniteCachePutRetryTransactionalSelfTest.java |  42 -------
 .../junits/common/GridCommonAbstractTest.java   |  41 +++++++
 3 files changed, 123 insertions(+), 70 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/67699564/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/CachePutAllFailoverAbstractTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/CachePutAllFailoverAbstractTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/CachePutAllFailoverAbstractTest.java
index 62fddda..f558ba0 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/CachePutAllFailoverAbstractTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/CachePutAllFailoverAbstractTest.java
@@ -17,7 +17,15 @@
 
 package org.apache.ignite.internal.processors.cache.distributed;
 
+import java.io.Serializable;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.TreeMap;
+import java.util.concurrent.Callable;
+import java.util.concurrent.ThreadLocalRandom;
+import java.util.concurrent.atomic.AtomicBoolean;
 import org.apache.ignite.IgniteCache;
+import org.apache.ignite.cache.CacheAtomicityMode;
 import org.apache.ignite.cache.CacheMode;
 import org.apache.ignite.configuration.CacheConfiguration;
 import org.apache.ignite.configuration.NearCacheConfiguration;
@@ -29,14 +37,6 @@ import org.apache.ignite.lang.IgniteFuture;
 import org.apache.ignite.testframework.GridTestUtils;
 import org.jetbrains.annotations.NotNull;
 
-import java.io.Serializable;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.TreeMap;
-import java.util.concurrent.Callable;
-import java.util.concurrent.ThreadLocalRandom;
-import java.util.concurrent.atomic.AtomicBoolean;
-
 import static org.apache.ignite.cache.CacheMode.PARTITIONED;
 
 /**
@@ -85,21 +85,31 @@ public abstract class CachePutAllFailoverAbstractTest extends GridCacheAbstractS
      * @throws Exception If failed.
      */
     public void testPutAllFailover() throws Exception {
-        testPutAllFailover(false);
+        testPutAllFailover(Test.PUT_ALL);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testPutAllFailoverPessimisticTx() throws Exception {
+        if (atomicityMode() == CacheAtomicityMode.ATOMIC)
+            return;
+
+        testPutAllFailover(Test.PUT_ALL_PESSIMISTIC_TX);
     }
 
     /**
      * @throws Exception If failed.
      */
     public void testPutAllFailoverAsync() throws Exception {
-        testPutAllFailover(true);
+        testPutAllFailover(Test.PUT_ALL_ASYNC);
     }
 
     /**
-     * @param async If {@code true} tests asynchronous operation.
+     * @param test Test type
      * @throws Exception If failed.
      */
-    private void testPutAllFailover(final boolean async) throws Exception {
+    private void testPutAllFailover(final Test test) throws Exception {
         final AtomicBoolean finished = new AtomicBoolean();
 
         final long endTime = System.currentTimeMillis() + TEST_TIME;
@@ -123,7 +133,7 @@ public abstract class CachePutAllFailoverAbstractTest extends GridCacheAbstractS
         try {
             IgniteCache<TestKey, TestValue> cache0 = ignite(0).cache(null);
 
-            final IgniteCache<TestKey, TestValue> cache = async ? cache0.withAsync() : cache0;
+            final IgniteCache<TestKey, TestValue> cache = test == Test.PUT_ALL_ASYNC ? cache0.withAsync() : cache0;
 
             GridTestUtils.runMultiThreaded(new Callable<Object>() {
                 @Override public Object call() throws Exception {
@@ -142,10 +152,8 @@ public abstract class CachePutAllFailoverAbstractTest extends GridCacheAbstractS
                             lastInfo = time;
                         }
 
-                        if (async) {
-                            Collection<IgniteFuture<?>> futs = new ArrayList<>();
-
-                            for (int i = 0 ; i < 10; i++) {
+                        switch (test) {
+                            case PUT_ALL: {
                                 TreeMap<TestKey, TestValue> map = new TreeMap<>();
 
                                 for (int k = 0; k < 100; k++)
@@ -153,23 +161,55 @@ public abstract class CachePutAllFailoverAbstractTest extends GridCacheAbstractS
 
                                 cache.putAll(map);
 
-                                IgniteFuture<?> fut = cache.future();
+                                break;
+                            }
+
+                            case PUT_ALL_ASYNC: {
+                                Collection<IgniteFuture<?>> futs = new ArrayList<>();
 
-                                assertNotNull(fut);
+                                for (int i = 0 ; i < 10; i++) {
+                                    TreeMap<TestKey, TestValue> map = new TreeMap<>();
 
-                                futs.add(fut);
+                                    for (int k = 0; k < 100; k++)
+                                        map.put(new TestKey(rnd.nextInt(200)), new TestValue(iter));
+
+                                    cache.putAll(map);
+
+                                    IgniteFuture<?> fut = cache.future();
+
+                                    assertNotNull(fut);
+
+                                    futs.add(fut);
+                                }
+
+                                for (IgniteFuture<?> fut : futs)
+                                    fut.get();
+
+                                break;
                             }
 
-                            for (IgniteFuture<?> fut : futs)
-                                fut.get();
-                        }
-                        else {
-                            TreeMap<TestKey, TestValue> map = new TreeMap<>();
+                            case PUT_ALL_PESSIMISTIC_TX: {
+                                final TreeMap<TestKey, TestValue> map = new TreeMap<>();
+
+                                for (int k = 0; k < 100; k++)
+                                    map.put(new TestKey(rnd.nextInt(200)), new TestValue(iter));
+
+                                doInTransaction(ignite(0), new Callable<Object>() {
+                                    @Override public Object call() throws Exception {
+                                        for (TestKey key : map.keySet())
+                                            cache.get(key);
+
+                                        cache.putAll(map);
 
-                            for (int k = 0; k < 100; k++)
-                                map.put(new TestKey(rnd.nextInt(200)), new TestValue(iter));
+                                        return null;
+                                    }
+                                });
 
-                            cache.putAll(map);
+                                break;
+                            }
+
+                            default:
+                                assert false;
                         }
 
                         iter++;
@@ -277,4 +317,18 @@ public abstract class CachePutAllFailoverAbstractTest extends GridCacheAbstractS
             return S.toString(TestValue.class, this);
         }
     }
+
+    /**
+     *
+     */
+    private enum Test {
+        /** */
+        PUT_ALL,
+
+        /** */
+        PUT_ALL_ASYNC,
+
+        /** */
+        PUT_ALL_PESSIMISTIC_TX
+    }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/67699564/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/IgniteCachePutRetryTransactionalSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/IgniteCachePutRetryTransactionalSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/IgniteCachePutRetryTransactionalSelfTest.java
index 997848b..7c66efc 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/IgniteCachePutRetryTransactionalSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/IgniteCachePutRetryTransactionalSelfTest.java
@@ -24,7 +24,6 @@ import java.util.concurrent.ThreadLocalRandom;
 import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.concurrent.atomic.AtomicInteger;
 import java.util.concurrent.atomic.AtomicReferenceArray;
-import javax.cache.CacheException;
 import javax.cache.processor.EntryProcessorException;
 import javax.cache.processor.MutableEntry;
 import org.apache.ignite.Ignite;
@@ -32,18 +31,12 @@ import org.apache.ignite.IgniteAtomicLong;
 import org.apache.ignite.IgniteCache;
 import org.apache.ignite.cache.CacheAtomicityMode;
 import org.apache.ignite.cache.CacheEntryProcessor;
-import org.apache.ignite.cluster.ClusterTopologyException;
 import org.apache.ignite.configuration.NearCacheConfiguration;
 import org.apache.ignite.internal.IgniteInternalFuture;
 import org.apache.ignite.internal.util.typedef.internal.U;
-import org.apache.ignite.lang.IgniteFuture;
 import org.apache.ignite.testframework.GridTestUtils;
-import org.apache.ignite.transactions.Transaction;
-import org.apache.ignite.transactions.TransactionRollbackException;
 
 import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL;
-import static org.apache.ignite.transactions.TransactionConcurrency.PESSIMISTIC;
-import static org.apache.ignite.transactions.TransactionIsolation.REPEATABLE_READ;
 
 /**
  *
@@ -171,41 +164,6 @@ public class IgniteCachePutRetryTransactionalSelfTest extends IgniteCachePutRetr
     }
 
     /**
-     * @param ignite Ignite instance.
-     * @param clo Closure.
-     * @return Result of closure execution.
-     * @throws Exception If failed.
-     */
-    private <T> T doInTransaction(Ignite ignite, Callable<T> clo) throws Exception {
-        while (true) {
-            try (Transaction tx = ignite.transactions().txStart(PESSIMISTIC, REPEATABLE_READ)) {
-                T res = clo.call();
-
-                tx.commit();
-
-                return res;
-            }
-            catch (CacheException e) {
-                if (e.getCause() instanceof ClusterTopologyException) {
-                    ClusterTopologyException topEx = (ClusterTopologyException)e.getCause();
-
-                    topEx.retryReadyFuture().get();
-                }
-                else
-                    throw e;
-            }
-            catch (ClusterTopologyException e) {
-                IgniteFuture<?> fut = e.retryReadyFuture();
-
-                fut.get();
-            }
-            catch (TransactionRollbackException ignore) {
-                // Safe to retry right away.
-            }
-        }
-    }
-
-    /**
      * Callable to process inside transaction.
      */
     private static class ProcessCallable implements Callable<Void> {

http://git-wip-us.apache.org/repos/asf/ignite/blob/67699564/modules/core/src/test/java/org/apache/ignite/testframework/junits/common/GridCommonAbstractTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testframework/junits/common/GridCommonAbstractTest.java b/modules/core/src/test/java/org/apache/ignite/testframework/junits/common/GridCommonAbstractTest.java
index 4d7e923..13ec665 100644
--- a/modules/core/src/test/java/org/apache/ignite/testframework/junits/common/GridCommonAbstractTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/testframework/junits/common/GridCommonAbstractTest.java
@@ -24,6 +24,7 @@ import java.util.HashSet;
 import java.util.Iterator;
 import java.util.List;
 import java.util.Set;
+import java.util.concurrent.Callable;
 import java.util.concurrent.CountDownLatch;
 import java.util.concurrent.atomic.AtomicReference;
 import javax.cache.Cache;
@@ -47,6 +48,7 @@ import org.apache.ignite.cache.affinity.Affinity;
 import org.apache.ignite.cache.affinity.AffinityFunction;
 import org.apache.ignite.cluster.ClusterGroup;
 import org.apache.ignite.cluster.ClusterNode;
+import org.apache.ignite.cluster.ClusterTopologyException;
 import org.apache.ignite.compute.ComputeTask;
 import org.apache.ignite.compute.ComputeTaskFuture;
 import org.apache.ignite.configuration.CacheConfiguration;
@@ -74,12 +76,16 @@ import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.lang.IgniteFuture;
 import org.apache.ignite.lang.IgnitePredicate;
 import org.apache.ignite.testframework.junits.GridAbstractTest;
+import org.apache.ignite.transactions.Transaction;
+import org.apache.ignite.transactions.TransactionRollbackException;
 import org.jetbrains.annotations.Nullable;
 
 import static org.apache.ignite.cache.CacheMode.LOCAL;
 import static org.apache.ignite.cache.CacheMode.PARTITIONED;
 import static org.apache.ignite.cache.CacheRebalanceMode.NONE;
 import static org.apache.ignite.internal.processors.cache.GridCacheUtils.isNearEnabled;
+import static org.apache.ignite.transactions.TransactionConcurrency.PESSIMISTIC;
+import static org.apache.ignite.transactions.TransactionIsolation.REPEATABLE_READ;
 
 /**
  * Super class for all common tests.
@@ -986,4 +992,39 @@ public abstract class GridCommonAbstractTest extends GridAbstractTest {
             idx++;
         }
     }
+
+    /**
+     * @param ignite Ignite instance.
+     * @param clo Closure.
+     * @return Result of closure execution.
+     * @throws Exception If failed.
+     */
+    protected <T> T doInTransaction(Ignite ignite, Callable<T> clo) throws Exception {
+        while (true) {
+            try (Transaction tx = ignite.transactions().txStart(PESSIMISTIC, REPEATABLE_READ)) {
+                T res = clo.call();
+
+                tx.commit();
+
+                return res;
+            }
+            catch (CacheException e) {
+                if (e.getCause() instanceof ClusterTopologyException) {
+                    ClusterTopologyException topEx = (ClusterTopologyException)e.getCause();
+
+                    topEx.retryReadyFuture().get();
+                }
+                else
+                    throw e;
+            }
+            catch (ClusterTopologyException e) {
+                IgniteFuture<?> fut = e.retryReadyFuture();
+
+                fut.get();
+            }
+            catch (TransactionRollbackException ignore) {
+                // Safe to retry right away.
+            }
+        }
+    }
 }
\ No newline at end of file


[22/39] ignite git commit: ignite-1273: added ability to modify arrays returned from PortableBuilder and fixed cyclic references processing for arrays and collections in PortableMarshaller

Posted by ag...@apache.org.
http://git-wip-us.apache.org/repos/asf/ignite/blob/9057a4c0/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableLazyMap.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableLazyMap.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableLazyMap.java
deleted file mode 100644
index eed8121..0000000
--- a/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableLazyMap.java
+++ /dev/null
@@ -1,218 +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;
-
-import java.util.AbstractMap;
-import java.util.AbstractSet;
-import java.util.Iterator;
-import java.util.LinkedHashMap;
-import java.util.Map;
-import java.util.Set;
-import org.jetbrains.annotations.Nullable;
-
-/**
- *
- */
-class PortableLazyMap extends AbstractMap<Object, Object> implements PortableBuilderSerializationAware {
-    /** */
-    private final PortableBuilderReader reader;
-
-    /** */
-    private final int off;
-
-    /** */
-    private Map<Object, Object> delegate;
-
-    /**
-     * @param reader Reader.
-     * @param off Offset.
-     */
-    private PortableLazyMap(PortableBuilderReader reader, int off) {
-        this.reader = reader;
-        this.off = off;
-    }
-
-    /**
-     * @param reader Reader.
-     * @return PortableLazyMap.
-     */
-    @Nullable public static PortableLazyMap parseMap(PortableBuilderReader reader) {
-        int off = reader.position() - 1;
-
-        int size = reader.readInt();
-
-        reader.skip(1); // map type.
-
-        for (int i = 0; i < size; i++) {
-            reader.skipValue(); // skip key
-            reader.skipValue(); // skip value
-        }
-
-        return new PortableLazyMap(reader, off);
-    }
-
-    /**
-     *
-     */
-    private void ensureDelegateInit() {
-        if (delegate == null) {
-            int size = reader.readIntAbsolute(off + 1);
-
-            reader.position(off + 1/* flag */ + 4/* size */ + 1/* col type */);
-
-            delegate = new LinkedHashMap<>();
-
-            for (int i = 0; i < size; i++)
-                delegate.put(PortableUtils.unwrapLazy(reader.parseValue()), reader.parseValue());
-        }
-    }
-
-    /** {@inheritDoc} */
-    @Override public void writeTo(PortableWriterExImpl writer, PortableBuilderSerializer ctx) {
-        if (delegate == null) {
-            int size = reader.readIntAbsolute(off + 1);
-
-            int hdrSize = 1 /* flag */ + 4 /* size */ + 1 /* col type */;
-            writer.write(reader.array(), off, hdrSize);
-
-            reader.position(off + hdrSize);
-
-            for (int i = 0; i < size; i++) {
-                ctx.writeValue(writer, reader.parseValue()); // key
-                ctx.writeValue(writer, reader.parseValue()); // value
-            }
-        }
-        else {
-            writer.writeByte(GridPortableMarshaller.MAP);
-            writer.writeInt(delegate.size());
-
-            byte colType = reader.array()[off + 1 /* flag */ + 4 /* size */];
-
-            writer.writeByte(colType);
-
-            for (Entry<Object, Object> entry : delegate.entrySet()) {
-                ctx.writeValue(writer, entry.getKey());
-                ctx.writeValue(writer, entry.getValue());
-            }
-        }
-    }
-
-    /** {@inheritDoc} */
-    @Override public int size() {
-        if (delegate == null)
-            return reader.readIntAbsolute(off + 1);
-
-        return delegate.size();
-    }
-
-    /** {@inheritDoc} */
-    @Override public boolean containsKey(Object key) {
-        ensureDelegateInit();
-
-        return delegate.containsKey(key);
-    }
-
-    /** {@inheritDoc} */
-    @Override public boolean containsValue(Object val) {
-        return values().contains(val);
-    }
-
-    /** {@inheritDoc} */
-    @Override public Set<Object> keySet() {
-        ensureDelegateInit();
-
-        return delegate.keySet();
-    }
-
-    /** {@inheritDoc} */
-    @Override public void clear() {
-        if (delegate == null)
-            delegate = new LinkedHashMap<>();
-        else
-            delegate.clear();
-    }
-
-    /** {@inheritDoc} */
-    @Override public Object get(Object key) {
-        ensureDelegateInit();
-
-        return PortableUtils.unwrapLazy(delegate.get(key));
-    }
-
-    /** {@inheritDoc} */
-    @Override public Object put(Object key, Object val) {
-        ensureDelegateInit();
-
-        return PortableUtils.unwrapLazy(delegate.put(key, val));
-    }
-
-    /** {@inheritDoc} */
-    @Override public Object remove(Object key) {
-        ensureDelegateInit();
-
-        return PortableUtils.unwrapLazy(delegate.remove(key));
-    }
-
-    /** {@inheritDoc} */
-    @Override public Set<Entry<Object, Object>> entrySet() {
-        ensureDelegateInit();
-
-        return new AbstractSet<Entry<Object, Object>>() {
-            @Override public boolean contains(Object o) {
-                throw new UnsupportedOperationException();
-            }
-
-            @Override public Iterator<Entry<Object, Object>> iterator() {
-                return new Iterator<Entry<Object, Object>>() {
-                    /** */
-                    private final Iterator<Entry<Object, Object>> itr = delegate.entrySet().iterator();
-
-                    @Override public boolean hasNext() {
-                        return itr.hasNext();
-                    }
-
-                    @Override public Entry<Object, Object> next() {
-                        Entry<Object, Object> res = itr.next();
-
-                        final Object val = res.getValue();
-
-                        if (val instanceof PortableLazyValue) {
-                            return new SimpleEntry<Object, Object>(res.getKey(), val) {
-                                private static final long serialVersionUID = 0L;
-
-                                @Override public Object getValue() {
-                                    return ((PortableLazyValue)val).value();
-                                }
-                            };
-                        }
-
-                        return res;
-                    }
-
-                    @Override public void remove() {
-                        itr.remove();
-                    }
-                };
-            }
-
-            @Override public int size() {
-                return delegate.size();
-            }
-        };
-    }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/9057a4c0/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableLazyMapEntry.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableLazyMapEntry.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableLazyMapEntry.java
deleted file mode 100644
index 1970d21..0000000
--- a/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableLazyMapEntry.java
+++ /dev/null
@@ -1,66 +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;
-
-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(PortableWriterExImpl writer, PortableBuilderSerializer ctx) {
-        writer.writeByte(GridPortableMarshaller.MAP_ENTRY);
-
-        ctx.writeValue(writer, key);
-        ctx.writeValue(writer, val);
-    }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/9057a4c0/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableLazySet.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableLazySet.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableLazySet.java
deleted file mode 100644
index 3e1dc92..0000000
--- a/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableLazySet.java
+++ /dev/null
@@ -1,89 +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;
-
-import java.util.Collection;
-import java.util.Set;
-import org.apache.ignite.internal.util.typedef.internal.U;
-
-/**
- *
- */
-class PortableLazySet extends PortableAbstractLazyValue {
-    /** */
-    private final int off;
-
-    /**
-     * @param reader Reader.
-     * @param size Size.
-     */
-    PortableLazySet(PortableBuilderReader reader, int size) {
-        super(reader, reader.position() - 1);
-
-        off = reader.position() - 1/* flag */ - 4/* size */ - 1/* col type */;
-
-        assert size >= 0;
-
-        for (int i = 0; i < size; i++)
-            reader.skipValue();
-    }
-
-    /** {@inheritDoc} */
-    @Override public void writeTo(PortableWriterExImpl writer, PortableBuilderSerializer ctx) {
-        if (val == null) {
-            int size = reader.readIntAbsolute(off + 1);
-
-            int hdrSize = 1 /* flag */ + 4 /* size */ + 1 /* col type */;
-            writer.write(reader.array(), off, hdrSize);
-
-            reader.position(off + hdrSize);
-
-            for (int i = 0; i < size; i++) {
-                Object o = reader.parseValue();
-
-                ctx.writeValue(writer, o);
-            }
-        }
-        else {
-            Collection<Object> c = (Collection<Object>)val;
-
-            writer.writeByte(GridPortableMarshaller.COL);
-            writer.writeInt(c.size());
-
-            byte colType = reader.array()[off + 1 /* flag */ + 4 /* size */];
-            writer.writeByte(colType);
-
-            for (Object o : c)
-                ctx.writeValue(writer, o);
-        }
-    }
-
-    /** {@inheritDoc} */
-    @Override protected Object init() {
-        int size = reader.readIntAbsolute(off + 1);
-
-        reader.position(off + 1/* flag */ + 4/* size */ + 1/* col type */);
-
-        Set<Object> res = U.newLinkedHashSet(size);
-
-        for (int i = 0; i < size; i++)
-            res.add(PortableUtils.unwrapLazy(reader.parseValue()));
-
-        return res;
-    }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/9057a4c0/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableLazyValue.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableLazyValue.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableLazyValue.java
deleted file mode 100644
index 43728b7..0000000
--- a/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableLazyValue.java
+++ /dev/null
@@ -1,28 +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;
-
-/**
- *
- */
-interface PortableLazyValue extends PortableBuilderSerializationAware {
-    /**
-     * @return Value.
-     */
-    public Object value();
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/9057a4c0/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableObjectArrayLazyValue.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableObjectArrayLazyValue.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableObjectArrayLazyValue.java
deleted file mode 100644
index 897e8e8..0000000
--- a/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableObjectArrayLazyValue.java
+++ /dev/null
@@ -1,89 +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;
-
-import org.apache.ignite.internal.util.typedef.internal.U;
-import org.apache.ignite.portable.PortableInvalidClassException;
-
-/**
- *
- */
-class PortableObjectArrayLazyValue extends PortableAbstractLazyValue {
-    /** */
-    private Object[] lazyValsArr;
-
-    /** */
-    private int compTypeId;
-
-    /** */
-    private String clsName;
-
-    /**
-     * @param reader Reader.
-     */
-    protected PortableObjectArrayLazyValue(PortableBuilderReader reader) {
-        super(reader, reader.position() - 1);
-
-        int typeId = reader.readInt();
-
-        if (typeId == GridPortableMarshaller.UNREGISTERED_TYPE_ID) {
-            clsName = reader.readString();
-
-            Class cls;
-
-            try {
-                // TODO: IGNITE-1272 - Is class loader needed here?
-                cls = U.forName(reader.readString(), null);
-            }
-            catch (ClassNotFoundException e) {
-                throw new PortableInvalidClassException("Failed to load the class: " + clsName, e);
-            }
-
-            compTypeId = reader.portableContext().descriptorForClass(cls).typeId();
-        }
-        else {
-            compTypeId = typeId;
-            clsName = null;
-        }
-
-        int size = reader.readInt();
-
-        lazyValsArr = new Object[size];
-
-        for (int i = 0; i < size; i++)
-            lazyValsArr[i] = reader.parseValue();
-    }
-
-    /** {@inheritDoc} */
-    @Override protected Object init() {
-        for (int i = 0; i < lazyValsArr.length; i++) {
-            if (lazyValsArr[i] instanceof PortableLazyValue)
-                lazyValsArr[i] = ((PortableLazyValue)lazyValsArr[i]).value();
-        }
-
-        return lazyValsArr;
-    }
-
-    /** {@inheritDoc} */
-    @Override public void writeTo(PortableWriterExImpl writer, PortableBuilderSerializer ctx) {
-        if (clsName == null)
-            ctx.writeArray(writer, GridPortableMarshaller.OBJ_ARR, lazyValsArr, compTypeId);
-        else
-            ctx.writeArray(writer, GridPortableMarshaller.OBJ_ARR, lazyValsArr, clsName);
-    }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/9057a4c0/modules/core/src/main/java/org/apache/ignite/internal/portable/PortablePlainLazyValue.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/PortablePlainLazyValue.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/PortablePlainLazyValue.java
deleted file mode 100644
index d08d09b..0000000
--- a/modules/core/src/main/java/org/apache/ignite/internal/portable/PortablePlainLazyValue.java
+++ /dev/null
@@ -1,47 +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;
-
-/**
- *
- */
-class PortablePlainLazyValue extends PortableAbstractLazyValue {
-    /** */
-    protected final int len;
-
-    /**
-     * @param reader Reader
-     * @param valOff Offset
-     * @param len Length.
-     */
-    protected PortablePlainLazyValue(PortableBuilderReader reader, int valOff, int len) {
-        super(reader, valOff);
-
-        this.len = len;
-    }
-
-    /** {@inheritDoc} */
-    @Override protected Object init() {
-        return reader.reader().unmarshal(valOff);
-    }
-
-    /** {@inheritDoc} */
-    @Override public void writeTo(PortableWriterExImpl writer, PortableBuilderSerializer ctx) {
-        writer.write(reader.array(), valOff, len);
-    }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/9057a4c0/modules/core/src/main/java/org/apache/ignite/internal/portable/PortablePlainPortableObject.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/PortablePlainPortableObject.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/PortablePlainPortableObject.java
deleted file mode 100644
index cfaa04f..0000000
--- a/modules/core/src/main/java/org/apache/ignite/internal/portable/PortablePlainPortableObject.java
+++ /dev/null
@@ -1,50 +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;
-
-import org.apache.ignite.portable.PortableObject;
-
-/**
- *
- */
-public class PortablePlainPortableObject implements PortableLazyValue {
-    /** */
-    private final PortableObject portableObj;
-
-    /**
-     * @param portableObj Portable object.
-     */
-    public PortablePlainPortableObject(PortableObject portableObj) {
-        this.portableObj = portableObj;
-    }
-
-    /** {@inheritDoc} */
-    @Override public Object value() {
-        return portableObj;
-    }
-
-    /** {@inheritDoc} */
-    @Override public void writeTo(PortableWriterExImpl writer, PortableBuilderSerializer ctx) {
-        PortableObject val = portableObj;
-
-        if (val instanceof PortableObjectOffheapImpl)
-            val = ((PortableObjectOffheapImpl)val).heapCopy();
-
-        writer.doWritePortableObject((PortableObjectImpl)val);
-    }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/9057a4c0/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableReaderExImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableReaderExImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableReaderExImpl.java
index f702e06..83ccb65 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableReaderExImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableReaderExImpl.java
@@ -156,7 +156,7 @@ public class PortableReaderExImpl implements PortableReader, PortableRawReaderEx
      * @param start Start.
      * @param ldr Class loader.
      */
-    PortableReaderExImpl(PortableContext ctx, byte[] arr, int start, ClassLoader ldr) {
+    public PortableReaderExImpl(PortableContext ctx, byte[] arr, int start, ClassLoader ldr) {
         this(ctx, new PortableHeapInputStream(arr), start, ldr, new PortableReaderContext());
     }
 
@@ -256,7 +256,7 @@ public class PortableReaderExImpl implements PortableReader, PortableRawReaderEx
      * @return Unmarshalled value.
      * @throws PortableException In case of error.
      */
-    Object unmarshal(int offset) throws PortableException {
+    public Object unmarshal(int offset) throws PortableException {
         off = offset;
 
         return off >= 0 ? unmarshal(false) : null;
@@ -586,6 +586,9 @@ public class PortableReaderExImpl implements PortableReader, PortableRawReaderEx
             if (flag == NULL)
                 return null;
 
+            if (flag == HANDLE)
+                return readHandleField();
+
             if (flag != BYTE_ARR)
                 throw new PortableException("Invalid flag value: " + flag);
 
@@ -609,6 +612,9 @@ public class PortableReaderExImpl implements PortableReader, PortableRawReaderEx
             if (flag == NULL)
                 return null;
 
+            if (flag == HANDLE)
+                return readHandleField();
+
             if (flag != SHORT_ARR)
                 throw new PortableException("Invalid flag value: " + flag);
 
@@ -632,6 +638,9 @@ public class PortableReaderExImpl implements PortableReader, PortableRawReaderEx
             if (flag == NULL)
                 return null;
 
+            if (flag == HANDLE)
+                return readHandleField();
+
             if (flag != INT_ARR)
                 throw new PortableException("Invalid flag value: " + flag);
 
@@ -655,6 +664,9 @@ public class PortableReaderExImpl implements PortableReader, PortableRawReaderEx
             if (flag == NULL)
                 return null;
 
+            if (flag == HANDLE)
+                return readHandleField();
+
             if (flag != LONG_ARR)
                 throw new PortableException("Invalid flag value: " + flag);
 
@@ -678,6 +690,9 @@ public class PortableReaderExImpl implements PortableReader, PortableRawReaderEx
             if (flag == NULL)
                 return null;
 
+            if (flag == HANDLE)
+                return readHandleField();
+
             if (flag != FLOAT_ARR)
                 throw new PortableException("Invalid flag value: " + flag);
 
@@ -701,6 +716,9 @@ public class PortableReaderExImpl implements PortableReader, PortableRawReaderEx
             if (flag == NULL)
                 return null;
 
+            if (flag == HANDLE)
+                return readHandleField();
+
             if (flag != DOUBLE_ARR)
                 throw new PortableException("Invalid flag value: " + flag);
 
@@ -724,6 +742,9 @@ public class PortableReaderExImpl implements PortableReader, PortableRawReaderEx
             if (flag == NULL)
                 return null;
 
+            if (flag == HANDLE)
+                return readHandleField();
+
             if (flag != CHAR_ARR)
                 throw new PortableException("Invalid flag value: " + flag);
 
@@ -747,6 +768,9 @@ public class PortableReaderExImpl implements PortableReader, PortableRawReaderEx
             if (flag == NULL)
                 return null;
 
+            if (flag == HANDLE)
+                return readHandleField();
+
             if (flag != BOOLEAN_ARR)
                 throw new PortableException("Invalid flag value: " + flag);
 
@@ -770,6 +794,9 @@ public class PortableReaderExImpl implements PortableReader, PortableRawReaderEx
             if (flag == NULL)
                 return null;
 
+            if (flag == HANDLE)
+                return readHandleField();
+
             if (flag != DECIMAL_ARR)
                 throw new PortableException("Invalid flag value: " + flag);
 
@@ -793,6 +820,9 @@ public class PortableReaderExImpl implements PortableReader, PortableRawReaderEx
             if (flag == NULL)
                 return null;
 
+            if (flag == HANDLE)
+                return readHandleField();
+
             if (flag != STRING_ARR)
                 throw new PortableException("Invalid flag value: " + flag);
 
@@ -816,6 +846,9 @@ public class PortableReaderExImpl implements PortableReader, PortableRawReaderEx
             if (flag == NULL)
                 return null;
 
+            if (flag == HANDLE)
+                return readHandleField();
+
             if (flag != UUID_ARR)
                 throw new PortableException("Invalid flag value: " + flag);
 
@@ -839,6 +872,9 @@ public class PortableReaderExImpl implements PortableReader, PortableRawReaderEx
             if (flag == NULL)
                 return null;
 
+            if (flag == HANDLE)
+                return readHandleField();
+
             if (flag != DATE_ARR)
                 throw new PortableException("Invalid flag value: " + flag);
 
@@ -862,6 +898,9 @@ public class PortableReaderExImpl implements PortableReader, PortableRawReaderEx
             if (flag == NULL)
                 return null;
 
+            if (flag == HANDLE)
+                return readHandleField();
+
             if (flag != OBJ_ARR)
                 throw new PortableException("Invalid flag value: " + flag);
 
@@ -887,6 +926,9 @@ public class PortableReaderExImpl implements PortableReader, PortableRawReaderEx
             if (flag == NULL)
                 return null;
 
+            if (flag == HANDLE)
+                return readHandleField();
+
             if (flag != COL)
                 throw new PortableException("Invalid flag value: " + flag);
 
@@ -912,6 +954,9 @@ public class PortableReaderExImpl implements PortableReader, PortableRawReaderEx
             if (flag == NULL)
                 return null;
 
+            if (flag == HANDLE)
+                return readHandleField();
+
             if (flag != MAP)
                 throw new PortableException("Invalid flag value: " + flag);
 
@@ -935,10 +980,13 @@ public class PortableReaderExImpl implements PortableReader, PortableRawReaderEx
             if (flag == NULL)
                 return null;
 
+            if (flag == HANDLE)
+                return readHandleField();
+
             if (flag != MAP_ENTRY)
                 throw new PortableException("Invalid flag value: " + flag);
 
-            return new GridMapEntry<>(doReadObject(false), doReadObject(false));
+            return doReadMapEntry(false, true);
         }
         else
             return null;
@@ -1059,6 +1107,33 @@ public class PortableReaderExImpl implements PortableReader, PortableRawReaderEx
         rCtx.setObjectHandler(start, obj);
     }
 
+    /**
+     * @param obj Object.
+     * @param pos Position.
+     */
+    void setHandler(Object obj, int pos) {
+        rCtx.setObjectHandler(pos, obj);
+    }
+
+    /**
+     * Recreating field value from a handle.
+     *
+     * @param <T> Field type.
+     * @return Field.
+     */
+    private <T> T readHandleField() {
+        int handle = (off - 1) - doReadInt(false);
+
+        Object obj = rCtx.getObjectByHandle(handle);
+
+        if (obj == null) {
+            off = handle;
+
+            obj = doReadObject(false);
+        }
+
+        return (T)obj;
+    }
     /** {@inheritDoc} */
     @Override public byte readByte(String fieldName) throws PortableException {
         Byte val = readByte(fieldId(fieldName));
@@ -1676,7 +1751,7 @@ public class PortableReaderExImpl implements PortableReader, PortableRawReaderEx
                 else
                     po = in.offheapPointer() > 0
                         ? new PortableObjectOffheapImpl(ctx, in.offheapPointer(), start,
-                                                            in.remaining() + in.position())
+                        in.remaining() + in.position())
                         : new PortableObjectImpl(ctx, in.array(), start);
 
                 rCtx.setPortableHandler(start, po);
@@ -1805,7 +1880,6 @@ public class PortableReaderExImpl implements PortableReader, PortableRawReaderEx
 
                 return obj;
 
-
             default:
                 throw new PortableException("Invalid flag value: " + flag);
         }
@@ -2306,12 +2380,16 @@ public class PortableReaderExImpl implements PortableReader, PortableRawReaderEx
      * @return Value.
      */
     private byte[] doReadByteArray(boolean raw) {
+        int hPos = (raw ? rawOff : off) - 1;
+
         int len = doReadInt(raw);
 
         in.position(raw ? rawOff : off);
 
         byte[] arr = in.readByteArray(len);
 
+        setHandler(arr, hPos);
+
         if (raw)
             rawOff += len;
         else
@@ -2325,12 +2403,16 @@ public class PortableReaderExImpl implements PortableReader, PortableRawReaderEx
      * @return Value.
      */
     private short[] doReadShortArray(boolean raw) {
+        int hPos = (raw ? rawOff : off) - 1;
+
         int len = doReadInt(raw);
 
         in.position(raw ? rawOff : off);
 
         short[] arr = in.readShortArray(len);
 
+        setHandler(arr, hPos);
+
         int bytes = len << 1;
 
         if (raw)
@@ -2346,12 +2428,16 @@ public class PortableReaderExImpl implements PortableReader, PortableRawReaderEx
      * @return Value.
      */
     private int[] doReadIntArray(boolean raw) {
+        int hPos = (raw ? rawOff : off) - 1;
+
         int len = doReadInt(raw);
 
         in.position(raw ? rawOff : off);
 
         int[] arr = in.readIntArray(len);
 
+        setHandler(arr, hPos);
+
         int bytes = len << 2;
 
         if (raw)
@@ -2367,12 +2453,16 @@ public class PortableReaderExImpl implements PortableReader, PortableRawReaderEx
      * @return Value.
      */
     private long[] doReadLongArray(boolean raw) {
+        int hPos = (raw ? rawOff : off) - 1;
+
         int len = doReadInt(raw);
 
         in.position(raw ? rawOff : off);
 
         long[] arr = in.readLongArray(len);
 
+        setHandler(arr, hPos);
+
         int bytes = len << 3;
 
         if (raw)
@@ -2388,12 +2478,16 @@ public class PortableReaderExImpl implements PortableReader, PortableRawReaderEx
      * @return Value.
      */
     private float[] doReadFloatArray(boolean raw) {
+        int hPos = (raw ? rawOff : off) - 1;
+
         int len = doReadInt(raw);
 
         in.position(raw ? rawOff : off);
 
         float[] arr = in.readFloatArray(len);
 
+        setHandler(arr, hPos);
+
         int bytes = len << 2;
 
         if (raw)
@@ -2409,12 +2503,16 @@ public class PortableReaderExImpl implements PortableReader, PortableRawReaderEx
      * @return Value.
      */
     private double[] doReadDoubleArray(boolean raw) {
+        int hPos = (raw ? rawOff : off) - 1;
+
         int len = doReadInt(raw);
 
         in.position(raw ? rawOff : off);
 
         double[] arr = in.readDoubleArray(len);
 
+        setHandler(arr, hPos);
+
         int bytes = len << 3;
 
         if (raw)
@@ -2430,12 +2528,16 @@ public class PortableReaderExImpl implements PortableReader, PortableRawReaderEx
      * @return Value.
      */
     private char[] doReadCharArray(boolean raw) {
+        int hPos = (raw ? rawOff : off) - 1;
+
         int len = doReadInt(raw);
 
         in.position(raw ? rawOff : off);
 
         char[] arr = in.readCharArray(len);
 
+        setHandler(arr, hPos);
+
         int bytes = len << 1;
 
         if (raw)
@@ -2451,12 +2553,16 @@ public class PortableReaderExImpl implements PortableReader, PortableRawReaderEx
      * @return Value.
      */
     private boolean[] doReadBooleanArray(boolean raw) {
+        int hPos = (raw ? rawOff : off) - 1;
+
         int len = doReadInt(raw);
 
         in.position(raw ? rawOff : off);
 
         boolean[] arr = in.readBooleanArray(len);
 
+        setHandler(arr, hPos);
+
         if (raw)
             rawOff += len;
         else
@@ -2471,10 +2577,14 @@ public class PortableReaderExImpl implements PortableReader, PortableRawReaderEx
      * @throws PortableException In case of error.
      */
     private BigDecimal[] doReadDecimalArray(boolean raw) throws PortableException {
+        int hPos = (raw ? rawOff : off) - 1;
+
         int len = doReadInt(raw);
 
         BigDecimal[] arr = new BigDecimal[len];
 
+        setHandler(arr, hPos);
+
         for (int i = 0; i < len; i++) {
             byte flag = doReadByte(raw);
 
@@ -2497,10 +2607,14 @@ public class PortableReaderExImpl implements PortableReader, PortableRawReaderEx
      * @throws PortableException In case of error.
      */
     private String[] doReadStringArray(boolean raw) throws PortableException {
+        int hPos = (raw ? rawOff : off) - 1;
+
         int len = doReadInt(raw);
 
         String[] arr = new String[len];
 
+        setHandler(arr, hPos);
+
         for (int i = 0; i < len; i++) {
             byte flag = doReadByte(raw);
 
@@ -2523,10 +2637,14 @@ public class PortableReaderExImpl implements PortableReader, PortableRawReaderEx
      * @throws PortableException In case of error.
      */
     private UUID[] doReadUuidArray(boolean raw) throws PortableException {
+        int hPos = (raw ? rawOff : off) - 1;
+
         int len = doReadInt(raw);
 
         UUID[] arr = new UUID[len];
 
+        setHandler(arr, hPos);
+
         for (int i = 0; i < len; i++) {
             byte flag = doReadByte(raw);
 
@@ -2549,10 +2667,14 @@ public class PortableReaderExImpl implements PortableReader, PortableRawReaderEx
      * @throws PortableException In case of error.
      */
     private Date[] doReadDateArray(boolean raw) throws PortableException {
+        int hPos = (raw ? rawOff : off) - 1;
+
         int len = doReadInt(raw);
 
         Date[] arr = new Date[len];
 
+        setHandler(arr, hPos);
+
         for (int i = 0; i < len; i++) {
             byte flag = doReadByte(raw);
 
@@ -2576,12 +2698,16 @@ public class PortableReaderExImpl implements PortableReader, PortableRawReaderEx
      * @throws PortableException In case of error.
      */
     private Object[] doReadObjectArray(boolean raw, boolean deep) throws PortableException {
+        int hPos = (raw ? rawOff : off) - 1;
+
         Class compType = doReadClass(raw);
 
         int len = doReadInt(raw);
 
         Object[] arr = deep ? (Object[])Array.newInstance(compType, len) : new Object[len];
 
+        setHandler(arr, hPos);
+
         for (int i = 0; i < len; i++)
             arr[i] = deep ? doReadObject(raw) : unmarshal(raw);
 
@@ -2598,6 +2724,8 @@ public class PortableReaderExImpl implements PortableReader, PortableRawReaderEx
     @SuppressWarnings("unchecked")
     private Collection<?> doReadCollection(boolean raw, boolean deep, @Nullable Class<? extends Collection> cls)
         throws PortableException {
+        int hPos = (raw ? rawOff : off) - 1;
+
         int size = doReadInt(raw);
 
         assert size >= 0;
@@ -2667,6 +2795,8 @@ public class PortableReaderExImpl implements PortableReader, PortableRawReaderEx
             }
         }
 
+        setHandler(col, hPos);
+
         for (int i = 0; i < size; i++)
             col.add(deep ? doReadObject(raw) : unmarshal(raw));
 
@@ -2683,6 +2813,8 @@ public class PortableReaderExImpl implements PortableReader, PortableRawReaderEx
     @SuppressWarnings("unchecked")
     private Map<?, ?> doReadMap(boolean raw, boolean deep, @Nullable Class<? extends Map> cls)
         throws PortableException {
+        int hPos = (raw ? rawOff : off) - 1;
+
         int size = doReadInt(raw);
 
         assert size >= 0;
@@ -2742,6 +2874,8 @@ public class PortableReaderExImpl implements PortableReader, PortableRawReaderEx
             }
         }
 
+        setHandler(map, hPos);
+
         for (int i = 0; i < size; i++)
             map.put(deep ? doReadObject(raw) : unmarshal(raw), deep ? doReadObject(raw) : unmarshal(raw));
 
@@ -2755,10 +2889,16 @@ public class PortableReaderExImpl implements PortableReader, PortableRawReaderEx
      * @throws PortableException In case of error.
      */
     private Map.Entry<?, ?> doReadMapEntry(boolean raw, boolean deep) throws PortableException {
+        int hPos = (raw ? rawOff : off) - 1;
+
         Object val1 = deep ? doReadObject(raw) : unmarshal(raw);
         Object val2 = deep ? doReadObject(raw) : unmarshal(raw);
 
-        return new GridMapEntry<>(val1, val2);
+        GridMapEntry entry = new GridMapEntry<>(val1, val2);
+
+        setHandler(entry, hPos);
+
+        return entry;
     }
 
     /**
@@ -3017,4 +3157,4 @@ public class PortableReaderExImpl implements PortableReader, PortableRawReaderEx
     @Override public void close() throws IOException {
         // No-op.
     }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/9057a4c0/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 ce77783..7259cc9 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
@@ -33,6 +33,7 @@ import java.util.TreeSet;
 import java.util.UUID;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.ConcurrentSkipListSet;
+import org.apache.ignite.internal.portable.builder.PortableLazyValue;
 import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.portable.PortableObject;
 import org.jetbrains.annotations.Nullable;
@@ -359,6 +360,16 @@ public class PortableUtils {
     }
 
     /**
+     * Checks whether an array type values can or can not contain references to other object.
+     *
+     * @param type Array type.
+     * @return {@code true} if content of serialized array value cannot contain references to other object.
+     */
+    public static boolean isPlainArrayType(int type) {
+        return type >= BYTE_ARR && type <= DATE_ARR;
+    }
+
+    /**
      * @param cls Class.
      * @return Portable field type.
      */

http://git-wip-us.apache.org/repos/asf/ignite/blob/9057a4c0/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableValueWithType.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableValueWithType.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableValueWithType.java
deleted file mode 100644
index ebc68c1..0000000
--- a/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableValueWithType.java
+++ /dev/null
@@ -1,74 +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;
-
-import org.apache.ignite.internal.processors.cache.portable.CacheObjectPortableProcessorImpl;
-import org.apache.ignite.internal.util.typedef.internal.S;
-
-/**
- *
- */
-class PortableValueWithType implements PortableLazyValue {
-    /** */
-    private byte type;
-
-    /** */
-    private Object val;
-
-    /**
-     * @param type Type
-     * @param val Value.
-     */
-    PortableValueWithType(byte type, Object val) {
-        this.type = type;
-        this.val = val;
-    }
-
-    /** {@inheritDoc} */
-    @Override public void writeTo(PortableWriterExImpl writer, PortableBuilderSerializer ctx) {
-        if (val instanceof PortableBuilderSerializationAware)
-            ((PortableBuilderSerializationAware)val).writeTo(writer, ctx);
-        else
-            ctx.writeValue(writer, val);
-    }
-
-    /** {@inheritDoc} */
-    public String typeName() {
-        return CacheObjectPortableProcessorImpl.fieldTypeName(type);
-    }
-
-    /** {@inheritDoc} */
-    @Override public Object value() {
-        if (val instanceof PortableLazyValue)
-            return ((PortableLazyValue)val).value();
-
-        return val;
-    }
-
-    /**
-     * @param val New value.
-     */
-    public void value(Object val) {
-        this.val = val;
-    }
-
-    /** {@inheritDoc} */
-    @Override public String toString() {
-        return S.toString(PortableValueWithType.class, this);
-    }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/9057a4c0/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableWriterExImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableWriterExImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableWriterExImpl.java
index 6bcce2b..364d5f8 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableWriterExImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableWriterExImpl.java
@@ -156,7 +156,7 @@ public class PortableWriterExImpl implements PortableWriter, PortableRawWriterEx
      * @param off Start offset.
      * @param typeId Type ID.
      */
-    PortableWriterExImpl(PortableContext ctx, int off, int typeId, boolean metaEnabled) {
+    public PortableWriterExImpl(PortableContext ctx, int off, int typeId, boolean metaEnabled) {
         this(ctx, off);
 
         this.typeId = typeId;
@@ -320,14 +320,14 @@ public class PortableWriterExImpl implements PortableWriter, PortableRawWriterEx
     /**
      * @return Array.
      */
-    byte[] array() {
+    public byte[] array() {
         return wCtx.out.arrayCopy();
     }
 
     /**
      * @return Output stream.
      */
-    PortableOutputStream outputStream() {
+    public PortableOutputStream outputStream() {
         return wCtx.out;
     }
 
@@ -351,7 +351,7 @@ public class PortableWriterExImpl implements PortableWriter, PortableRawWriterEx
      * @param bytes Number of bytes to reserve.
      * @return Offset.
      */
-    int reserve(int bytes) {
+    public int reserve(int bytes) {
         int pos = wCtx.out.position();
 
         wCtx.out.position(pos + bytes);
@@ -363,7 +363,7 @@ public class PortableWriterExImpl implements PortableWriter, PortableRawWriterEx
      * @param bytes Number of bytes to reserve.
      * @return Offset.
      */
-    int reserveAndMark(int bytes) {
+    public int reserveAndMark(int bytes) {
         int off0 = reserve(bytes);
 
         mark = wCtx.out.position();
@@ -374,21 +374,21 @@ public class PortableWriterExImpl implements PortableWriter, PortableRawWriterEx
     /**
      * @param off Offset.
      */
-    void writeDelta(int off) {
+    public void writeDelta(int off) {
         wCtx.out.writeInt(off, wCtx.out.position() - mark);
     }
 
     /**
      *
      */
-    void writeLength() {
+    public void writeLength() {
         wCtx.out.writeInt(start + TOTAL_LEN_POS, wCtx.out.position() - start);
     }
 
     /**
      *
      */
-    void writeRawOffsetIfNeeded() {
+    public void writeRawOffsetIfNeeded() {
         if (allowFields)
             wCtx.out.writeInt(start + RAW_DATA_OFF_POS, wCtx.out.position() - start);
     }
@@ -416,63 +416,63 @@ public class PortableWriterExImpl implements PortableWriter, PortableRawWriterEx
     /**
      * @param val Value.
      */
-    void doWriteByte(byte val) {
+    public void doWriteByte(byte val) {
         wCtx.out.writeByte(val);
     }
 
     /**
      * @param val Value.
      */
-    void doWriteShort(short val) {
+    public void doWriteShort(short val) {
         wCtx.out.writeShort(val);
     }
 
     /**
      * @param val Value.
      */
-    void doWriteInt(int val) {
+    public void doWriteInt(int val) {
         wCtx.out.writeInt(val);
     }
 
     /**
      * @param val Value.
      */
-    void doWriteLong(long val) {
+    public void doWriteLong(long val) {
         wCtx.out.writeLong(val);
     }
 
     /**
      * @param val Value.
      */
-    void doWriteFloat(float val) {
+    public void doWriteFloat(float val) {
         wCtx.out.writeFloat(val);
     }
 
     /**
      * @param val Value.
      */
-    void doWriteDouble(double val) {
+    public void doWriteDouble(double val) {
         wCtx.out.writeDouble(val);
     }
 
     /**
      * @param val Value.
      */
-    void doWriteChar(char val) {
+    public void doWriteChar(char val) {
         wCtx.out.writeChar(val);
     }
 
     /**
      * @param val Value.
      */
-    void doWriteBoolean(boolean val) {
+    public void doWriteBoolean(boolean val) {
         wCtx.out.writeBoolean(val);
     }
 
     /**
      * @param val String value.
      */
-    void doWriteDecimal(@Nullable BigDecimal val) {
+    public void doWriteDecimal(@Nullable BigDecimal val) {
         if (val == null)
             doWriteByte(NULL);
         else {
@@ -498,7 +498,7 @@ public class PortableWriterExImpl implements PortableWriter, PortableRawWriterEx
     /**
      * @param val String value.
      */
-    void doWriteString(@Nullable String val) {
+    public void doWriteString(@Nullable String val) {
         if (val == null)
             doWriteByte(NULL);
         else {
@@ -528,7 +528,7 @@ public class PortableWriterExImpl implements PortableWriter, PortableRawWriterEx
     /**
      * @param uuid UUID.
      */
-    void doWriteUuid(@Nullable UUID uuid) {
+    public void doWriteUuid(@Nullable UUID uuid) {
         if (uuid == null)
             doWriteByte(NULL);
         else {
@@ -541,7 +541,7 @@ public class PortableWriterExImpl implements PortableWriter, PortableRawWriterEx
     /**
      * @param date Date.
      */
-    void doWriteDate(@Nullable Date date) {
+    public void doWriteDate(@Nullable Date date) {
         if (date == null)
             doWriteByte(NULL);
         else {
@@ -554,7 +554,7 @@ public class PortableWriterExImpl implements PortableWriter, PortableRawWriterEx
     /**
      * @param ts Timestamp.
      */
-    void doWriteTimestamp(@Nullable Timestamp ts) {
+    public void doWriteTimestamp(@Nullable Timestamp ts) {
         if (ts == null)
             doWriteByte(NULL);
         else {
@@ -569,7 +569,7 @@ public class PortableWriterExImpl implements PortableWriter, PortableRawWriterEx
      * @param detached Detached or not.
      * @throws PortableException In case of error.
      */
-    void doWriteObject(@Nullable Object obj, boolean detached) throws PortableException {
+    public void doWriteObject(@Nullable Object obj, boolean detached) throws PortableException {
         if (obj == null)
             doWriteByte(NULL);
         else {
@@ -591,6 +591,9 @@ public class PortableWriterExImpl implements PortableWriter, PortableRawWriterEx
         if (val == null)
             doWriteByte(NULL);
         else {
+            if (tryWriteAsHandle(val))
+                return;
+
             doWriteByte(BYTE_ARR);
             doWriteInt(val.length);
 
@@ -605,6 +608,9 @@ public class PortableWriterExImpl implements PortableWriter, PortableRawWriterEx
         if (val == null)
             doWriteByte(NULL);
         else {
+            if (tryWriteAsHandle(val))
+                return;
+
             doWriteByte(SHORT_ARR);
             doWriteInt(val.length);
 
@@ -619,6 +625,9 @@ public class PortableWriterExImpl implements PortableWriter, PortableRawWriterEx
         if (val == null)
             doWriteByte(NULL);
         else {
+            if (tryWriteAsHandle(val))
+                return;
+
             doWriteByte(INT_ARR);
             doWriteInt(val.length);
 
@@ -633,6 +642,9 @@ public class PortableWriterExImpl implements PortableWriter, PortableRawWriterEx
         if (val == null)
             doWriteByte(NULL);
         else {
+            if (tryWriteAsHandle(val))
+                return;
+
             doWriteByte(LONG_ARR);
             doWriteInt(val.length);
 
@@ -647,6 +659,9 @@ public class PortableWriterExImpl implements PortableWriter, PortableRawWriterEx
         if (val == null)
             doWriteByte(NULL);
         else {
+            if (tryWriteAsHandle(val))
+                return;
+
             doWriteByte(FLOAT_ARR);
             doWriteInt(val.length);
 
@@ -661,6 +676,9 @@ public class PortableWriterExImpl implements PortableWriter, PortableRawWriterEx
         if (val == null)
             doWriteByte(NULL);
         else {
+            if (tryWriteAsHandle(val))
+                return;
+
             doWriteByte(DOUBLE_ARR);
             doWriteInt(val.length);
 
@@ -675,6 +693,9 @@ public class PortableWriterExImpl implements PortableWriter, PortableRawWriterEx
         if (val == null)
             doWriteByte(NULL);
         else {
+            if (tryWriteAsHandle(val))
+                return;
+
             doWriteByte(CHAR_ARR);
             doWriteInt(val.length);
 
@@ -689,6 +710,9 @@ public class PortableWriterExImpl implements PortableWriter, PortableRawWriterEx
         if (val == null)
             doWriteByte(NULL);
         else {
+            if (tryWriteAsHandle(val))
+                return;
+
             doWriteByte(BOOLEAN_ARR);
             doWriteInt(val.length);
 
@@ -703,6 +727,9 @@ public class PortableWriterExImpl implements PortableWriter, PortableRawWriterEx
         if (val == null)
             doWriteByte(NULL);
         else {
+            if (tryWriteAsHandle(val))
+                return;
+
             doWriteByte(DECIMAL_ARR);
             doWriteInt(val.length);
 
@@ -718,6 +745,9 @@ public class PortableWriterExImpl implements PortableWriter, PortableRawWriterEx
         if (val == null)
             doWriteByte(NULL);
         else {
+            if (tryWriteAsHandle(val))
+                return;
+
             doWriteByte(STRING_ARR);
             doWriteInt(val.length);
 
@@ -733,6 +763,9 @@ public class PortableWriterExImpl implements PortableWriter, PortableRawWriterEx
         if (val == null)
             doWriteByte(NULL);
         else {
+            if (tryWriteAsHandle(val))
+                return;
+
             doWriteByte(UUID_ARR);
             doWriteInt(val.length);
 
@@ -748,6 +781,9 @@ public class PortableWriterExImpl implements PortableWriter, PortableRawWriterEx
         if (val == null)
             doWriteByte(NULL);
         else {
+            if (tryWriteAsHandle(val))
+                return;
+
             doWriteByte(DATE_ARR);
             doWriteInt(val.length);
 
@@ -764,6 +800,9 @@ public class PortableWriterExImpl implements PortableWriter, PortableRawWriterEx
         if (val == null)
             doWriteByte(NULL);
         else {
+            if (tryWriteAsHandle(val))
+                return;
+
             PortableContext.Type type = ctx.typeId(val.getClass().getComponentType());
 
             doWriteByte(OBJ_ARR);
@@ -790,6 +829,9 @@ public class PortableWriterExImpl implements PortableWriter, PortableRawWriterEx
         if (col == null)
             doWriteByte(NULL);
         else {
+            if (tryWriteAsHandle(col))
+                return;
+
             doWriteByte(COL);
             doWriteInt(col.size());
             doWriteByte(ctx.collectionType(col.getClass()));
@@ -807,6 +849,9 @@ public class PortableWriterExImpl implements PortableWriter, PortableRawWriterEx
         if (map == null)
             doWriteByte(NULL);
         else {
+            if (tryWriteAsHandle(map))
+                return;
+
             doWriteByte(MAP);
             doWriteInt(map.size());
             doWriteByte(ctx.mapType(map.getClass()));
@@ -826,6 +871,9 @@ public class PortableWriterExImpl implements PortableWriter, PortableRawWriterEx
         if (e == null)
             doWriteByte(NULL);
         else {
+            if (tryWriteAsHandle(e))
+                return;
+
             doWriteByte(MAP_ENTRY);
             doWriteObject(e.getKey(), false);
             doWriteObject(e.getValue(), false);
@@ -905,7 +953,7 @@ public class PortableWriterExImpl implements PortableWriter, PortableRawWriterEx
     /**
      * @param po Portable object.
      */
-    void doWritePortableObject(@Nullable PortableObjectImpl po) {
+    public void doWritePortableObject(@Nullable PortableObjectImpl po) {
         if (po == null)
             doWriteByte(NULL);
         else {
@@ -1106,64 +1154,88 @@ public class PortableWriterExImpl implements PortableWriter, PortableRawWriterEx
      * @param val Value.
      */
     void writeByteArrayField(@Nullable byte[] val) {
-        doWriteInt(val != null ? 5 + val.length : 1);
+        int lenPos = reserveAndMark(4);
+
         doWriteByteArray(val);
+
+        writeDelta(lenPos);
     }
 
     /**
      * @param val Value.
      */
     void writeShortArrayField(@Nullable short[] val) {
-        doWriteInt(val != null ? 5 + (val.length << 1) : 1);
+        int lenPos = reserveAndMark(4);
+
         doWriteShortArray(val);
+
+        writeDelta(lenPos);
     }
 
     /**
      * @param val Value.
      */
     void writeIntArrayField(@Nullable int[] val) {
-        doWriteInt(val != null ? 5 + (val.length << 2) : 1);
+        int lenPos = reserveAndMark(4);
+
         doWriteIntArray(val);
+
+        writeDelta(lenPos);
     }
 
     /**
      * @param val Value.
      */
     void writeLongArrayField(@Nullable long[] val) {
-        doWriteInt(val != null ? 5 + (val.length << 3) : 1);
+        int lenPos = reserveAndMark(4);
+
         doWriteLongArray(val);
+
+        writeDelta(lenPos);
     }
 
     /**
      * @param val Value.
      */
     void writeFloatArrayField(@Nullable float[] val) {
-        doWriteInt(val != null ? 5 + (val.length << 2) : 1);
+        int lenPos = reserveAndMark(4);
+
         doWriteFloatArray(val);
+
+        writeDelta(lenPos);
     }
 
     /**
      * @param val Value.
      */
     void writeDoubleArrayField(@Nullable double[] val) {
-        doWriteInt(val != null ? 5 + (val.length << 3) : 1);
+        int lenPos = reserveAndMark(4);
+
         doWriteDoubleArray(val);
+
+        writeDelta(lenPos);
     }
 
     /**
      * @param val Value.
      */
     void writeCharArrayField(@Nullable char[] val) {
-        doWriteInt(val != null ? 5 + (val.length << 1) : 1);
+        int lenPos = reserveAndMark(4);
+
         doWriteCharArray(val);
+
+        writeDelta(lenPos);
     }
 
     /**
      * @param val Value.
      */
     void writeBooleanArrayField(@Nullable boolean[] val) {
-        doWriteInt(val != null ? 5 + val.length : 1);
+        int lenPos = reserveAndMark(4);
+
         doWriteBooleanArray(val);
+
+        writeDelta(lenPos);
     }
 
     /**
@@ -1739,12 +1811,31 @@ public class PortableWriterExImpl implements PortableWriter, PortableRawWriterEx
         doWriteInt(id);
     }
 
+     /**
+      * Attempts to write the object as a handle.
+      *
+      * @param obj Object to write.
+      * @return {@code true} if the object has been written as a handle.
+      */
+     boolean tryWriteAsHandle(Object obj) {
+         int handle = handle(obj);
+
+         if (handle >= 0) {
+             doWriteByte(GridPortableMarshaller.HANDLE);
+             doWriteInt(handle);
+
+             return true;
+         }
+
+         return false;
+     }
+
     /**
      * Create new writer with same context.
      * @param typeId type
      * @return New writer.
      */
-    PortableWriterExImpl newWriter(int typeId) {
+    public PortableWriterExImpl newWriter(int typeId) {
         PortableWriterExImpl res = new PortableWriterExImpl(ctx, wCtx);
 
         res.typeId = typeId;
@@ -1755,7 +1846,7 @@ public class PortableWriterExImpl implements PortableWriter, PortableRawWriterEx
     /**
      * @return Portable context.
      */
-    PortableContext context() {
+    public PortableContext context() {
         return ctx;
     }
 
@@ -1803,4 +1894,4 @@ public class PortableWriterExImpl implements PortableWriter, PortableRawWriterEx
             handles = new IdentityHashMap<>();
         }
     }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/9057a4c0/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/PortableAbstractLazyValue.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/PortableAbstractLazyValue.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/PortableAbstractLazyValue.java
new file mode 100644
index 0000000..1f521ac
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/PortableAbstractLazyValue.java
@@ -0,0 +1,57 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.portable.builder;
+
+/**
+ *
+ */
+abstract class PortableAbstractLazyValue implements PortableLazyValue {
+    /** */
+    protected Object val;
+
+    /** */
+    protected final PortableBuilderReader reader;
+
+    /** */
+    protected final int valOff;
+
+    /**
+     * @param reader Reader.
+     * @param valOff Value.
+     */
+    protected PortableAbstractLazyValue(PortableBuilderReader reader, int valOff) {
+        this.reader = reader;
+        this.valOff = valOff;
+    }
+
+    /**
+     * @return Value.
+     */
+    protected abstract Object init();
+
+    /** {@inheritDoc} */
+    @Override public Object value() {
+        if (val == null) {
+            val = init();
+
+            assert val != null;
+        }
+
+        return val;
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/9057a4c0/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/PortableBuilderEnum.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/PortableBuilderEnum.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/PortableBuilderEnum.java
new file mode 100644
index 0000000..1472d56
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/PortableBuilderEnum.java
@@ -0,0 +1,116 @@
+/*
+ * 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.GridPortableMarshaller;
+import org.apache.ignite.internal.portable.PortableWriterExImpl;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.portable.PortableInvalidClassException;
+
+/**
+ *
+ */
+public class PortableBuilderEnum implements PortableBuilderSerializationAware {
+    /** */
+    private final int ordinal;
+
+    /** */
+    private final int typeId;
+
+    /** */
+    private final String clsName;
+
+    /**
+     * @param typeId Type ID.
+     * @param anEnum Enum instance.
+     */
+    public PortableBuilderEnum(int typeId, Enum anEnum) {
+        ordinal = anEnum.ordinal();
+        this.typeId = typeId;
+        clsName = null;
+    }
+
+    /**
+     * @param reader PortableBuilderReader.
+     */
+    public PortableBuilderEnum(PortableBuilderReader reader) {
+        int typeId = reader.readInt();
+
+        if (typeId == GridPortableMarshaller.UNREGISTERED_TYPE_ID) {
+            clsName = reader.readString();
+
+            Class cls;
+
+            try {
+                // TODO: IGNITE-1272 - Is class loader needed here?
+                cls = U.forName(reader.readString(), null);
+            }
+            catch (ClassNotFoundException e) {
+                throw new PortableInvalidClassException("Failed to load the class: " + clsName, e);
+            }
+
+            this.typeId = reader.portableContext().descriptorForClass(cls).typeId();
+        }
+        else {
+            this.typeId = typeId;
+            this.clsName = null;
+        }
+
+        ordinal = reader.readInt();
+    }
+
+    /**
+     * @return Ordinal.
+     */
+    public int getOrdinal() {
+        return ordinal;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeTo(PortableWriterExImpl writer, PortableBuilderSerializer ctx) {
+        writer.writeByte(GridPortableMarshaller.ENUM);
+
+        if (typeId == GridPortableMarshaller.UNREGISTERED_TYPE_ID) {
+            writer.writeInt(GridPortableMarshaller.UNREGISTERED_TYPE_ID);
+            writer.writeString(clsName);
+        }
+        else
+            writer.writeInt(typeId);
+
+        writer.writeInt(ordinal);
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean equals(Object o) {
+        if (o == null || getClass() != o.getClass())
+            return false;
+
+        PortableBuilderEnum that = (PortableBuilderEnum)o;
+
+        return ordinal == that.ordinal && typeId == that.typeId;
+    }
+
+    /** {@inheritDoc} */
+    @Override public int hashCode() {
+        int result = ordinal;
+
+        result = 31 * result + typeId;
+
+        return result;
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/9057a4c0/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/PortableBuilderImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/PortableBuilderImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/PortableBuilderImpl.java
new file mode 100644
index 0000000..b2e4c0d
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/PortableBuilderImpl.java
@@ -0,0 +1,537 @@
+/*
+ * 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 java.util.Collections;
+import java.util.HashMap;
+import java.util.LinkedHashMap;
+import java.util.Map;
+import java.util.Set;
+import org.apache.ignite.internal.processors.cache.portable.CacheObjectPortableProcessorImpl;
+import org.apache.ignite.internal.util.GridArgumentCheck;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.portable.PortableBuilder;
+import org.apache.ignite.portable.PortableException;
+import org.apache.ignite.portable.PortableInvalidClassException;
+import org.apache.ignite.portable.PortableMetadata;
+import org.apache.ignite.portable.PortableObject;
+import org.jetbrains.annotations.Nullable;
+import org.apache.ignite.internal.portable.*;
+import org.apache.ignite.internal.processors.cache.portable.*;
+import org.apache.ignite.internal.util.*;
+import org.apache.ignite.internal.util.typedef.internal.*;
+import org.apache.ignite.portable.*;
+
+import static org.apache.ignite.internal.portable.GridPortableMarshaller.CLS_NAME_POS;
+import static org.apache.ignite.internal.portable.GridPortableMarshaller.DFLT_HDR_LEN;
+import static org.apache.ignite.internal.portable.GridPortableMarshaller.HASH_CODE_POS;
+import static org.apache.ignite.internal.portable.GridPortableMarshaller.RAW_DATA_OFF_POS;
+import static org.apache.ignite.internal.portable.GridPortableMarshaller.TOTAL_LEN_POS;
+import static org.apache.ignite.internal.portable.GridPortableMarshaller.TYPE_ID_POS;
+import static org.apache.ignite.internal.portable.GridPortableMarshaller.UNREGISTERED_TYPE_ID;
+
+/**
+ *
+ */
+public class PortableBuilderImpl implements PortableBuilder {
+    /** */
+    private static final Object REMOVED_FIELD_MARKER = new Object();
+
+    /** */
+    private final PortableContext ctx;
+
+    /** */
+    private final int typeId;
+
+    /** May be null. */
+    private String typeName;
+
+    /** May be null. */
+    private String clsNameToWrite;
+
+    /** */
+    private boolean registeredType = true;
+
+    /** */
+    private Map<String, Object> assignedVals;
+
+    /** */
+    private Map<Integer, Object> readCache;
+
+    /** Position of object in source array, or -1 if object is not created from PortableObject. */
+    private final int start;
+
+    /** Total header length */
+    private final int hdrLen;
+
+    /**
+     * Context of PortableObject reading process. Or {@code null} if object is not created from PortableObject.
+     */
+    private final PortableBuilderReader reader;
+
+    /** */
+    private int hashCode;
+
+    /**
+     * @param clsName Class name.
+     * @param ctx Portable context.
+     */
+    public PortableBuilderImpl(PortableContext ctx, String clsName) {
+        this(ctx, ctx.typeId(clsName), PortableContext.typeName(clsName));
+    }
+
+    /**
+     * @param typeId Type ID.
+     * @param ctx Portable context.
+     */
+    public PortableBuilderImpl(PortableContext ctx, int typeId) {
+        this(ctx, typeId, null);
+    }
+
+    /**
+     * @param typeName Type name.
+     * @param ctx Context.
+     * @param typeId Type id.
+     */
+    public PortableBuilderImpl(PortableContext ctx, int typeId, String typeName) {
+        this.typeId = typeId;
+        this.typeName = typeName;
+        this.ctx = ctx;
+
+        start = -1;
+        reader = null;
+        hdrLen = DFLT_HDR_LEN;
+
+        readCache = Collections.emptyMap();
+    }
+
+    /**
+     * @param obj Object to wrap.
+     */
+    public PortableBuilderImpl(PortableObjectImpl obj) {
+        this(new PortableBuilderReader(obj), obj.start());
+
+        reader.registerObject(this);
+    }
+
+    /**
+     * @param reader ctx
+     * @param start Start.
+     */
+    PortableBuilderImpl(PortableBuilderReader reader, int start) {
+        this.reader = reader;
+        this.start = start;
+
+        int typeId = reader.readIntAbsolute(start + TYPE_ID_POS);
+        ctx = reader.portableContext();
+        hashCode = reader.readIntAbsolute(start + HASH_CODE_POS);
+
+        if (typeId == UNREGISTERED_TYPE_ID) {
+            int mark = reader.position();
+
+            reader.position(start + CLS_NAME_POS);
+
+            clsNameToWrite = reader.readString();
+
+            Class cls;
+
+            try {
+                // TODO: IGNITE-1272 - Is class loader needed here?
+                cls = U.forName(clsNameToWrite, null);
+            }
+            catch (ClassNotFoundException e) {
+                throw new PortableInvalidClassException("Failed to load the class: " + clsNameToWrite, e);
+            }
+
+            this.typeId = ctx.descriptorForClass(cls).typeId();
+
+            registeredType = false;
+
+            hdrLen = reader.position() - mark;
+
+            reader.position(mark);
+        }
+        else {
+            this.typeId = typeId;
+            hdrLen = DFLT_HDR_LEN;
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public PortableObject build() {
+        try (PortableWriterExImpl writer = new PortableWriterExImpl(ctx, 0, typeId, false)) {
+
+            PortableBuilderSerializer serializationCtx = new PortableBuilderSerializer();
+
+            serializationCtx.registerObjectWriting(this, 0);
+
+            serializeTo(writer, serializationCtx);
+
+            byte[] arr = writer.array();
+
+            return new PortableObjectImpl(ctx, arr, 0);
+        }
+    }
+
+    /**
+     * @param writer Writer.
+     * @param serializer Serializer.
+     */
+    void serializeTo(PortableWriterExImpl writer, PortableBuilderSerializer serializer) {
+        writer.doWriteByte(GridPortableMarshaller.OBJ);
+        writer.doWriteBoolean(true);
+        writer.doWriteInt(registeredType ? typeId : UNREGISTERED_TYPE_ID);
+        writer.doWriteInt(hashCode);
+
+        // Length and raw offset.
+        writer.reserve(8);
+
+        if (!registeredType)
+            writer.writeString(clsNameToWrite);
+
+        Set<Integer> remainsFlds = null;
+
+        if (reader != null) {
+            Map<Integer, Object> assignedFldsById;
+
+            if (assignedVals != null) {
+                assignedFldsById = U.newHashMap(assignedVals.size());
+
+                for (Map.Entry<String, Object> entry : assignedVals.entrySet()) {
+                    int fldId = ctx.fieldId(typeId, entry.getKey());
+
+                    assignedFldsById.put(fldId, entry.getValue());
+                }
+
+                remainsFlds = assignedFldsById.keySet();
+            }
+            else
+                assignedFldsById = Collections.emptyMap();
+
+            int rawOff = start + reader.readIntAbsolute(start + RAW_DATA_OFF_POS);
+
+            reader.position(start + hdrLen);
+
+            int cpStart = -1;
+
+            while (reader.position() < rawOff) {
+                int fldId = reader.readInt();
+
+                int len = reader.readInt();
+
+                if (assignedFldsById.containsKey(fldId)) {
+                    if (cpStart >= 0) {
+                        writer.write(reader.array(), cpStart, reader.position() - 4 - 4 - cpStart);
+
+                        cpStart = -1;
+                    }
+
+                    Object assignedVal = assignedFldsById.remove(fldId);
+
+                    reader.skip(len);
+
+                    if (assignedVal != REMOVED_FIELD_MARKER) {
+                        writer.writeInt(fldId);
+
+                        int lenPos = writer.reserveAndMark(4);
+
+                        serializer.writeValue(writer, assignedVal);
+
+                        writer.writeDelta(lenPos);
+                    }
+                }
+                else {
+                    int type = len != 0 ? reader.readByte(0) : 0;
+
+                    if (len != 0 && !PortableUtils.isPlainArrayType(type) && PortableUtils.isPlainType(type)) {
+                        if (cpStart < 0)
+                            cpStart = reader.position() - 4 - 4;
+
+                        reader.skip(len);
+                    }
+                    else {
+                        if (cpStart >= 0) {
+                            writer.write(reader.array(), cpStart, reader.position() - 4 - cpStart);
+
+                            cpStart = -1;
+                        }
+                        else
+                            writer.writeInt(fldId);
+
+                        Object val;
+
+                        if (len == 0)
+                            val = null;
+                        else if (readCache == null) {
+                            int savedPos = reader.position();
+
+                            val = reader.parseValue();
+
+                            assert reader.position() == savedPos + len;
+                        }
+                        else {
+                            val = readCache.get(fldId);
+
+                            reader.skip(len);
+                        }
+
+                        int lenPos = writer.reserveAndMark(4);
+
+                        serializer.writeValue(writer, val);
+
+                        writer.writeDelta(lenPos);
+                    }
+                }
+            }
+
+            if (cpStart >= 0)
+                writer.write(reader.array(), cpStart, reader.position() - cpStart);
+        }
+
+        if (assignedVals != null && (remainsFlds == null || !remainsFlds.isEmpty())) {
+            boolean metadataEnabled = ctx.isMetaDataEnabled(typeId);
+
+            PortableMetadata metadata = null;
+
+            if (metadataEnabled)
+                metadata = ctx.metaData(typeId);
+
+            Map<String, String> newFldsMetadata = null;
+
+            for (Map.Entry<String, Object> entry : assignedVals.entrySet()) {
+                Object val = entry.getValue();
+
+                if (val == REMOVED_FIELD_MARKER)
+                    continue;
+
+                String name = entry.getKey();
+
+                int fldId = ctx.fieldId(typeId, name);
+
+                if (remainsFlds != null && !remainsFlds.contains(fldId))
+                    continue;
+
+                writer.writeInt(fldId);
+
+                int lenPos = writer.reserveAndMark(4);
+
+                serializer.writeValue(writer, val);
+
+                writer.writeDelta(lenPos);
+
+                if (metadataEnabled) {
+                    String oldFldTypeName = metadata == null ? null : metadata.fieldTypeName(name);
+
+                    String newFldTypeName;
+
+                    if (val instanceof PortableValueWithType)
+                        newFldTypeName = ((PortableValueWithType)val).typeName();
+                    else {
+                        byte type = PortableUtils.typeByClass(val.getClass());
+
+                        newFldTypeName = CacheObjectPortableProcessorImpl.fieldTypeName(type);
+                    }
+
+                    if (oldFldTypeName == null) {
+                        // It's a new field, we have to add it to metadata.
+
+                        if (newFldsMetadata == null)
+                            newFldsMetadata = new HashMap<>();
+
+                        newFldsMetadata.put(name, newFldTypeName);
+                    }
+                    else {
+                        if (!"Object".equals(oldFldTypeName) && !oldFldTypeName.equals(newFldTypeName)) {
+                            throw new PortableException(
+                                "Wrong value has been set [" +
+                                    "typeName=" + (typeName == null ? metadata.typeName() : typeName) +
+                                    ", fieldName=" + name +
+                                    ", fieldType=" + oldFldTypeName +
+                                    ", assignedValueType=" + newFldTypeName +
+                                    ", assignedValue=" + (((PortableValueWithType)val).value()) + ']'
+                            );
+                        }
+                    }
+                }
+            }
+
+            if (newFldsMetadata != null) {
+                String typeName = this.typeName;
+
+                if (typeName == null)
+                    typeName = metadata.typeName();
+
+                ctx.updateMetaData(typeId, typeName, newFldsMetadata);
+            }
+        }
+
+        writer.writeRawOffsetIfNeeded();
+
+        if (reader != null) {
+            int rawOff = reader.readIntAbsolute(start + RAW_DATA_OFF_POS);
+            int len = reader.readIntAbsolute(start + TOTAL_LEN_POS);
+
+            if (rawOff < len)
+                writer.write(reader.array(), rawOff, len - rawOff);
+        }
+
+        writer.writeLength();
+    }
+
+    /** {@inheritDoc} */
+    @Override public PortableBuilderImpl hashCode(int hashCode) {
+        this.hashCode = hashCode;
+
+        return this;
+    }
+
+    /**
+     *
+     */
+    private void ensureReadCacheInit() {
+        if (readCache == null) {
+            Map<Integer, Object> readCache = new HashMap<>();
+
+            int pos = start + hdrLen;
+            int end = start + reader.readIntAbsolute(start + RAW_DATA_OFF_POS);
+
+            while (pos < end) {
+                int fieldId = reader.readIntAbsolute(pos);
+
+                pos += 4;
+
+                int len = reader.readIntAbsolute(pos);
+
+                pos += 4;
+
+                Object val = reader.getValueQuickly(pos, len);
+
+                readCache.put(fieldId, val);
+
+                pos += len;
+            }
+
+            this.readCache = readCache;
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public <F> F getField(String name) {
+        Object val;
+
+        if (assignedVals != null && assignedVals.containsKey(name)) {
+            val = assignedVals.get(name);
+
+            if (val == REMOVED_FIELD_MARKER)
+                return null;
+        }
+        else {
+            ensureReadCacheInit();
+
+            int fldId = ctx.fieldId(typeId, name);
+
+            val = readCache.get(fldId);
+        }
+
+        return (F)PortableUtils.unwrapLazy(val);
+    }
+
+    /** {@inheritDoc} */
+    @Override public PortableBuilder setField(String name, Object val) {
+        GridArgumentCheck.notNull(val, name);
+
+        if (assignedVals == null)
+            assignedVals = new LinkedHashMap<>();
+
+        Object oldVal = assignedVals.put(name, val);
+
+        if (oldVal instanceof PortableValueWithType) {
+            ((PortableValueWithType)oldVal).value(val);
+
+            assignedVals.put(name, oldVal);
+        }
+
+        return this;
+    }
+
+    /** {@inheritDoc} */
+    @Override public <T> PortableBuilder setField(String name, @Nullable T val, Class<? super T> type) {
+        if (assignedVals == null)
+            assignedVals = new LinkedHashMap<>();
+
+        //int fldId = ctx.fieldId(typeId, fldName);
+
+        assignedVals.put(name, new PortableValueWithType(PortableUtils.typeByClass(type), val));
+
+        return this;
+    }
+
+    /** {@inheritDoc} */
+    @Override public PortableBuilder setField(String name, @Nullable PortableBuilder builder) {
+        if (builder == null)
+            return setField(name, null, Object.class);
+        else
+            return setField(name, (Object)builder);
+    }
+
+    /**
+     * Removes field from portable object.
+     *
+     * @param name Field name.
+     * @return {@code this} instance for chaining.
+     */
+    @Override public PortableBuilderImpl removeField(String name) {
+        if (assignedVals == null)
+            assignedVals = new LinkedHashMap<>();
+
+        assignedVals.put(name, REMOVED_FIELD_MARKER);
+
+        return this;
+    }
+
+    /**
+     * Creates builder initialized by specified portable object.
+     *
+     * @param obj Portable object to initialize builder.
+     * @return New builder.
+     */
+    public static PortableBuilderImpl wrap(PortableObject obj) {
+        PortableObjectImpl heapObj;
+
+        if (obj instanceof PortableObjectOffheapImpl)
+            heapObj = (PortableObjectImpl)((PortableObjectOffheapImpl)obj).heapCopy();
+        else
+            heapObj = (PortableObjectImpl)obj;
+
+        return new PortableBuilderImpl(heapObj);
+    }
+
+    /**
+     * @return Object start position in source array.
+     */
+    int start() {
+        return start;
+    }
+
+    /**
+     * @return Object type id.
+     */
+    public int typeId() {
+        return typeId;
+    }
+}
\ No newline at end of file


[10/39] ignite git commit: Merge remote-tracking branch 'remotes/origin/ignite-1.3.3-p7' into ignite-1.4

Posted by ag...@apache.org.
Merge remote-tracking branch 'remotes/origin/ignite-1.3.3-p7' into ignite-1.4


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

Branch: refs/heads/ignite-264
Commit: ba4a582ce19c548475e0f87f5bdedab0d6f4c66c
Parents: 7809d26
Author: sboikov <sb...@gridgain.com>
Authored: Wed Sep 2 14:01:21 2015 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Wed Sep 2 14:01:21 2015 +0300

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


http://git-wip-us.apache.org/repos/asf/ignite/blob/ba4a582c/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 57c1977..cf8c8d6 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
@@ -1871,7 +1871,7 @@ public class GridCacheProcessor extends GridProcessorAdapter {
                     if (!sysCache) {
                         DynamicCacheDescriptor desc = registeredCaches.get(maskNull(req.cacheName()));
 
-                        if (desc != null && !desc.cancelled() && desc.deploymentId().equals(req.deploymentId())) {
+                        if (desc != null && desc.deploymentId().equals(req.deploymentId())) {
                             Map<UUID, Boolean> nodes = batch.clientNodes().get(name);
 
                             assert nodes != null : req;


[17/39] ignite git commit: ignite-1273: fixed cyclic references processing by PortableMarshaller and ability to modify array fields returned by PortableBuilder

Posted by ag...@apache.org.
http://git-wip-us.apache.org/repos/asf/ignite/blob/6ac9557e/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
new file mode 100644
index 0000000..45355d7
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/PortableBuilderReader.java
@@ -0,0 +1,800 @@
+/*
+ * 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 java.sql.Timestamp;
+import java.util.Date;
+import java.util.HashMap;
+import java.util.Map;
+import org.apache.ignite.internal.portable.GridPortableMarshaller;
+import org.apache.ignite.internal.portable.PortableContext;
+import org.apache.ignite.internal.portable.PortableObjectImpl;
+import org.apache.ignite.internal.portable.PortablePrimitives;
+import org.apache.ignite.internal.portable.PortableReaderExImpl;
+import org.apache.ignite.internal.portable.PortableUtils;
+import org.apache.ignite.internal.portable.PortableWriterExImpl;
+import org.apache.ignite.portable.PortableException;
+
+import static java.nio.charset.StandardCharsets.UTF_8;
+import static org.apache.ignite.internal.portable.GridPortableMarshaller.NULL;
+import static org.apache.ignite.internal.portable.GridPortableMarshaller.STRING;
+
+/**
+ *
+ */
+class PortableBuilderReader {
+    /** */
+    private static final PortablePrimitives PRIM = PortablePrimitives.get();
+
+    /** */
+    private final Map<Integer, PortableBuilderImpl> objMap = new HashMap<>();
+
+    /** */
+    private final PortableContext ctx;
+
+    /** */
+    private final PortableReaderExImpl reader;
+
+    /** */
+    private byte[] arr;
+
+    /** */
+    private int pos;
+
+    /**
+     * @param objImpl Portable object
+     */
+    PortableBuilderReader(PortableObjectImpl objImpl) {
+        ctx = objImpl.context();
+        arr = objImpl.array();
+        pos = objImpl.start();
+
+        // TODO: IGNITE-1272 - Is class loader needed here?
+        reader = new PortableReaderExImpl(portableContext(), arr, pos, null);
+    }
+
+    /**
+     * @return Portable context.
+     */
+    public PortableContext portableContext() {
+        return ctx;
+    }
+
+    /**
+     * @param obj Mutable portable object.
+     */
+    public void registerObject(PortableBuilderImpl obj) {
+        objMap.put(obj.start(), obj);
+    }
+
+    /**
+     * @return Read int value.
+     */
+    public int readInt() {
+        int res = readInt(0);
+
+        pos += 4;
+
+        return res;
+    }
+
+    /**
+     * @return Read int value.
+     */
+    public byte readByte() {
+        return arr[pos++];
+    }
+
+    /**
+     * @return Read boolean value.
+     */
+    public boolean readBoolean() {
+        return readByte() == 1;
+    }
+
+    /**
+     * @return Read int value.
+     */
+    public byte readByte(int off) {
+        return arr[pos + off];
+    }
+
+    /**
+     * @param off Offset related to {@link #pos}
+     * @return Read int value.
+     */
+    public int readInt(int off) {
+        return PRIM.readInt(arr, pos + off);
+    }
+
+    /**
+     * @param pos Position in the source array.
+     * @return Read int value.
+     */
+    public int readIntAbsolute(int pos) {
+        return PRIM.readInt(arr, pos);
+    }
+
+    /**
+     * @return Read length of array.
+     */
+    public int readLength() {
+        return PRIM.readInt(arr, pos);
+    }
+
+    /**
+     * Read string length.
+     *
+     * @return String length.
+     */
+    public int readStringLength() {
+        boolean utf = PRIM.readBoolean(arr, pos);
+
+        int arrLen = PRIM.readInt(arr, pos + 1);
+
+        return 1 + (utf ? arrLen : arrLen << 1);
+    }
+
+    /**
+     * Reads string.
+     *
+     * @return String.
+     */
+    public String readString() {
+        byte flag = readByte();
+
+        if (flag == NULL)
+            return null;
+
+        if (flag != STRING)
+            throw new PortableException("Failed to deserialize String.");
+
+        boolean convert = readBoolean();
+        int len = readInt();
+
+        String str;
+
+        if (convert) {
+            str = new String(arr, pos, len, UTF_8);
+
+            pos += len;
+        }
+        else {
+            str = String.valueOf(PRIM.readCharArray(arr, pos, len));
+
+            pos += len << 1;
+        }
+
+        return str;
+    }
+
+    /**
+     *
+     */
+    public void skipValue() {
+        byte type = arr[pos++];
+
+        int len;
+
+        switch (type) {
+            case GridPortableMarshaller.NULL:
+                return;
+
+            case GridPortableMarshaller.OBJ:
+                pos += readInt(GridPortableMarshaller.TOTAL_LEN_POS - 1) - 1;
+
+                return;
+
+            case GridPortableMarshaller.BOOLEAN:
+            case GridPortableMarshaller.BYTE:
+                len = 1;
+                break;
+
+            case GridPortableMarshaller.CHAR:
+            case GridPortableMarshaller.SHORT:
+                len = 2;
+
+                break;
+
+            case GridPortableMarshaller.HANDLE:
+            case GridPortableMarshaller.FLOAT:
+            case GridPortableMarshaller.INT:
+                len = 4;
+
+                break;
+
+            case GridPortableMarshaller.ENUM:
+                //skipping type id and ordinal value
+                len = 8;
+
+                break;
+
+            case GridPortableMarshaller.LONG:
+            case GridPortableMarshaller.DOUBLE:
+                len = 8;
+
+                break;
+
+            case GridPortableMarshaller.BYTE_ARR:
+            case GridPortableMarshaller.BOOLEAN_ARR:
+                len = 4 + readLength();
+
+                break;
+
+            case GridPortableMarshaller.STRING:
+                len = 4 + readStringLength();
+
+                break;
+
+            case GridPortableMarshaller.DECIMAL:
+                len = /** scale */ 4  + /** mag len */ 4  + /** mag bytes count */ readInt(4);
+
+                break;
+
+            case GridPortableMarshaller.UUID:
+                len = 8 + 8;
+
+                break;
+
+            case GridPortableMarshaller.DATE:
+                len = 8 + 4;
+
+                break;
+
+            case GridPortableMarshaller.CHAR_ARR:
+            case GridPortableMarshaller.SHORT_ARR:
+                len = 4 + readLength() * 2;
+
+                break;
+
+            case GridPortableMarshaller.INT_ARR:
+            case GridPortableMarshaller.FLOAT_ARR:
+                len = 4 + readLength() * 4;
+
+                break;
+
+            case GridPortableMarshaller.LONG_ARR:
+            case GridPortableMarshaller.DOUBLE_ARR:
+                len = 4 + readLength() * 8;
+
+                break;
+
+            case GridPortableMarshaller.DECIMAL_ARR:
+            case GridPortableMarshaller.DATE_ARR:
+            case GridPortableMarshaller.OBJ_ARR:
+            case GridPortableMarshaller.ENUM_ARR:
+            case GridPortableMarshaller.UUID_ARR:
+            case GridPortableMarshaller.STRING_ARR: {
+                int size = readInt();
+
+                for (int i = 0; i < size; i++)
+                    skipValue();
+
+                return;
+            }
+
+            case GridPortableMarshaller.COL: {
+                int size = readInt();
+
+                pos++; // skip collection type
+
+                for (int i = 0; i < size; i++)
+                    skipValue();
+
+                return;
+            }
+
+            case GridPortableMarshaller.MAP: {
+                int size = readInt();
+
+                pos++; // skip collection type
+
+                for (int i = 0; i < size; i++) {
+                    skipValue(); // skip key.
+                    skipValue(); // skip value.
+                }
+
+                return;
+            }
+
+            case GridPortableMarshaller.MAP_ENTRY:
+                skipValue();
+                skipValue();
+
+                return;
+
+            case GridPortableMarshaller.PORTABLE_OBJ:
+                len = readInt() + 4;
+
+                break;
+
+            default:
+                throw new PortableException("Invalid flag value: " + type);
+        }
+
+        pos += len;
+    }
+
+    /**
+     * @param pos Position.
+     * @param len Length.
+     * @return Object.
+     */
+    public Object getValueQuickly(int pos, int len) {
+        byte type = arr[pos];
+
+        switch (type) {
+            case GridPortableMarshaller.NULL:
+                return null;
+
+            case GridPortableMarshaller.HANDLE: {
+                int objStart = pos - readIntAbsolute(pos + 1);
+
+                PortableBuilderImpl res = objMap.get(objStart);
+
+                if (res == null) {
+                    res = new PortableBuilderImpl(this, objStart);
+
+                    objMap.put(objStart, res);
+                }
+
+                return res;
+            }
+
+            case GridPortableMarshaller.OBJ: {
+                PortableBuilderImpl res = objMap.get(pos);
+
+                if (res == null) {
+                    res = new PortableBuilderImpl(this, pos);
+
+                    objMap.put(pos, res);
+                }
+
+                return res;
+            }
+
+            case GridPortableMarshaller.BYTE:
+                return arr[pos + 1];
+
+            case GridPortableMarshaller.SHORT:
+                return PRIM.readShort(arr, pos + 1);
+
+            case GridPortableMarshaller.INT:
+                return PRIM.readInt(arr, pos + 1);
+
+            case GridPortableMarshaller.LONG:
+                return PRIM.readLong(arr, pos + 1);
+
+            case GridPortableMarshaller.FLOAT:
+                return PRIM.readFloat(arr, pos + 1);
+
+            case GridPortableMarshaller.DOUBLE:
+                return PRIM.readDouble(arr, pos + 1);
+
+            case GridPortableMarshaller.CHAR:
+                return PRIM.readChar(arr, pos + 1);
+
+            case GridPortableMarshaller.BOOLEAN:
+                return arr[pos + 1] != 0;
+
+            case GridPortableMarshaller.DECIMAL:
+            case GridPortableMarshaller.STRING:
+            case GridPortableMarshaller.UUID:
+            case GridPortableMarshaller.DATE:
+                return new PortablePlainLazyValue(this, pos, len);
+
+            case GridPortableMarshaller.BYTE_ARR:
+            case GridPortableMarshaller.SHORT_ARR:
+            case GridPortableMarshaller.INT_ARR:
+            case GridPortableMarshaller.LONG_ARR:
+            case GridPortableMarshaller.FLOAT_ARR:
+            case GridPortableMarshaller.DOUBLE_ARR:
+            case GridPortableMarshaller.CHAR_ARR:
+            case GridPortableMarshaller.BOOLEAN_ARR:
+            case GridPortableMarshaller.DECIMAL_ARR:
+            case GridPortableMarshaller.DATE_ARR:
+            case GridPortableMarshaller.UUID_ARR:
+            case GridPortableMarshaller.STRING_ARR:
+            case GridPortableMarshaller.ENUM_ARR:
+            case GridPortableMarshaller.OBJ_ARR:
+            case GridPortableMarshaller.COL:
+            case GridPortableMarshaller.MAP:
+            case GridPortableMarshaller.MAP_ENTRY:
+                return new LazyCollection(pos);
+
+            case GridPortableMarshaller.ENUM: {
+                if (len == 1) {
+                    assert readByte(pos) == GridPortableMarshaller.NULL;
+
+                    return null;
+                }
+
+                int mark = position();
+                position(pos + 1);
+
+                PortableBuilderEnum builderEnum = new PortableBuilderEnum(this);
+
+                position(mark);
+
+                return builderEnum;
+            }
+
+            case GridPortableMarshaller.PORTABLE_OBJ: {
+                int size = readIntAbsolute(pos + 1);
+
+                int start = readIntAbsolute(pos + 4 + size);
+
+                PortableObjectImpl portableObj = new PortableObjectImpl(ctx, arr, pos + 4 + start);
+
+                return new PortablePlainPortableObject(portableObj);
+            }
+
+            default:
+                throw new PortableException("Invalid flag value: " + type);
+        }
+    }
+
+    /**
+     * @return Parsed value.
+     */
+    public Object parseValue() {
+        int valPos = pos;
+
+        byte type = arr[pos++];
+
+        int plainLazyValLen;
+
+        boolean modifiableLazyVal = false;
+
+        switch (type) {
+            case GridPortableMarshaller.NULL:
+                return null;
+
+            case GridPortableMarshaller.HANDLE: {
+                int objStart = pos - 1 - readInt();
+
+                PortableBuilderImpl res = objMap.get(objStart);
+
+                if (res == null) {
+                    res = new PortableBuilderImpl(this, objStart);
+
+                    objMap.put(objStart, res);
+                }
+
+                return res;
+            }
+
+            case GridPortableMarshaller.OBJ: {
+                pos--;
+
+                PortableBuilderImpl res = objMap.get(pos);
+
+                if (res == null) {
+                    res = new PortableBuilderImpl(this, pos);
+
+                    objMap.put(pos, res);
+                }
+
+                pos += readInt(GridPortableMarshaller.TOTAL_LEN_POS);
+
+                return res;
+            }
+
+            case GridPortableMarshaller.BYTE:
+                return arr[pos++];
+
+            case GridPortableMarshaller.SHORT: {
+                Object res = PRIM.readShort(arr, pos);
+                pos += 2;
+                return res;
+            }
+
+            case GridPortableMarshaller.INT:
+                return readInt();
+
+            case GridPortableMarshaller.LONG:
+                plainLazyValLen = 8;
+
+                break;
+
+            case GridPortableMarshaller.FLOAT:
+                plainLazyValLen = 4;
+
+                break;
+
+            case GridPortableMarshaller.DOUBLE:
+                plainLazyValLen = 8;
+
+                break;
+
+            case GridPortableMarshaller.CHAR:
+                plainLazyValLen = 2;
+
+                break;
+
+            case GridPortableMarshaller.BOOLEAN:
+                return arr[pos++] != 0;
+
+            case GridPortableMarshaller.DECIMAL:
+                plainLazyValLen = /** scale */ 4  + /** mag len */ 4  + /** mag bytes count */ readInt(4);
+
+                break;
+
+            case GridPortableMarshaller.STRING:
+                plainLazyValLen = 4 + readStringLength();
+
+                break;
+
+            case GridPortableMarshaller.UUID:
+                plainLazyValLen = 8 + 8;
+
+                break;
+
+            case GridPortableMarshaller.DATE:
+                plainLazyValLen = 8 + 4;
+
+                break;
+
+            case GridPortableMarshaller.BYTE_ARR:
+                plainLazyValLen = 4 + readLength();
+                modifiableLazyVal = true;
+
+                break;
+
+            case GridPortableMarshaller.SHORT_ARR:
+                plainLazyValLen = 4 + readLength() * 2;
+                modifiableLazyVal = true;
+
+                break;
+
+            case GridPortableMarshaller.INT_ARR:
+                plainLazyValLen = 4 + readLength() * 4;
+                modifiableLazyVal = true;
+
+                break;
+
+            case GridPortableMarshaller.LONG_ARR:
+                plainLazyValLen = 4 + readLength() * 8;
+                modifiableLazyVal = true;
+
+                break;
+
+            case GridPortableMarshaller.FLOAT_ARR:
+                plainLazyValLen = 4 + readLength() * 4;
+                modifiableLazyVal = true;
+
+                break;
+
+            case GridPortableMarshaller.DOUBLE_ARR:
+                plainLazyValLen = 4 + readLength() * 8;
+                modifiableLazyVal = true;
+
+                break;
+
+            case GridPortableMarshaller.CHAR_ARR:
+                plainLazyValLen = 4 + readLength() * 2;
+                modifiableLazyVal = true;
+
+                break;
+
+            case GridPortableMarshaller.BOOLEAN_ARR:
+                plainLazyValLen = 4 + readLength();
+                modifiableLazyVal = true;
+
+                break;
+
+            case GridPortableMarshaller.OBJ_ARR:
+                return new PortableObjectArrayLazyValue(this);
+
+            case GridPortableMarshaller.DATE_ARR: {
+                int size = readInt();
+
+                Date[] res = new Date[size];
+
+                for (int i = 0; i < res.length; i++) {
+                    byte flag = arr[pos++];
+
+                    if (flag == GridPortableMarshaller.NULL) continue;
+
+                    if (flag != GridPortableMarshaller.DATE)
+                        throw new PortableException("Invalid flag value: " + flag);
+
+                    long time = PRIM.readLong(arr, pos);
+
+                    pos += 8;
+
+                    if (ctx.isUseTimestamp()) {
+                        Timestamp ts = new Timestamp(time);
+
+                        ts.setNanos(ts.getNanos() + readInt());
+
+                        res[i] = ts;
+                    }
+                    else {
+                        res[i] = new Date(time);
+
+                        pos += 4;
+                    }
+                }
+
+                return res;
+            }
+
+            case GridPortableMarshaller.UUID_ARR:
+            case GridPortableMarshaller.STRING_ARR:
+            case GridPortableMarshaller.DECIMAL_ARR: {
+                int size = readInt();
+
+                for (int i = 0; i < size; i++) {
+                    byte flag = arr[pos++];
+
+                    if (flag == GridPortableMarshaller.UUID)
+                        pos += 8 + 8;
+                    else if (flag == GridPortableMarshaller.STRING)
+                        pos += 4 + readStringLength();
+                    else if (flag == GridPortableMarshaller.DECIMAL) {
+                        pos += 4; // scale value
+                        pos += 4 + readLength();
+                    }
+                    else
+                        assert flag == GridPortableMarshaller.NULL;
+                }
+
+                return new PortableModifiableLazyValue(this, valPos, pos - valPos);
+            }
+
+            case GridPortableMarshaller.COL: {
+                int size = readInt();
+                byte colType = arr[pos++];
+
+                switch (colType) {
+                    case GridPortableMarshaller.USER_COL:
+                    case GridPortableMarshaller.ARR_LIST:
+                        return new PortableLazyArrayList(this, size);
+
+                    case GridPortableMarshaller.LINKED_LIST:
+                        return new PortableLazyLinkedList(this, size);
+
+                    case GridPortableMarshaller.HASH_SET:
+                    case GridPortableMarshaller.LINKED_HASH_SET:
+                    case GridPortableMarshaller.TREE_SET:
+                    case GridPortableMarshaller.CONC_SKIP_LIST_SET:
+                        return new PortableLazySet(this, size);
+                }
+
+                throw new PortableException("Unknown collection type: " + colType);
+            }
+
+            case GridPortableMarshaller.MAP:
+                return PortableLazyMap.parseMap(this);
+
+            case GridPortableMarshaller.ENUM:
+                return new PortableBuilderEnum(this);
+
+            case GridPortableMarshaller.ENUM_ARR:
+                return new PortableEnumArrayLazyValue(this);
+
+            case GridPortableMarshaller.MAP_ENTRY:
+                return new PortableLazyMapEntry(this);
+
+            case GridPortableMarshaller.PORTABLE_OBJ: {
+                int size = readInt();
+
+                pos += size;
+
+                int start = readInt();
+
+                PortableObjectImpl portableObj = new PortableObjectImpl(ctx, arr,
+                    pos - 4 - size + start);
+
+                return new PortablePlainPortableObject(portableObj);
+            }
+
+
+            default:
+                throw new PortableException("Invalid flag value: " + type);
+        }
+
+        PortableAbstractLazyValue res;
+
+        if (modifiableLazyVal)
+            res = new PortableModifiableLazyValue(this, valPos, 1 + plainLazyValLen);
+        else
+            res = new PortablePlainLazyValue(this, valPos, 1 + plainLazyValLen);
+
+        pos += plainLazyValLen;
+
+        return res;
+    }
+
+    /**
+     * @return Array.
+     */
+    public byte[] array() {
+        return arr;
+    }
+
+    /**
+     * @return Position of reader.
+     */
+    public int position() {
+        return pos;
+    }
+
+    /**
+     * @param pos New pos.
+     */
+    public void position(int pos) {
+        this.pos = pos;
+    }
+
+    /**
+     * @param n Number of bytes to skip.
+     */
+    public void skip(int n) {
+        pos += n;
+    }
+
+    /**
+     * @return Reader.
+     */
+    PortableReaderExImpl reader() {
+        return reader;
+    }
+
+    /**
+     *
+     */
+    private class LazyCollection implements PortableLazyValue {
+        /** */
+        private final int valOff;
+
+        /** */
+        private Object col;
+
+        /**
+         * @param valOff Value.
+         */
+        protected LazyCollection(int valOff) {
+            this.valOff = valOff;
+        }
+
+        /**
+         * @return Object.
+         */
+        private Object wrappedCollection() {
+            if (col == null) {
+                position(valOff);
+
+                col = parseValue();
+            }
+
+            return col;
+        }
+
+        /** {@inheritDoc} */
+        @Override public void writeTo(PortableWriterExImpl writer, PortableBuilderSerializer ctx) {
+            ctx.writeValue(writer, wrappedCollection());
+        }
+
+        /** {@inheritDoc} */
+        @Override public Object value() {
+            return PortableUtils.unwrapLazy(wrappedCollection());
+        }
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/6ac9557e/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/PortableBuilderSerializationAware.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/PortableBuilderSerializationAware.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/PortableBuilderSerializationAware.java
new file mode 100644
index 0000000..976059a
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/PortableBuilderSerializationAware.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.portable.builder;
+
+import org.apache.ignite.internal.portable.*;
+
+/**
+ *
+ */
+interface PortableBuilderSerializationAware {
+    /**
+     * @param writer Writer.
+     * @param ctx Context.
+     */
+    public void writeTo(PortableWriterExImpl writer, PortableBuilderSerializer ctx);
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/6ac9557e/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
new file mode 100644
index 0000000..2d9c961
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/PortableBuilderSerializer.java
@@ -0,0 +1,214 @@
+/*
+ * 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.GridPortableMarshaller;
+import org.apache.ignite.internal.portable.PortableObjectEx;
+import org.apache.ignite.internal.portable.PortableUtils;
+import org.apache.ignite.internal.portable.PortableWriterExImpl;
+import org.apache.ignite.internal.util.*;
+import org.apache.ignite.portable.*;
+
+import java.util.*;
+
+/**
+ *
+ */
+class PortableBuilderSerializer {
+    /** */
+    private final Map<PortableBuilderImpl, Integer> objToPos = new IdentityHashMap<>();
+
+    /** */
+    private Map<PortableObject, PortableBuilderImpl> portableObjToWrapper;
+
+    /**
+     * @param obj Mutable object.
+     * @param posInResArr Object position in the array.
+     */
+    public void registerObjectWriting(PortableBuilderImpl obj, int posInResArr) {
+        objToPos.put(obj, posInResArr);
+    }
+
+    /**
+     * @param writer Writer.
+     * @param val Value.
+     */
+    public void writeValue(PortableWriterExImpl writer, Object val) {
+        if (val == null) {
+            writer.writeByte(GridPortableMarshaller.NULL);
+
+            return;
+        }
+
+        if (val instanceof PortableBuilderSerializationAware) {
+            ((PortableBuilderSerializationAware)val).writeTo(writer, this);
+
+            return;
+        }
+
+        if (val instanceof PortableObjectEx) {
+            if (portableObjToWrapper == null)
+                portableObjToWrapper = new IdentityHashMap<>();
+
+            PortableBuilderImpl wrapper = portableObjToWrapper.get(val);
+
+            if (wrapper == null) {
+                wrapper = PortableBuilderImpl.wrap((PortableObject)val);
+
+                portableObjToWrapper.put((PortableObject)val, wrapper);
+            }
+
+            val = wrapper;
+        }
+
+        if (val instanceof PortableBuilderImpl) {
+            PortableBuilderImpl obj = (PortableBuilderImpl)val;
+
+            Integer posInResArr = objToPos.get(obj);
+
+            if (posInResArr == null) {
+                objToPos.put(obj, writer.outputStream().position());
+
+                obj.serializeTo(writer.newWriter(obj.typeId()), this);
+            }
+            else {
+                int handle = writer.outputStream().position() - posInResArr;
+
+                writer.writeByte(GridPortableMarshaller.HANDLE);
+                writer.writeInt(handle);
+            }
+
+            return;
+        }
+
+        if (val.getClass().isEnum()) {
+            writer.writeByte(GridPortableMarshaller.ENUM);
+            writer.writeInt(writer.context().typeId(val.getClass().getName()));
+            writer.writeInt(((Enum)val).ordinal());
+
+            return;
+        }
+
+        if (val instanceof Collection) {
+            Collection<?> c = (Collection<?>)val;
+
+            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());
+
+
+            writer.writeByte(colType);
+
+            for (Object obj : c)
+                writeValue(writer, obj);
+
+            return;
+        }
+
+        if (val instanceof Map) {
+            Map<?, ?> map = (Map<?, ?>)val;
+
+            writer.writeByte(GridPortableMarshaller.MAP);
+            writer.writeInt(map.size());
+
+            writer.writeByte(writer.context().mapType(map.getClass()));
+
+            for (Map.Entry<?, ?> entry : map.entrySet()) {
+                writeValue(writer, entry.getKey());
+                writeValue(writer, entry.getValue());
+            }
+
+            return;
+        }
+
+        Byte flag = PortableUtils.PLAIN_CLASS_TO_FLAG.get(val.getClass());
+
+        if (flag != null) {
+            PortableUtils.writePlainObject(writer, val);
+
+            return;
+        }
+
+        if (val instanceof Object[]) {
+            int compTypeId = writer.context().typeId(((Object[])val).getClass().getComponentType().getName());
+
+            if (val instanceof PortableBuilderEnum[]) {
+                writeArray(writer, GridPortableMarshaller.ENUM_ARR, (Object[])val, compTypeId);
+
+                return;
+            }
+
+            if (((Object[])val).getClass().getComponentType().isEnum()) {
+                Enum[] enumArr = (Enum[])val;
+
+                writer.writeByte(GridPortableMarshaller.ENUM_ARR);
+                writer.writeInt(compTypeId);
+                writer.writeInt(enumArr.length);
+
+                for (Enum anEnum : enumArr)
+                    writeValue(writer, anEnum);
+
+                return;
+            }
+
+            writeArray(writer, GridPortableMarshaller.OBJ_ARR, (Object[])val, compTypeId);
+
+            return;
+        }
+
+        writer.doWriteObject(val, false);
+    }
+
+    /**
+     * @param writer Writer.
+     * @param elementType Element type.
+     * @param arr The array.
+     * @param compTypeId Component type ID.
+     */
+    public void writeArray(PortableWriterExImpl writer, byte elementType, Object[] arr, int compTypeId) {
+        writer.writeByte(elementType);
+        writer.writeInt(compTypeId);
+        writer.writeInt(arr.length);
+
+        for (Object obj : arr)
+            writeValue(writer, obj);
+    }
+
+    /**
+     * @param writer Writer.
+     * @param elementType Element type.
+     * @param arr The array.
+     * @param clsName Component class name.
+     */
+    public void writeArray(PortableWriterExImpl writer, byte elementType, Object[] arr, String clsName) {
+        writer.writeByte(elementType);
+        writer.writeInt(GridPortableMarshaller.UNREGISTERED_TYPE_ID);
+        writer.writeString(clsName);
+        writer.writeInt(arr.length);
+
+        for (Object obj : arr)
+            writeValue(writer, obj);
+    }
+
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/6ac9557e/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/PortableEnumArrayLazyValue.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/PortableEnumArrayLazyValue.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/PortableEnumArrayLazyValue.java
new file mode 100644
index 0000000..d864a6e
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/PortableEnumArrayLazyValue.java
@@ -0,0 +1,114 @@
+/*
+ * 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.GridPortableMarshaller;
+import org.apache.ignite.internal.portable.PortableWriterExImpl;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.portable.PortableException;
+import org.apache.ignite.portable.PortableInvalidClassException;
+
+/**
+ *
+ */
+class PortableEnumArrayLazyValue extends PortableAbstractLazyValue {
+    /** */
+    private final int len;
+
+    /** */
+    private final int compTypeId;
+
+    /** */
+    private final String clsName;
+
+    /**
+     * @param reader Reader.
+     */
+    protected PortableEnumArrayLazyValue(PortableBuilderReader reader) {
+        super(reader, reader.position() - 1);
+
+        int typeId = reader.readInt();
+
+        if (typeId == GridPortableMarshaller.UNREGISTERED_TYPE_ID) {
+            clsName = reader.readString();
+
+            Class cls;
+
+            try {
+                // TODO: IGNITE-1272 - Is class loader needed here?
+                cls = U.forName(reader.readString(), null);
+            }
+            catch (ClassNotFoundException e) {
+                throw new PortableInvalidClassException("Failed to load the class: " + clsName, e);
+            }
+
+            compTypeId = reader.portableContext().descriptorForClass(cls).typeId();
+        }
+        else {
+            compTypeId = typeId;
+            clsName = null;
+        }
+
+        int size = reader.readInt();
+
+        for (int i = 0; i < size; i++)
+            reader.skipValue();
+
+        len = reader.position() - valOff;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected Object init() {
+        reader.position(valOff + 1);
+
+        //skipping component type id
+        reader.readInt();
+
+        int size = reader.readInt();
+
+        PortableBuilderEnum[] res = new PortableBuilderEnum[size];
+
+        for (int i = 0; i < size; i++) {
+            byte flag = reader.readByte();
+
+            if (flag == GridPortableMarshaller.NULL)
+                continue;
+
+            if (flag != GridPortableMarshaller.ENUM)
+                throw new PortableException("Invalid flag value: " + flag);
+
+            res[i] = new PortableBuilderEnum(reader);
+        }
+
+        return res;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeTo(PortableWriterExImpl writer, PortableBuilderSerializer ctx) {
+        if (val != null) {
+            if (clsName != null)
+                ctx.writeArray(writer, GridPortableMarshaller.ENUM_ARR, (Object[])val, clsName);
+            else
+                ctx.writeArray(writer, GridPortableMarshaller.ENUM_ARR, (Object[])val, compTypeId);
+
+            return;
+        }
+
+        writer.write(reader.array(), valOff, len);
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/6ac9557e/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/PortableLazyArrayList.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/PortableLazyArrayList.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/PortableLazyArrayList.java
new file mode 100644
index 0000000..a08cfdd
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/PortableLazyArrayList.java
@@ -0,0 +1,166 @@
+/*
+ * 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.AbstractList;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+
+/**
+ *
+ */
+class PortableLazyArrayList extends AbstractList<Object> implements PortableBuilderSerializationAware {
+    /** */
+    private final PortableBuilderReader reader;
+
+    /** */
+    private final int off;
+
+    /** */
+    private List<Object> delegate;
+
+    /**
+     * @param reader Reader.
+     * @param size Size,
+     */
+    PortableLazyArrayList(PortableBuilderReader reader, int size) {
+        this.reader = reader;
+        off = reader.position() - 1/* flag */ - 4/* size */ - 1/* col type */;
+
+        assert size >= 0;
+
+        for (int i = 0; i < size; i++)
+            reader.skipValue();
+    }
+
+    /**
+     *
+     */
+    private void ensureDelegateInit() {
+        if (delegate == null) {
+            int size = reader.readIntAbsolute(off + 1);
+
+            reader.position(off + 1/* flag */ + 4/* size */ + 1/* col type */);
+
+            delegate = new ArrayList<>(size);
+
+            for (int i = 0; i < size; i++)
+                delegate.add(reader.parseValue());
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public Object get(int idx) {
+        ensureDelegateInit();
+
+        return PortableUtils.unwrapLazy(delegate.get(idx));
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean add(Object o) {
+        ensureDelegateInit();
+
+        return delegate.add(o);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void add(int idx, Object element) {
+        ensureDelegateInit();
+
+        delegate.add(idx, element);
+    }
+
+    /** {@inheritDoc} */
+    @Override public Object set(int idx, Object element) {
+        ensureDelegateInit();
+
+        return PortableUtils.unwrapLazy(delegate.set(idx, element));
+    }
+
+    /** {@inheritDoc} */
+    @Override public Object remove(int idx) {
+        ensureDelegateInit();
+
+        return PortableUtils.unwrapLazy(delegate.remove(idx));
+    }
+
+    /** {@inheritDoc} */
+    @Override public void clear() {
+        if (delegate == null)
+            delegate = new ArrayList<>();
+        else
+            delegate.clear();
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean addAll(int idx, Collection<?> c) {
+        return delegate.addAll(idx, c);
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void removeRange(int fromIdx, int toIdx) {
+        ensureDelegateInit();
+
+        delegate.subList(fromIdx, toIdx).clear();
+    }
+
+    /** {@inheritDoc} */
+    @Override public int size() {
+        if (delegate == null)
+            return reader.readIntAbsolute(off + 1);
+
+        return delegate.size();
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeTo(PortableWriterExImpl writer, PortableBuilderSerializer ctx) {
+        if (delegate == null) {
+            int size = reader.readIntAbsolute(off + 1);
+
+            int hdrSize = 1 /* flag */ + 4 /* size */ + 1 /* col type */;
+
+            writer.write(reader.array(), off, hdrSize);
+
+            reader.position(off + hdrSize);
+
+            for (int i = 0; i < size; i++) {
+                Object o = reader.parseValue();
+
+                ctx.writeValue(writer, o);
+            }
+        }
+        else {
+            writer.writeByte(GridPortableMarshaller.COL);
+            writer.writeInt(delegate.size());
+
+            byte colType = reader.array()[off + 1 /* flag */ + 4 /* size */];
+            writer.writeByte(colType);
+
+            int oldPos = reader.position();
+
+            for (Object o : delegate)
+                ctx.writeValue(writer, o);
+
+            // PortableBuilderImpl might have been written. It could override reader's position.
+            reader.position(oldPos);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/6ac9557e/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/PortableLazyLinkedList.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/PortableLazyLinkedList.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/PortableLazyLinkedList.java
new file mode 100644
index 0000000..f793d7a
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/PortableLazyLinkedList.java
@@ -0,0 +1,217 @@
+/*
+ * 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.AbstractList;
+import java.util.Collection;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.ListIterator;
+
+/**
+ *
+ */
+class PortableLazyLinkedList extends AbstractList<Object> implements PortableBuilderSerializationAware {
+    /** */
+    private final PortableBuilderReader reader;
+
+    /** */
+    private final int off;
+
+    /** */
+    private List<Object> delegate;
+
+    /**
+     * @param reader Reader.
+     * @param size Size,
+     */
+    PortableLazyLinkedList(PortableBuilderReader reader, int size) {
+        this.reader = reader;
+        off = reader.position() - 1/* flag */ - 4/* size */ - 1/* col type */;
+
+        assert size >= 0;
+
+        for (int i = 0; i < size; i++)
+            reader.skipValue();
+    }
+
+    /**
+     *
+     */
+    private void ensureDelegateInit() {
+        if (delegate == null) {
+            int size = reader.readIntAbsolute(off + 1);
+
+            reader.position(off + 1/* flag */ + 4/* size */ + 1/* col type */);
+
+            delegate = new LinkedList<>();
+
+            for (int i = 0; i < size; i++)
+                delegate.add(reader.parseValue());
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public Object get(int idx) {
+        ensureDelegateInit();
+
+        return PortableUtils.unwrapLazy(delegate.get(idx));
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean add(Object o) {
+        ensureDelegateInit();
+
+        return delegate.add(o);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void add(int idx, Object element) {
+        ensureDelegateInit();
+
+        delegate.add(idx, element);
+    }
+
+    /** {@inheritDoc} */
+    @Override public Object set(int idx, Object element) {
+        ensureDelegateInit();
+
+        return PortableUtils.unwrapLazy(delegate.set(idx, element));
+    }
+
+    /** {@inheritDoc} */
+    @Override public Object remove(int idx) {
+        ensureDelegateInit();
+
+        return PortableUtils.unwrapLazy(delegate.remove(idx));
+    }
+
+    /** {@inheritDoc} */
+    @Override public void clear() {
+        if (delegate == null)
+            delegate = new LinkedList<>();
+        else
+            delegate.clear();
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean addAll(int idx, Collection<?> c) {
+        ensureDelegateInit();
+
+        return delegate.addAll(idx, c);
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void removeRange(int fromIdx, int toIdx) {
+        ensureDelegateInit();
+
+        delegate.subList(fromIdx, toIdx).clear();
+    }
+
+    /** {@inheritDoc} */
+    @Override public int size() {
+        if (delegate == null)
+            return reader.readIntAbsolute(off + 1);
+
+        return delegate.size();
+    }
+
+    /** {@inheritDoc} */
+    @Override public ListIterator<Object> listIterator(final int idx) {
+        ensureDelegateInit();
+
+        return new ListIterator<Object>() {
+            /** */
+            private final ListIterator<Object> delegate = PortableLazyLinkedList.super.listIterator(idx);
+
+            @Override public boolean hasNext() {
+                return delegate.hasNext();
+            }
+
+            @Override public Object next() {
+                return PortableUtils.unwrapLazy(delegate.next());
+            }
+
+            @Override public boolean hasPrevious() {
+                return delegate.hasPrevious();
+            }
+
+            @Override public Object previous() {
+                return PortableUtils.unwrapLazy(delegate.previous());
+            }
+
+            @Override public int nextIndex() {
+                return delegate.nextIndex();
+            }
+
+            @Override public int previousIndex() {
+                return delegate.previousIndex();
+            }
+
+            @Override public void remove() {
+                delegate.remove();
+            }
+
+            @Override public void set(Object o) {
+                delegate.set(o);
+            }
+
+            @Override public void add(Object o) {
+                delegate.add(o);
+            }
+        };
+    }
+
+    /** {@inheritDoc} */
+    @Override public Iterator<Object> iterator() {
+        ensureDelegateInit();
+
+        return PortableUtils.unwrapLazyIterator(super.iterator());
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeTo(PortableWriterExImpl writer, PortableBuilderSerializer ctx) {
+        if (delegate == null) {
+            int size = reader.readIntAbsolute(off + 1);
+
+            int hdrSize = 1 /* flag */ + 4 /* size */ + 1 /* col type */;
+            writer.write(reader.array(), off, hdrSize);
+
+            reader.position(off + hdrSize);
+
+            for (int i = 0; i < size; i++) {
+                Object o = reader.parseValue();
+
+                ctx.writeValue(writer, o);
+            }
+        }
+        else {
+            writer.writeByte(GridPortableMarshaller.COL);
+            writer.writeInt(delegate.size());
+
+            byte colType = reader.array()[off + 1 /* flag */ + 4 /* size */];
+            writer.writeByte(colType);
+
+            for (Object o : delegate)
+                ctx.writeValue(writer, o);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/6ac9557e/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/PortableLazyMap.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/PortableLazyMap.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/PortableLazyMap.java
new file mode 100644
index 0000000..12cbfd6
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/PortableLazyMap.java
@@ -0,0 +1,220 @@
+/*
+ * 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.AbstractMap;
+import java.util.AbstractSet;
+import java.util.Iterator;
+import java.util.LinkedHashMap;
+import java.util.Map;
+import java.util.Set;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ *
+ */
+class PortableLazyMap extends AbstractMap<Object, Object> implements PortableBuilderSerializationAware {
+    /** */
+    private final PortableBuilderReader reader;
+
+    /** */
+    private final int off;
+
+    /** */
+    private Map<Object, Object> delegate;
+
+    /**
+     * @param reader Reader.
+     * @param off Offset.
+     */
+    private PortableLazyMap(PortableBuilderReader reader, int off) {
+        this.reader = reader;
+        this.off = off;
+    }
+
+    /**
+     * @param reader Reader.
+     * @return PortableLazyMap.
+     */
+    @Nullable public static PortableLazyMap parseMap(PortableBuilderReader reader) {
+        int off = reader.position() - 1;
+
+        int size = reader.readInt();
+
+        reader.skip(1); // map type.
+
+        for (int i = 0; i < size; i++) {
+            reader.skipValue(); // skip key
+            reader.skipValue(); // skip value
+        }
+
+        return new PortableLazyMap(reader, off);
+    }
+
+    /**
+     *
+     */
+    private void ensureDelegateInit() {
+        if (delegate == null) {
+            int size = reader.readIntAbsolute(off + 1);
+
+            reader.position(off + 1/* flag */ + 4/* size */ + 1/* col type */);
+
+            delegate = new LinkedHashMap<>();
+
+            for (int i = 0; i < size; i++)
+                delegate.put(PortableUtils.unwrapLazy(reader.parseValue()), reader.parseValue());
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeTo(PortableWriterExImpl writer, PortableBuilderSerializer ctx) {
+        if (delegate == null) {
+            int size = reader.readIntAbsolute(off + 1);
+
+            int hdrSize = 1 /* flag */ + 4 /* size */ + 1 /* col type */;
+            writer.write(reader.array(), off, hdrSize);
+
+            reader.position(off + hdrSize);
+
+            for (int i = 0; i < size; i++) {
+                ctx.writeValue(writer, reader.parseValue()); // key
+                ctx.writeValue(writer, reader.parseValue()); // value
+            }
+        }
+        else {
+            writer.writeByte(GridPortableMarshaller.MAP);
+            writer.writeInt(delegate.size());
+
+            byte colType = reader.array()[off + 1 /* flag */ + 4 /* size */];
+
+            writer.writeByte(colType);
+
+            for (Entry<Object, Object> entry : delegate.entrySet()) {
+                ctx.writeValue(writer, entry.getKey());
+                ctx.writeValue(writer, entry.getValue());
+            }
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public int size() {
+        if (delegate == null)
+            return reader.readIntAbsolute(off + 1);
+
+        return delegate.size();
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean containsKey(Object key) {
+        ensureDelegateInit();
+
+        return delegate.containsKey(key);
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean containsValue(Object val) {
+        return values().contains(val);
+    }
+
+    /** {@inheritDoc} */
+    @Override public Set<Object> keySet() {
+        ensureDelegateInit();
+
+        return delegate.keySet();
+    }
+
+    /** {@inheritDoc} */
+    @Override public void clear() {
+        if (delegate == null)
+            delegate = new LinkedHashMap<>();
+        else
+            delegate.clear();
+    }
+
+    /** {@inheritDoc} */
+    @Override public Object get(Object key) {
+        ensureDelegateInit();
+
+        return PortableUtils.unwrapLazy(delegate.get(key));
+    }
+
+    /** {@inheritDoc} */
+    @Override public Object put(Object key, Object val) {
+        ensureDelegateInit();
+
+        return PortableUtils.unwrapLazy(delegate.put(key, val));
+    }
+
+    /** {@inheritDoc} */
+    @Override public Object remove(Object key) {
+        ensureDelegateInit();
+
+        return PortableUtils.unwrapLazy(delegate.remove(key));
+    }
+
+    /** {@inheritDoc} */
+    @Override public Set<Entry<Object, Object>> entrySet() {
+        ensureDelegateInit();
+
+        return new AbstractSet<Entry<Object, Object>>() {
+            @Override public boolean contains(Object o) {
+                throw new UnsupportedOperationException();
+            }
+
+            @Override public Iterator<Entry<Object, Object>> iterator() {
+                return new Iterator<Entry<Object, Object>>() {
+                    /** */
+                    private final Iterator<Entry<Object, Object>> itr = delegate.entrySet().iterator();
+
+                    @Override public boolean hasNext() {
+                        return itr.hasNext();
+                    }
+
+                    @Override public Entry<Object, Object> next() {
+                        Entry<Object, Object> res = itr.next();
+
+                        final Object val = res.getValue();
+
+                        if (val instanceof PortableLazyValue) {
+                            return new SimpleEntry<Object, Object>(res.getKey(), val) {
+                                private static final long serialVersionUID = 0L;
+
+                                @Override public Object getValue() {
+                                    return ((PortableLazyValue)val).value();
+                                }
+                            };
+                        }
+
+                        return res;
+                    }
+
+                    @Override public void remove() {
+                        itr.remove();
+                    }
+                };
+            }
+
+            @Override public int size() {
+                return delegate.size();
+            }
+        };
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/6ac9557e/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
new file mode 100644
index 0000000..bd027f5
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/PortableLazyMapEntry.java
@@ -0,0 +1,68 @@
+/*
+ * 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(PortableWriterExImpl 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/6ac9557e/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/PortableLazySet.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/PortableLazySet.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/PortableLazySet.java
new file mode 100644
index 0000000..16772af
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/PortableLazySet.java
@@ -0,0 +1,92 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.portable.builder;
+
+import java.util.Collection;
+import java.util.Set;
+import org.apache.ignite.internal.portable.GridPortableMarshaller;
+import org.apache.ignite.internal.portable.PortableUtils;
+import org.apache.ignite.internal.portable.PortableWriterExImpl;
+import org.apache.ignite.internal.util.typedef.internal.U;
+
+/**
+ *
+ */
+class PortableLazySet extends PortableAbstractLazyValue {
+    /** */
+    private final int off;
+
+    /**
+     * @param reader Reader.
+     * @param size Size.
+     */
+    PortableLazySet(PortableBuilderReader reader, int size) {
+        super(reader, reader.position() - 1);
+
+        off = reader.position() - 1/* flag */ - 4/* size */ - 1/* col type */;
+
+        assert size >= 0;
+
+        for (int i = 0; i < size; i++)
+            reader.skipValue();
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeTo(PortableWriterExImpl writer, PortableBuilderSerializer ctx) {
+        if (val == null) {
+            int size = reader.readIntAbsolute(off + 1);
+
+            int hdrSize = 1 /* flag */ + 4 /* size */ + 1 /* col type */;
+            writer.write(reader.array(), off, hdrSize);
+
+            reader.position(off + hdrSize);
+
+            for (int i = 0; i < size; i++) {
+                Object o = reader.parseValue();
+
+                ctx.writeValue(writer, o);
+            }
+        }
+        else {
+            Collection<Object> c = (Collection<Object>)val;
+
+            writer.writeByte(GridPortableMarshaller.COL);
+            writer.writeInt(c.size());
+
+            byte colType = reader.array()[off + 1 /* flag */ + 4 /* size */];
+            writer.writeByte(colType);
+
+            for (Object o : c)
+                ctx.writeValue(writer, o);
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override protected Object init() {
+        int size = reader.readIntAbsolute(off + 1);
+
+        reader.position(off + 1/* flag */ + 4/* size */ + 1/* col type */);
+
+        Set<Object> res = U.newLinkedHashSet(size);
+
+        for (int i = 0; i < size; i++)
+            res.add(PortableUtils.unwrapLazy(reader.parseValue()));
+
+        return res;
+    }
+}
\ No newline at end of file

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

http://git-wip-us.apache.org/repos/asf/ignite/blob/6ac9557e/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/PortableModifiableLazyValue.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/PortableModifiableLazyValue.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/PortableModifiableLazyValue.java
new file mode 100644
index 0000000..09fb844
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/PortableModifiableLazyValue.java
@@ -0,0 +1,52 @@
+/*
+ * 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.*;
+
+/**
+ *
+ */
+public class PortableModifiableLazyValue extends PortableAbstractLazyValue {
+    /** */
+    protected final int len;
+
+    /**
+     * @param reader
+     * @param valOff
+     * @param len
+     */
+    public PortableModifiableLazyValue(PortableBuilderReader reader, int valOff, int len) {
+        super(reader, valOff);
+
+        this.len = len;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected Object init() {
+        return reader.reader().unmarshal(valOff);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeTo(PortableWriterExImpl writer, PortableBuilderSerializer ctx) {
+        if (val == null)
+            writer.write(reader.array(), valOff, len);
+        else
+            writer.writeObject(val);
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/6ac9557e/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/PortableObjectArrayLazyValue.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/PortableObjectArrayLazyValue.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/PortableObjectArrayLazyValue.java
new file mode 100644
index 0000000..1126a3c
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/PortableObjectArrayLazyValue.java
@@ -0,0 +1,91 @@
+/*
+ * 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.GridPortableMarshaller;
+import org.apache.ignite.internal.portable.PortableWriterExImpl;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.portable.PortableInvalidClassException;
+
+/**
+ *
+ */
+class PortableObjectArrayLazyValue extends PortableAbstractLazyValue {
+    /** */
+    private Object[] lazyValsArr;
+
+    /** */
+    private int compTypeId;
+
+    /** */
+    private String clsName;
+
+    /**
+     * @param reader Reader.
+     */
+    protected PortableObjectArrayLazyValue(PortableBuilderReader reader) {
+        super(reader, reader.position() - 1);
+
+        int typeId = reader.readInt();
+
+        if (typeId == GridPortableMarshaller.UNREGISTERED_TYPE_ID) {
+            clsName = reader.readString();
+
+            Class cls;
+
+            try {
+                // TODO: IGNITE-1272 - Is class loader needed here?
+                cls = U.forName(reader.readString(), null);
+            }
+            catch (ClassNotFoundException e) {
+                throw new PortableInvalidClassException("Failed to load the class: " + clsName, e);
+            }
+
+            compTypeId = reader.portableContext().descriptorForClass(cls).typeId();
+        }
+        else {
+            compTypeId = typeId;
+            clsName = null;
+        }
+
+        int size = reader.readInt();
+
+        lazyValsArr = new Object[size];
+
+        for (int i = 0; i < size; i++)
+            lazyValsArr[i] = reader.parseValue();
+    }
+
+    /** {@inheritDoc} */
+    @Override protected Object init() {
+        for (int i = 0; i < lazyValsArr.length; i++) {
+            if (lazyValsArr[i] instanceof PortableLazyValue)
+                lazyValsArr[i] = ((PortableLazyValue)lazyValsArr[i]).value();
+        }
+
+        return lazyValsArr;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeTo(PortableWriterExImpl writer, PortableBuilderSerializer ctx) {
+        if (clsName == null)
+            ctx.writeArray(writer, GridPortableMarshaller.OBJ_ARR, lazyValsArr, compTypeId);
+        else
+            ctx.writeArray(writer, GridPortableMarshaller.OBJ_ARR, lazyValsArr, clsName);
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/6ac9557e/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/PortablePlainLazyValue.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/PortablePlainLazyValue.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/PortablePlainLazyValue.java
new file mode 100644
index 0000000..136958a
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/PortablePlainLazyValue.java
@@ -0,0 +1,49 @@
+/*
+ * 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.*;
+
+/**
+ *
+ */
+class PortablePlainLazyValue extends PortableAbstractLazyValue {
+    /** */
+    protected final int len;
+
+    /**
+     * @param reader Reader
+     * @param valOff Offset
+     * @param len Length.
+     */
+    protected PortablePlainLazyValue(PortableBuilderReader reader, int valOff, int len) {
+        super(reader, valOff);
+
+        this.len = len;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected Object init() {
+        return reader.reader().unmarshal(valOff);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeTo(PortableWriterExImpl writer, PortableBuilderSerializer ctx) {
+        writer.write(reader.array(), valOff, len);
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/6ac9557e/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/PortablePlainPortableObject.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/PortablePlainPortableObject.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/PortablePlainPortableObject.java
new file mode 100644
index 0000000..8743fbe
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/PortablePlainPortableObject.java
@@ -0,0 +1,53 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.portable.builder;
+
+import org.apache.ignite.internal.portable.PortableObjectImpl;
+import org.apache.ignite.internal.portable.PortableObjectOffheapImpl;
+import org.apache.ignite.internal.portable.PortableWriterExImpl;
+import org.apache.ignite.portable.PortableObject;
+
+/**
+ *
+ */
+public class PortablePlainPortableObject implements PortableLazyValue {
+    /** */
+    private final PortableObject portableObj;
+
+    /**
+     * @param portableObj Portable object.
+     */
+    public PortablePlainPortableObject(PortableObject portableObj) {
+        this.portableObj = portableObj;
+    }
+
+    /** {@inheritDoc} */
+    @Override public Object value() {
+        return portableObj;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeTo(PortableWriterExImpl writer, PortableBuilderSerializer ctx) {
+        PortableObject val = portableObj;
+
+        if (val instanceof PortableObjectOffheapImpl)
+            val = ((PortableObjectOffheapImpl)val).heapCopy();
+
+        writer.doWritePortableObject((PortableObjectImpl)val);
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/6ac9557e/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/PortableValueWithType.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/PortableValueWithType.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/PortableValueWithType.java
new file mode 100644
index 0000000..2e031f0
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/PortableValueWithType.java
@@ -0,0 +1,75 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.portable.builder;
+
+import org.apache.ignite.internal.portable.PortableWriterExImpl;
+import org.apache.ignite.internal.processors.cache.portable.CacheObjectPortableProcessorImpl;
+import org.apache.ignite.internal.util.typedef.internal.S;
+
+/**
+ *
+ */
+class PortableValueWithType implements PortableLazyValue {
+    /** */
+    private byte type;
+
+    /** */
+    private Object val;
+
+    /**
+     * @param type Type
+     * @param val Value.
+     */
+    PortableValueWithType(byte type, Object val) {
+        this.type = type;
+        this.val = val;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeTo(PortableWriterExImpl writer, PortableBuilderSerializer ctx) {
+        if (val instanceof PortableBuilderSerializationAware)
+            ((PortableBuilderSerializationAware)val).writeTo(writer, ctx);
+        else
+            ctx.writeValue(writer, val);
+    }
+
+    /** {@inheritDoc} */
+    public String typeName() {
+        return CacheObjectPortableProcessorImpl.fieldTypeName(type);
+    }
+
+    /** {@inheritDoc} */
+    @Override public Object value() {
+        if (val instanceof PortableLazyValue)
+            return ((PortableLazyValue)val).value();
+
+        return val;
+    }
+
+    /**
+     * @param val New value.
+     */
+    public void value(Object val) {
+        this.val = val;
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return S.toString(PortableValueWithType.class, this);
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/6ac9557e/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/package-info.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/package-info.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/package-info.java
new file mode 100644
index 0000000..e069f3e
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/package-info.java
@@ -0,0 +1,22 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+/**
+ * <!-- Package description. -->
+ * Contains classes specific to portable builder API.
+ */
+package org.apache.ignite.internal.portable.builder;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/6ac9557e/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/portable/CacheObjectPortableProcessorImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/portable/CacheObjectPortableProcessorImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/portable/CacheObjectPortableProcessorImpl.java
index b33c643..1be5aea 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/portable/CacheObjectPortableProcessorImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/portable/CacheObjectPortableProcessorImpl.java
@@ -45,13 +45,13 @@ import org.apache.ignite.cluster.ClusterNode;
 import org.apache.ignite.configuration.CacheConfiguration;
 import org.apache.ignite.internal.GridKernalContext;
 import org.apache.ignite.internal.portable.GridPortableMarshaller;
-import org.apache.ignite.internal.portable.PortableBuilderImpl;
 import org.apache.ignite.internal.portable.PortableContext;
 import org.apache.ignite.internal.portable.PortableMetaDataHandler;
 import org.apache.ignite.internal.portable.PortableMetaDataImpl;
 import org.apache.ignite.internal.portable.PortableObjectImpl;
 import org.apache.ignite.internal.portable.PortableObjectOffheapImpl;
 import org.apache.ignite.internal.portable.PortableUtils;
+import org.apache.ignite.internal.portable.builder.PortableBuilderImpl;
 import org.apache.ignite.internal.portable.streams.PortableInputStream;
 import org.apache.ignite.internal.portable.streams.PortableOffheapInputStream;
 import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;

http://git-wip-us.apache.org/repos/asf/ignite/blob/6ac9557e/modules/core/src/main/resources/META-INF/classnames.properties
----------------------------------------------------------------------
diff --git a/modules/core/src/main/resources/META-INF/classnames.properties b/modules/core/src/main/resources/META-INF/classnames.properties
index d80cc49..dd8c3f3 100644
--- a/modules/core/src/main/resources/META-INF/classnames.properties
+++ b/modules/core/src/main/resources/META-INF/classnames.properties
@@ -281,7 +281,7 @@ org.apache.ignite.internal.managers.loadbalancer.GridLoadBalancerManager$1
 org.apache.ignite.internal.processors.platform.PlatformAwareEventFilter
 org.apache.ignite.internal.portable.PortableClassDescriptor$Mode
 org.apache.ignite.internal.portable.PortableContext
-org.apache.ignite.internal.portable.PortableLazyMap$1$1$1
+org.apache.ignite.internal.portable.builder.PortableLazyMap$1$1$1
 org.apache.ignite.internal.portable.PortableMetaDataImpl
 org.apache.ignite.internal.portable.PortableObjectEx
 org.apache.ignite.internal.portable.PortableObjectImpl


[32/39] ignite git commit: Disabled hanging tests.

Posted by ag...@apache.org.
Disabled hanging tests.


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

Branch: refs/heads/ignite-264
Commit: 66a49d75e3cdd001135f32a81dbfb931b9e1584b
Parents: 27cd615
Author: sboikov <sb...@gridgain.com>
Authored: Thu Sep 3 15:32:26 2015 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Thu Sep 3 15:32:26 2015 +0300

----------------------------------------------------------------------
 .../processors/cache/IgniteCacheP2pUnmarshallingErrorTest.java  | 5 +++++
 1 file changed, 5 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/66a49d75/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheP2pUnmarshallingErrorTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheP2pUnmarshallingErrorTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheP2pUnmarshallingErrorTest.java
index b62cc48..ecb2698 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheP2pUnmarshallingErrorTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheP2pUnmarshallingErrorTest.java
@@ -42,6 +42,11 @@ public class IgniteCacheP2pUnmarshallingErrorTest extends IgniteCacheAbstractTes
     protected static int key = 0;
 
     /** {@inheritDoc} */
+    @Override protected void beforeTestsStarted() throws Exception {
+        fail("https://issues.apache.org/jira/browse/IGNITE-1366");
+    }
+
+    /** {@inheritDoc} */
     @Override protected int gridCount() {
         return 3;
     }


[02/39] ignite git commit: 1.3.3-p7-SNAPSHOT

Posted by ag...@apache.org.
1.3.3-p7-SNAPSHOT


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

Branch: refs/heads/ignite-264
Commit: c1b537fa5b8d29048745b8aa59c52a583e9e4931
Parents: a61cf24
Author: Ignite Teamcity <ig...@apache.org>
Authored: Mon Aug 31 23:08:47 2015 +0300
Committer: Ignite Teamcity <ig...@apache.org>
Committed: Mon Aug 31 23:08:47 2015 +0300

----------------------------------------------------------------------
 examples/pom.xml                                   | 2 +-
 modules/aop/pom.xml                                | 2 +-
 modules/apache-license-gen/pom.xml                 | 2 +-
 modules/aws/pom.xml                                | 2 +-
 modules/clients/pom.xml                            | 2 +-
 modules/cloud/pom.xml                              | 2 +-
 modules/codegen/pom.xml                            | 2 +-
 modules/core/pom.xml                               | 2 +-
 modules/core/src/main/resources/ignite.properties  | 2 +-
 modules/extdata/p2p/pom.xml                        | 2 +-
 modules/extdata/uri/modules/uri-dependency/pom.xml | 2 +-
 modules/extdata/uri/pom.xml                        | 2 +-
 modules/gce/pom.xml                                | 2 +-
 modules/geospatial/pom.xml                         | 2 +-
 modules/hadoop/pom.xml                             | 2 +-
 modules/hibernate/pom.xml                          | 2 +-
 modules/indexing/pom.xml                           | 2 +-
 modules/jcl/pom.xml                                | 2 +-
 modules/jta/pom.xml                                | 2 +-
 modules/kafka/pom.xml                              | 2 +-
 modules/log4j/pom.xml                              | 2 +-
 modules/mesos/pom.xml                              | 2 +-
 modules/rest-http/pom.xml                          | 2 +-
 modules/scalar-2.10/pom.xml                        | 2 +-
 modules/scalar/pom.xml                             | 2 +-
 modules/schedule/pom.xml                           | 2 +-
 modules/schema-import/pom.xml                      | 2 +-
 modules/slf4j/pom.xml                              | 2 +-
 modules/spark-2.10/pom.xml                         | 2 +-
 modules/spark/pom.xml                              | 2 +-
 modules/spring/pom.xml                             | 2 +-
 modules/ssh/pom.xml                                | 2 +-
 modules/tools/pom.xml                              | 2 +-
 modules/urideploy/pom.xml                          | 2 +-
 modules/visor-console-2.10/pom.xml                 | 2 +-
 modules/visor-console/pom.xml                      | 2 +-
 modules/visor-plugins/pom.xml                      | 2 +-
 modules/web/pom.xml                                | 2 +-
 modules/yardstick/pom.xml                          | 2 +-
 modules/yarn/pom.xml                               | 2 +-
 pom.xml                                            | 2 +-
 41 files changed, 41 insertions(+), 41 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/c1b537fa/examples/pom.xml
----------------------------------------------------------------------
diff --git a/examples/pom.xml b/examples/pom.xml
index c4445fa..6abf904 100644
--- a/examples/pom.xml
+++ b/examples/pom.xml
@@ -28,7 +28,7 @@
     </parent>
 
     <artifactId>ignite-examples</artifactId>
-    <version>1.3.3-p6-SNAPSHOT</version>
+    <version>1.3.3-p7-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/ignite/blob/c1b537fa/modules/aop/pom.xml
----------------------------------------------------------------------
diff --git a/modules/aop/pom.xml b/modules/aop/pom.xml
index 13915f1..d10e9d4 100644
--- a/modules/aop/pom.xml
+++ b/modules/aop/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-aop</artifactId>
-    <version>1.3.3-p6-SNAPSHOT</version>
+    <version>1.3.3-p7-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/ignite/blob/c1b537fa/modules/apache-license-gen/pom.xml
----------------------------------------------------------------------
diff --git a/modules/apache-license-gen/pom.xml b/modules/apache-license-gen/pom.xml
index 144cb45..bbfe24d 100644
--- a/modules/apache-license-gen/pom.xml
+++ b/modules/apache-license-gen/pom.xml
@@ -31,5 +31,5 @@
 
     <groupId>org.apache.ignite</groupId>
     <artifactId>ignite-apache-license-gen</artifactId>
-    <version>1.3.3-p6-SNAPSHOT</version>
+    <version>1.3.3-p7-SNAPSHOT</version>
 </project>

http://git-wip-us.apache.org/repos/asf/ignite/blob/c1b537fa/modules/aws/pom.xml
----------------------------------------------------------------------
diff --git a/modules/aws/pom.xml b/modules/aws/pom.xml
index 60aaa17..e52d6dc 100644
--- a/modules/aws/pom.xml
+++ b/modules/aws/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-aws</artifactId>
-    <version>1.3.3-p6-SNAPSHOT</version>
+    <version>1.3.3-p7-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/ignite/blob/c1b537fa/modules/clients/pom.xml
----------------------------------------------------------------------
diff --git a/modules/clients/pom.xml b/modules/clients/pom.xml
index 8f44d23..58d32ef 100644
--- a/modules/clients/pom.xml
+++ b/modules/clients/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-clients</artifactId>
-    <version>1.3.3-p6-SNAPSHOT</version>
+    <version>1.3.3-p7-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/ignite/blob/c1b537fa/modules/cloud/pom.xml
----------------------------------------------------------------------
diff --git a/modules/cloud/pom.xml b/modules/cloud/pom.xml
index 6204d35..ac1858f 100644
--- a/modules/cloud/pom.xml
+++ b/modules/cloud/pom.xml
@@ -29,7 +29,7 @@
     </parent>
 
     <artifactId>ignite-cloud</artifactId>
-    <version>1.3.3-p6-SNAPSHOT</version>
+    <version>1.3.3-p7-SNAPSHOT</version>
 
     <properties>
         <jcloud.version>1.9.0</jcloud.version>

http://git-wip-us.apache.org/repos/asf/ignite/blob/c1b537fa/modules/codegen/pom.xml
----------------------------------------------------------------------
diff --git a/modules/codegen/pom.xml b/modules/codegen/pom.xml
index 7ada8fc..85897d5 100644
--- a/modules/codegen/pom.xml
+++ b/modules/codegen/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-codegen</artifactId>
-    <version>1.3.3-p6-SNAPSHOT</version>
+    <version>1.3.3-p7-SNAPSHOT</version>
 
     <properties>
         <project.build.sourceEncoding>UTF-8</project.build.sourceEncoding>

http://git-wip-us.apache.org/repos/asf/ignite/blob/c1b537fa/modules/core/pom.xml
----------------------------------------------------------------------
diff --git a/modules/core/pom.xml b/modules/core/pom.xml
index 1539550..d093d92 100644
--- a/modules/core/pom.xml
+++ b/modules/core/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-core</artifactId>
-    <version>1.3.3-p6-SNAPSHOT</version>
+    <version>1.3.3-p7-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/ignite/blob/c1b537fa/modules/core/src/main/resources/ignite.properties
----------------------------------------------------------------------
diff --git a/modules/core/src/main/resources/ignite.properties b/modules/core/src/main/resources/ignite.properties
index 8e38b9e..a8e23fb 100644
--- a/modules/core/src/main/resources/ignite.properties
+++ b/modules/core/src/main/resources/ignite.properties
@@ -15,7 +15,7 @@
 # limitations under the License.
 #
 
-ignite.version=1.3.3-p6-SNAPSHOT
+ignite.version=1.3.3-p7-SNAPSHOT
 ignite.build=0
 ignite.revision=DEV
 ignite.rel.date=01011970

http://git-wip-us.apache.org/repos/asf/ignite/blob/c1b537fa/modules/extdata/p2p/pom.xml
----------------------------------------------------------------------
diff --git a/modules/extdata/p2p/pom.xml b/modules/extdata/p2p/pom.xml
index 97a3efd..76b2a1b 100644
--- a/modules/extdata/p2p/pom.xml
+++ b/modules/extdata/p2p/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-extdata-p2p</artifactId>
-    <version>1.3.3-p6-SNAPSHOT</version>
+    <version>1.3.3-p7-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/ignite/blob/c1b537fa/modules/extdata/uri/modules/uri-dependency/pom.xml
----------------------------------------------------------------------
diff --git a/modules/extdata/uri/modules/uri-dependency/pom.xml b/modules/extdata/uri/modules/uri-dependency/pom.xml
index aee39b4..c147131 100644
--- a/modules/extdata/uri/modules/uri-dependency/pom.xml
+++ b/modules/extdata/uri/modules/uri-dependency/pom.xml
@@ -27,7 +27,7 @@
     <artifactId>ignite-extdata-uri-dep</artifactId>
     <packaging>jar</packaging>
 
-    <version>1.3.3-p6-SNAPSHOT</version>
+    <version>1.3.3-p7-SNAPSHOT</version>
     <modelVersion>4.0.0</modelVersion>
 
     <dependencies>

http://git-wip-us.apache.org/repos/asf/ignite/blob/c1b537fa/modules/extdata/uri/pom.xml
----------------------------------------------------------------------
diff --git a/modules/extdata/uri/pom.xml b/modules/extdata/uri/pom.xml
index ad45998..cc337eb 100644
--- a/modules/extdata/uri/pom.xml
+++ b/modules/extdata/uri/pom.xml
@@ -32,7 +32,7 @@
     </parent>
 
     <artifactId>ignite-extdata-uri</artifactId>
-    <version>1.3.3-p6-SNAPSHOT</version>
+    <version>1.3.3-p7-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/ignite/blob/c1b537fa/modules/gce/pom.xml
----------------------------------------------------------------------
diff --git a/modules/gce/pom.xml b/modules/gce/pom.xml
index c399c02..ab3c14e 100644
--- a/modules/gce/pom.xml
+++ b/modules/gce/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-gce</artifactId>
-    <version>1.3.3-p6-SNAPSHOT</version>
+    <version>1.3.3-p7-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/ignite/blob/c1b537fa/modules/geospatial/pom.xml
----------------------------------------------------------------------
diff --git a/modules/geospatial/pom.xml b/modules/geospatial/pom.xml
index d5d2b17..dd9c6d5 100644
--- a/modules/geospatial/pom.xml
+++ b/modules/geospatial/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-geospatial</artifactId>
-    <version>1.3.3-p6-SNAPSHOT</version>
+    <version>1.3.3-p7-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/ignite/blob/c1b537fa/modules/hadoop/pom.xml
----------------------------------------------------------------------
diff --git a/modules/hadoop/pom.xml b/modules/hadoop/pom.xml
index 2dfae51..4097e9f 100644
--- a/modules/hadoop/pom.xml
+++ b/modules/hadoop/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-hadoop</artifactId>
-    <version>1.3.3-p6-SNAPSHOT</version>
+    <version>1.3.3-p7-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/ignite/blob/c1b537fa/modules/hibernate/pom.xml
----------------------------------------------------------------------
diff --git a/modules/hibernate/pom.xml b/modules/hibernate/pom.xml
index 7ed8bf7..121738b 100644
--- a/modules/hibernate/pom.xml
+++ b/modules/hibernate/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-hibernate</artifactId>
-    <version>1.3.3-p6-SNAPSHOT</version>
+    <version>1.3.3-p7-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/ignite/blob/c1b537fa/modules/indexing/pom.xml
----------------------------------------------------------------------
diff --git a/modules/indexing/pom.xml b/modules/indexing/pom.xml
index b05c963..12010d9 100644
--- a/modules/indexing/pom.xml
+++ b/modules/indexing/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-indexing</artifactId>
-    <version>1.3.3-p6-SNAPSHOT</version>
+    <version>1.3.3-p7-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/ignite/blob/c1b537fa/modules/jcl/pom.xml
----------------------------------------------------------------------
diff --git a/modules/jcl/pom.xml b/modules/jcl/pom.xml
index 5b48182..7d349d4 100644
--- a/modules/jcl/pom.xml
+++ b/modules/jcl/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-jcl</artifactId>
-    <version>1.3.3-p6-SNAPSHOT</version>
+    <version>1.3.3-p7-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/ignite/blob/c1b537fa/modules/jta/pom.xml
----------------------------------------------------------------------
diff --git a/modules/jta/pom.xml b/modules/jta/pom.xml
index 52f7717..d66d9b5 100644
--- a/modules/jta/pom.xml
+++ b/modules/jta/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-jta</artifactId>
-    <version>1.3.3-p6-SNAPSHOT</version>
+    <version>1.3.3-p7-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/ignite/blob/c1b537fa/modules/kafka/pom.xml
----------------------------------------------------------------------
diff --git a/modules/kafka/pom.xml b/modules/kafka/pom.xml
index 691e445..576d75d 100644
--- a/modules/kafka/pom.xml
+++ b/modules/kafka/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-kafka</artifactId>
-    <version>1.3.3-p6-SNAPSHOT</version>
+    <version>1.3.3-p7-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/ignite/blob/c1b537fa/modules/log4j/pom.xml
----------------------------------------------------------------------
diff --git a/modules/log4j/pom.xml b/modules/log4j/pom.xml
index 7b52dc0..fcf4143 100644
--- a/modules/log4j/pom.xml
+++ b/modules/log4j/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-log4j</artifactId>
-    <version>1.3.3-p6-SNAPSHOT</version>
+    <version>1.3.3-p7-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/ignite/blob/c1b537fa/modules/mesos/pom.xml
----------------------------------------------------------------------
diff --git a/modules/mesos/pom.xml b/modules/mesos/pom.xml
index cb40805..202af29 100644
--- a/modules/mesos/pom.xml
+++ b/modules/mesos/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-mesos</artifactId>
-    <version>1.3.3-p6-SNAPSHOT</version>
+    <version>1.3.3-p7-SNAPSHOT</version>
 
     <properties>
         <mesos.version>0.22.0</mesos.version>

http://git-wip-us.apache.org/repos/asf/ignite/blob/c1b537fa/modules/rest-http/pom.xml
----------------------------------------------------------------------
diff --git a/modules/rest-http/pom.xml b/modules/rest-http/pom.xml
index 2ae943a..4a0bd77 100644
--- a/modules/rest-http/pom.xml
+++ b/modules/rest-http/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-rest-http</artifactId>
-    <version>1.3.3-p6-SNAPSHOT</version>
+    <version>1.3.3-p7-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/ignite/blob/c1b537fa/modules/scalar-2.10/pom.xml
----------------------------------------------------------------------
diff --git a/modules/scalar-2.10/pom.xml b/modules/scalar-2.10/pom.xml
index 86b80e6..b6d6b6a 100644
--- a/modules/scalar-2.10/pom.xml
+++ b/modules/scalar-2.10/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-scalar_2.10</artifactId>
-    <version>1.3.3-p6-SNAPSHOT</version>
+    <version>1.3.3-p7-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/ignite/blob/c1b537fa/modules/scalar/pom.xml
----------------------------------------------------------------------
diff --git a/modules/scalar/pom.xml b/modules/scalar/pom.xml
index c853592..91e5736 100644
--- a/modules/scalar/pom.xml
+++ b/modules/scalar/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-scalar</artifactId>
-    <version>1.3.3-p6-SNAPSHOT</version>
+    <version>1.3.3-p7-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/ignite/blob/c1b537fa/modules/schedule/pom.xml
----------------------------------------------------------------------
diff --git a/modules/schedule/pom.xml b/modules/schedule/pom.xml
index b748944..a3b9657 100644
--- a/modules/schedule/pom.xml
+++ b/modules/schedule/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-schedule</artifactId>
-    <version>1.3.3-p6-SNAPSHOT</version>
+    <version>1.3.3-p7-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/ignite/blob/c1b537fa/modules/schema-import/pom.xml
----------------------------------------------------------------------
diff --git a/modules/schema-import/pom.xml b/modules/schema-import/pom.xml
index 4b5ae9f..7e26a63 100644
--- a/modules/schema-import/pom.xml
+++ b/modules/schema-import/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-schema-import</artifactId>
-    <version>1.3.3-p6-SNAPSHOT</version>
+    <version>1.3.3-p7-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/ignite/blob/c1b537fa/modules/slf4j/pom.xml
----------------------------------------------------------------------
diff --git a/modules/slf4j/pom.xml b/modules/slf4j/pom.xml
index e00a38a..0b45a4e 100644
--- a/modules/slf4j/pom.xml
+++ b/modules/slf4j/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-slf4j</artifactId>
-    <version>1.3.3-p6-SNAPSHOT</version>
+    <version>1.3.3-p7-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/ignite/blob/c1b537fa/modules/spark-2.10/pom.xml
----------------------------------------------------------------------
diff --git a/modules/spark-2.10/pom.xml b/modules/spark-2.10/pom.xml
index cae8086..6a743a6 100644
--- a/modules/spark-2.10/pom.xml
+++ b/modules/spark-2.10/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-spark_2.10</artifactId>
-    <version>1.3.3-p6-SNAPSHOT</version>
+    <version>1.3.3-p7-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/ignite/blob/c1b537fa/modules/spark/pom.xml
----------------------------------------------------------------------
diff --git a/modules/spark/pom.xml b/modules/spark/pom.xml
index f61b3f9..3222c6a 100644
--- a/modules/spark/pom.xml
+++ b/modules/spark/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-spark</artifactId>
-    <version>1.3.3-p6-SNAPSHOT</version>
+    <version>1.3.3-p7-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/ignite/blob/c1b537fa/modules/spring/pom.xml
----------------------------------------------------------------------
diff --git a/modules/spring/pom.xml b/modules/spring/pom.xml
index 4e38ddc..f3f4e76 100644
--- a/modules/spring/pom.xml
+++ b/modules/spring/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-spring</artifactId>
-    <version>1.3.3-p6-SNAPSHOT</version>
+    <version>1.3.3-p7-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/ignite/blob/c1b537fa/modules/ssh/pom.xml
----------------------------------------------------------------------
diff --git a/modules/ssh/pom.xml b/modules/ssh/pom.xml
index b25822f..17c7362 100644
--- a/modules/ssh/pom.xml
+++ b/modules/ssh/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-ssh</artifactId>
-    <version>1.3.3-p6-SNAPSHOT</version>
+    <version>1.3.3-p7-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/ignite/blob/c1b537fa/modules/tools/pom.xml
----------------------------------------------------------------------
diff --git a/modules/tools/pom.xml b/modules/tools/pom.xml
index 67940d6..7d446d1 100644
--- a/modules/tools/pom.xml
+++ b/modules/tools/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-tools</artifactId>
-    <version>1.3.3-p6-SNAPSHOT</version>
+    <version>1.3.3-p7-SNAPSHOT</version>
 
     <properties>
         <project.build.sourceEncoding>UTF-8</project.build.sourceEncoding>

http://git-wip-us.apache.org/repos/asf/ignite/blob/c1b537fa/modules/urideploy/pom.xml
----------------------------------------------------------------------
diff --git a/modules/urideploy/pom.xml b/modules/urideploy/pom.xml
index d08a5fb..832fe8f 100644
--- a/modules/urideploy/pom.xml
+++ b/modules/urideploy/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-urideploy</artifactId>
-    <version>1.3.3-p6-SNAPSHOT</version>
+    <version>1.3.3-p7-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/ignite/blob/c1b537fa/modules/visor-console-2.10/pom.xml
----------------------------------------------------------------------
diff --git a/modules/visor-console-2.10/pom.xml b/modules/visor-console-2.10/pom.xml
index c43e452..b652c11 100644
--- a/modules/visor-console-2.10/pom.xml
+++ b/modules/visor-console-2.10/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-visor-console_2.10</artifactId>
-    <version>1.3.3-p6-SNAPSHOT</version>
+    <version>1.3.3-p7-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/ignite/blob/c1b537fa/modules/visor-console/pom.xml
----------------------------------------------------------------------
diff --git a/modules/visor-console/pom.xml b/modules/visor-console/pom.xml
index db33d7b..e520f5e 100644
--- a/modules/visor-console/pom.xml
+++ b/modules/visor-console/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-visor-console</artifactId>
-    <version>1.3.3-p6-SNAPSHOT</version>
+    <version>1.3.3-p7-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/ignite/blob/c1b537fa/modules/visor-plugins/pom.xml
----------------------------------------------------------------------
diff --git a/modules/visor-plugins/pom.xml b/modules/visor-plugins/pom.xml
index a1a67ea..56ea80d 100644
--- a/modules/visor-plugins/pom.xml
+++ b/modules/visor-plugins/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-visor-plugins</artifactId>
-    <version>1.3.3-p6-SNAPSHOT</version>
+    <version>1.3.3-p7-SNAPSHOT</version>
 
     <dependencies>
         <!-- Ignite dependencies -->

http://git-wip-us.apache.org/repos/asf/ignite/blob/c1b537fa/modules/web/pom.xml
----------------------------------------------------------------------
diff --git a/modules/web/pom.xml b/modules/web/pom.xml
index 1f930af..6316425 100644
--- a/modules/web/pom.xml
+++ b/modules/web/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-web</artifactId>
-    <version>1.3.3-p6-SNAPSHOT</version>
+    <version>1.3.3-p7-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/ignite/blob/c1b537fa/modules/yardstick/pom.xml
----------------------------------------------------------------------
diff --git a/modules/yardstick/pom.xml b/modules/yardstick/pom.xml
index f48ce2e..43626a6 100644
--- a/modules/yardstick/pom.xml
+++ b/modules/yardstick/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-yardstick</artifactId>
-    <version>1.3.3-p6-SNAPSHOT</version>
+    <version>1.3.3-p7-SNAPSHOT</version>
 
     <properties>
         <yardstick.version>0.7.0</yardstick.version>

http://git-wip-us.apache.org/repos/asf/ignite/blob/c1b537fa/modules/yarn/pom.xml
----------------------------------------------------------------------
diff --git a/modules/yarn/pom.xml b/modules/yarn/pom.xml
index ebac7b2..d75c104 100644
--- a/modules/yarn/pom.xml
+++ b/modules/yarn/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-yarn</artifactId>
-    <version>1.3.3-p6-SNAPSHOT</version>
+    <version>1.3.3-p7-SNAPSHOT</version>
 
     <properties>
         <hadoop.version>2.7.0</hadoop.version>

http://git-wip-us.apache.org/repos/asf/ignite/blob/c1b537fa/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 642e344..40612b6 100644
--- a/pom.xml
+++ b/pom.xml
@@ -32,7 +32,7 @@
 
     <groupId>org.apache.ignite</groupId>
     <artifactId>apache-ignite</artifactId>
-    <version>1.3.3-p6-SNAPSHOT</version>
+    <version>1.3.3-p7-SNAPSHOT</version>
     <packaging>pom</packaging>
 
     <properties>


[33/39] ignite git commit: Merge remote-tracking branch 'origin/ignite-1.4' into ignite-1.4

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


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

Branch: refs/heads/ignite-264
Commit: c5d303bac7a9b17f88260b1bde175fbfbc053988
Parents: 66a49d7 7f4c9ac
Author: sboikov <sb...@gridgain.com>
Authored: Thu Sep 3 15:34:47 2015 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Thu Sep 3 15:34:47 2015 +0300

----------------------------------------------------------------------
 .../java8/datagrid/CacheAffinityExample.java    | 15 +++++++++++++
 .../java8/messaging/MessagingExample.java       | 11 +++++++++-
 .../java8/examples/BasicExamplesSelfTest.java   | 10 ++++++---
 .../java8/examples/CacheExamplesSelfTest.java   |  8 ++++---
 .../examples/CheckpointExamplesSelfTest.java    |  8 +++----
 .../examples/ClusterGroupExampleSelfTest.java   |  4 +++-
 .../examples/ContinuationExamplesSelfTest.java  |  8 ++++---
 .../ContinuousMapperExamplesSelfTest.java       |  8 ++++---
 .../examples/DeploymentExamplesSelfTest.java    |  6 ++++--
 .../java8/examples/EventsExamplesSelfTest.java  |  5 +++--
 .../HibernateL2CacheExampleSelfTest.java        |  8 ++++---
 .../java8/examples/IgfsExamplesSelfTest.java    |  6 ++++--
 .../examples/LifecycleExamplesSelfTest.java     |  8 ++++---
 .../examples/MemcacheRestExamplesSelfTest.java  |  4 +++-
 .../examples/MessagingExamplesSelfTest.java     |  6 ++++--
 .../examples/MonteCarloExamplesSelfTest.java    |  8 ++++---
 .../examples/SpringBeanExamplesSelfTest.java    |  8 ++++---
 .../java8/examples/TaskExamplesSelfTest.java    |  4 +++-
 .../IgniteExamplesJ8SelfTestSuite.java          | 12 +++++++++--
 .../ignite/platform/cpp/package-info.java       | 22 ++++++++++++++++++++
 .../ignite/platform/dotnet/package-info.java    | 22 ++++++++++++++++++++
 .../apache/ignite/platform/package-info.java    | 22 ++++++++++++++++++++
 parent/pom.xml                                  |  4 ++++
 23 files changed, 175 insertions(+), 42 deletions(-)
----------------------------------------------------------------------



[23/39] ignite git commit: ignite-1273: added ability to modify arrays returned from PortableBuilder and fixed cyclic references processing for arrays and collections in PortableMarshaller

Posted by ag...@apache.org.
ignite-1273: added ability to modify arrays returned from PortableBuilder and fixed cyclic references processing for arrays and collections in PortableMarshaller


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

Branch: refs/heads/ignite-264
Commit: 9057a4c0c7b2242dc9396432d700a87f21a52f1f
Parents: 6769956
Author: Denis Magda <dm...@gridgain.com>
Authored: Thu Sep 3 09:06:54 2015 +0300
Committer: Denis Magda <dm...@gridgain.com>
Committed: Thu Sep 3 09:06:54 2015 +0300

----------------------------------------------------------------------
 .../portable/GridPortableMarshaller.java        |  44 +-
 .../portable/PortableAbstractLazyValue.java     |  57 --
 .../internal/portable/PortableBuilderEnum.java  | 114 ---
 .../internal/portable/PortableBuilderImpl.java  | 531 ------------
 .../portable/PortableBuilderReader.java         | 776 ------------------
 .../PortableBuilderSerializationAware.java      |  29 -
 .../portable/PortableBuilderSerializer.java     | 211 -----
 .../portable/PortableClassDescriptor.java       |  57 +-
 .../internal/portable/PortableContext.java      |   6 +-
 .../portable/PortableEnumArrayLazyValue.java    | 112 ---
 .../portable/PortableLazyArrayList.java         | 159 ----
 .../portable/PortableLazyLinkedList.java        | 215 -----
 .../internal/portable/PortableLazyMap.java      | 218 -----
 .../internal/portable/PortableLazyMapEntry.java |  66 --
 .../internal/portable/PortableLazySet.java      |  89 ---
 .../internal/portable/PortableLazyValue.java    |  28 -
 .../portable/PortableObjectArrayLazyValue.java  |  89 ---
 .../portable/PortablePlainLazyValue.java        |  47 --
 .../portable/PortablePlainPortableObject.java   |  50 --
 .../internal/portable/PortableReaderExImpl.java | 154 +++-
 .../ignite/internal/portable/PortableUtils.java |  11 +
 .../portable/PortableValueWithType.java         |  74 --
 .../internal/portable/PortableWriterExImpl.java | 159 +++-
 .../builder/PortableAbstractLazyValue.java      |  57 ++
 .../portable/builder/PortableBuilderEnum.java   | 116 +++
 .../portable/builder/PortableBuilderImpl.java   | 537 +++++++++++++
 .../portable/builder/PortableBuilderReader.java | 800 +++++++++++++++++++
 .../PortableBuilderSerializationAware.java      |  31 +
 .../builder/PortableBuilderSerializer.java      | 214 +++++
 .../builder/PortableEnumArrayLazyValue.java     | 114 +++
 .../portable/builder/PortableLazyArrayList.java | 166 ++++
 .../builder/PortableLazyLinkedList.java         | 217 +++++
 .../portable/builder/PortableLazyMap.java       | 220 +++++
 .../portable/builder/PortableLazyMapEntry.java  |  68 ++
 .../portable/builder/PortableLazySet.java       |  92 +++
 .../portable/builder/PortableLazyValue.java     |  28 +
 .../builder/PortableModifiableLazyValue.java    |  52 ++
 .../builder/PortableObjectArrayLazyValue.java   |  91 +++
 .../builder/PortablePlainLazyValue.java         |  49 ++
 .../builder/PortablePlainPortableObject.java    |  53 ++
 .../portable/builder/PortableValueWithType.java |  75 ++
 .../internal/portable/builder/package-info.java |  22 +
 .../CacheObjectPortableProcessorImpl.java       |   2 +-
 .../resources/META-INF/classnames.properties    |   2 +-
 .../GridPortableBuilderAdditionalSelfTest.java  | 232 +++++-
 .../portable/GridPortableBuilderSelfTest.java   |   1 +
 .../GridPortableMarshallerSelfTest.java         |  72 +-
 .../GridPortableMetaDataDisabledSelfTest.java   |  17 +
 .../portable/GridPortableMetaDataSelfTest.java  |  17 +
 .../mutabletest/GridPortableTestClasses.java    |  38 +-
 ...ClientNodePortableMetadataMultinodeTest.java |  11 +
 51 files changed, 3687 insertions(+), 3003 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/9057a4c0/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 3b2357e..c7a9e6f 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
@@ -142,67 +142,67 @@ public class GridPortableMarshaller {
     public static final byte OBJ = (byte)103;
 
     /** */
-    static final byte USER_SET = -1;
+    public static final byte USER_SET = -1;
 
     /** */
-    static final byte USER_COL = 0;
+    public static final byte USER_COL = 0;
 
     /** */
-    static final byte ARR_LIST = 1;
+    public static final byte ARR_LIST = 1;
 
     /** */
-    static final byte LINKED_LIST = 2;
+    public static final byte LINKED_LIST = 2;
 
     /** */
-    static final byte HASH_SET = 3;
+    public static final byte HASH_SET = 3;
 
     /** */
-    static final byte LINKED_HASH_SET = 4;
+    public static final byte LINKED_HASH_SET = 4;
 
     /** */
-    static final byte TREE_SET = 5;
+    public static final byte TREE_SET = 5;
 
     /** */
-    static final byte CONC_SKIP_LIST_SET = 6;
+    public static final byte CONC_SKIP_LIST_SET = 6;
 
     /** */
-    static final byte HASH_MAP = 1;
+    public static final byte HASH_MAP = 1;
 
     /** */
-    static final byte LINKED_HASH_MAP = 2;
+    public static final byte LINKED_HASH_MAP = 2;
 
     /** */
-    static final byte TREE_MAP = 3;
+    public static final byte TREE_MAP = 3;
 
     /** */
-    static final byte CONC_HASH_MAP = 4;
+    public static final byte CONC_HASH_MAP = 4;
 
     /** */
-    static final byte PROPERTIES_MAP = 5;
+    public static final byte PROPERTIES_MAP = 5;
 
     /** */
-    static final int OBJECT_TYPE_ID = -1;
+    public static final int OBJECT_TYPE_ID = -1;
 
     /** */
-    static final int UNREGISTERED_TYPE_ID = 0;
+    public static final int UNREGISTERED_TYPE_ID = 0;
 
     /** */
-    static final int TYPE_ID_POS = 2;
+    public static final int TYPE_ID_POS = 2;
 
     /** */
-    static final int HASH_CODE_POS = 6;
+    public static final int HASH_CODE_POS = 6;
 
     /** */
-    static final int TOTAL_LEN_POS = 10;
+    public static final int TOTAL_LEN_POS = 10;
 
     /** */
-    static final byte RAW_DATA_OFF_POS = 14;
+    public static final byte RAW_DATA_OFF_POS = 14;
 
     /** */
-    static final int CLS_NAME_POS = 18;
+    public static final int CLS_NAME_POS = 18;
 
     /** */
-    static final byte DFLT_HDR_LEN = 18;
+    public static final byte DFLT_HDR_LEN = 18;
 
     /** */
     private final PortableContext ctx;
@@ -301,4 +301,4 @@ public class GridPortableMarshaller {
     public PortableContext context() {
         return ctx;
     }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/9057a4c0/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableAbstractLazyValue.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableAbstractLazyValue.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableAbstractLazyValue.java
deleted file mode 100644
index 2b1c4b7..0000000
--- a/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableAbstractLazyValue.java
+++ /dev/null
@@ -1,57 +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;
-
-/**
- *
- */
-abstract class PortableAbstractLazyValue implements PortableLazyValue {
-    /** */
-    protected Object val;
-
-    /** */
-    protected final PortableBuilderReader reader;
-
-    /** */
-    protected final int valOff;
-
-    /**
-     * @param reader Reader.
-     * @param valOff Value.
-     */
-    protected PortableAbstractLazyValue(PortableBuilderReader reader, int valOff) {
-        this.reader = reader;
-        this.valOff = valOff;
-    }
-
-    /**
-     * @return Value.
-     */
-    protected abstract Object init();
-
-    /** {@inheritDoc} */
-    @Override public Object value() {
-        if (val == null) {
-            val = init();
-
-            assert val != null;
-        }
-
-        return val;
-    }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/9057a4c0/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableBuilderEnum.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableBuilderEnum.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableBuilderEnum.java
deleted file mode 100644
index b6ace99..0000000
--- a/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableBuilderEnum.java
+++ /dev/null
@@ -1,114 +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;
-
-import org.apache.ignite.internal.util.typedef.internal.U;
-import org.apache.ignite.portable.PortableInvalidClassException;
-
-/**
- *
- */
-public class PortableBuilderEnum implements PortableBuilderSerializationAware {
-    /** */
-    private final int ordinal;
-
-    /** */
-    private final int typeId;
-
-    /** */
-    private final String clsName;
-
-    /**
-     * @param typeId Type ID.
-     * @param anEnum Enum instance.
-     */
-    public PortableBuilderEnum(int typeId, Enum anEnum) {
-        ordinal = anEnum.ordinal();
-        this.typeId = typeId;
-        clsName = null;
-    }
-
-    /**
-     * @param reader PortableBuilderReader.
-     */
-    public PortableBuilderEnum(PortableBuilderReader reader) {
-        int typeId = reader.readInt();
-
-        if (typeId == GridPortableMarshaller.UNREGISTERED_TYPE_ID) {
-            clsName = reader.readString();
-
-            Class cls;
-
-            try {
-                // TODO: IGNITE-1272 - Is class loader needed here?
-                cls = U.forName(reader.readString(), null);
-            }
-            catch (ClassNotFoundException e) {
-                throw new PortableInvalidClassException("Failed to load the class: " + clsName, e);
-            }
-
-            this.typeId = reader.portableContext().descriptorForClass(cls).typeId();
-        }
-        else {
-            this.typeId = typeId;
-            this.clsName = null;
-        }
-
-        ordinal = reader.readInt();
-    }
-
-    /**
-     * @return Ordinal.
-     */
-    public int getOrdinal() {
-        return ordinal;
-    }
-
-    /** {@inheritDoc} */
-    @Override public void writeTo(PortableWriterExImpl writer, PortableBuilderSerializer ctx) {
-        writer.writeByte(GridPortableMarshaller.ENUM);
-
-        if (typeId == GridPortableMarshaller.UNREGISTERED_TYPE_ID) {
-            writer.writeInt(GridPortableMarshaller.UNREGISTERED_TYPE_ID);
-            writer.writeString(clsName);
-        }
-        else
-            writer.writeInt(typeId);
-
-        writer.writeInt(ordinal);
-    }
-
-    /** {@inheritDoc} */
-    @Override public boolean equals(Object o) {
-        if (o == null || getClass() != o.getClass())
-            return false;
-
-        PortableBuilderEnum that = (PortableBuilderEnum)o;
-
-        return ordinal == that.ordinal && typeId == that.typeId;
-    }
-
-    /** {@inheritDoc} */
-    @Override public int hashCode() {
-        int result = ordinal;
-
-        result = 31 * result + typeId;
-
-        return result;
-    }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/9057a4c0/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableBuilderImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableBuilderImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableBuilderImpl.java
deleted file mode 100644
index dac199a..0000000
--- a/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableBuilderImpl.java
+++ /dev/null
@@ -1,531 +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;
-
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.LinkedHashMap;
-import java.util.Map;
-import java.util.Set;
-import org.apache.ignite.internal.processors.cache.portable.CacheObjectPortableProcessorImpl;
-import org.apache.ignite.internal.util.GridArgumentCheck;
-import org.apache.ignite.internal.util.typedef.internal.U;
-import org.apache.ignite.portable.PortableBuilder;
-import org.apache.ignite.portable.PortableException;
-import org.apache.ignite.portable.PortableInvalidClassException;
-import org.apache.ignite.portable.PortableMetadata;
-import org.apache.ignite.portable.PortableObject;
-import org.jetbrains.annotations.Nullable;
-
-import static org.apache.ignite.internal.portable.GridPortableMarshaller.CLS_NAME_POS;
-import static org.apache.ignite.internal.portable.GridPortableMarshaller.DFLT_HDR_LEN;
-import static org.apache.ignite.internal.portable.GridPortableMarshaller.HASH_CODE_POS;
-import static org.apache.ignite.internal.portable.GridPortableMarshaller.RAW_DATA_OFF_POS;
-import static org.apache.ignite.internal.portable.GridPortableMarshaller.TOTAL_LEN_POS;
-import static org.apache.ignite.internal.portable.GridPortableMarshaller.TYPE_ID_POS;
-import static org.apache.ignite.internal.portable.GridPortableMarshaller.UNREGISTERED_TYPE_ID;
-
-/**
- *
- */
-public class PortableBuilderImpl implements PortableBuilder {
-    /** */
-    private static final Object REMOVED_FIELD_MARKER = new Object();
-
-    /** */
-    private final PortableContext ctx;
-
-    /** */
-    private final int typeId;
-
-    /** May be null. */
-    private String typeName;
-
-    /** May be null. */
-    private String clsNameToWrite;
-
-    /** */
-    private boolean registeredType = true;
-
-    /** */
-    private Map<String, Object> assignedVals;
-
-    /** */
-    private Map<Integer, Object> readCache;
-
-    /** Position of object in source array, or -1 if object is not created from PortableObject. */
-    private final int start;
-
-    /** Total header length */
-    private final int hdrLen;
-
-    /**
-     * Context of PortableObject reading process. Or {@code null} if object is not created from PortableObject.
-     */
-    private final PortableBuilderReader reader;
-
-    /** */
-    private int hashCode;
-
-    /**
-     * @param clsName Class name.
-     * @param ctx Portable context.
-     */
-    public PortableBuilderImpl(PortableContext ctx, String clsName) {
-        this(ctx, ctx.typeId(clsName), PortableContext.typeName(clsName));
-    }
-
-    /**
-     * @param typeId Type ID.
-     * @param ctx Portable context.
-     */
-    public PortableBuilderImpl(PortableContext ctx, int typeId) {
-        this(ctx, typeId, null);
-    }
-
-    /**
-     * @param typeName Type name.
-     * @param ctx Context.
-     * @param typeId Type id.
-     */
-    public PortableBuilderImpl(PortableContext ctx, int typeId, String typeName) {
-        this.typeId = typeId;
-        this.typeName = typeName;
-        this.ctx = ctx;
-
-        start = -1;
-        reader = null;
-        hdrLen = DFLT_HDR_LEN;
-
-        readCache = Collections.emptyMap();
-    }
-
-    /**
-     * @param obj Object to wrap.
-     */
-    public PortableBuilderImpl(PortableObjectImpl obj) {
-        this(new PortableBuilderReader(obj), obj.start());
-
-        reader.registerObject(this);
-    }
-
-    /**
-     * @param reader ctx
-     * @param start Start.
-     */
-    PortableBuilderImpl(PortableBuilderReader reader, int start) {
-        this.reader = reader;
-        this.start = start;
-
-        int typeId = reader.readIntAbsolute(start + TYPE_ID_POS);
-        ctx = reader.portableContext();
-        hashCode = reader.readIntAbsolute(start + HASH_CODE_POS);
-
-        if (typeId == UNREGISTERED_TYPE_ID) {
-            int mark = reader.position();
-
-            reader.position(start + CLS_NAME_POS);
-
-            clsNameToWrite = reader.readString();
-
-            Class cls;
-
-            try {
-                // TODO: IGNITE-1272 - Is class loader needed here?
-                cls = U.forName(clsNameToWrite, null);
-            }
-            catch (ClassNotFoundException e) {
-                throw new PortableInvalidClassException("Failed to load the class: " + clsNameToWrite, e);
-            }
-
-            this.typeId = ctx.descriptorForClass(cls).typeId();
-
-            registeredType = false;
-
-            hdrLen = reader.position() - mark;
-
-            reader.position(mark);
-        }
-        else {
-            this.typeId = typeId;
-            hdrLen = DFLT_HDR_LEN;
-        }
-    }
-
-    /** {@inheritDoc} */
-    @Override public PortableObject build() {
-        try (PortableWriterExImpl writer = new PortableWriterExImpl(ctx, 0, typeId, false)) {
-
-            PortableBuilderSerializer serializationCtx = new PortableBuilderSerializer();
-
-            serializationCtx.registerObjectWriting(this, 0);
-
-            serializeTo(writer, serializationCtx);
-
-            byte[] arr = writer.array();
-
-            return new PortableObjectImpl(ctx, arr, 0);
-        }
-    }
-
-    /**
-     * @param writer Writer.
-     * @param serializer Serializer.
-     */
-    void serializeTo(PortableWriterExImpl writer, PortableBuilderSerializer serializer) {
-        writer.doWriteByte(GridPortableMarshaller.OBJ);
-        writer.doWriteBoolean(true);
-        writer.doWriteInt(registeredType ? typeId : UNREGISTERED_TYPE_ID);
-        writer.doWriteInt(hashCode);
-
-        // Length and raw offset.
-        writer.reserve(8);
-
-        if (!registeredType)
-            writer.writeString(clsNameToWrite);
-
-
-        Set<Integer> remainsFlds = null;
-
-        if (reader != null) {
-            Map<Integer, Object> assignedFldsById;
-
-            if (assignedVals != null) {
-                assignedFldsById = U.newHashMap(assignedVals.size());
-
-                for (Map.Entry<String, Object> entry : assignedVals.entrySet()) {
-                    int fldId = ctx.fieldId(typeId, entry.getKey());
-
-                    assignedFldsById.put(fldId, entry.getValue());
-                }
-
-                remainsFlds = assignedFldsById.keySet();
-            }
-            else
-                assignedFldsById = Collections.emptyMap();
-
-            int rawOff = start + reader.readIntAbsolute(start + RAW_DATA_OFF_POS);
-
-            reader.position(start + hdrLen);
-
-            int cpStart = -1;
-
-            while (reader.position() < rawOff) {
-                int fldId = reader.readInt();
-
-                int len = reader.readInt();
-
-                if (assignedFldsById.containsKey(fldId)) {
-                    if (cpStart >= 0) {
-                        writer.write(reader.array(), cpStart, reader.position() - 4 - 4 - cpStart);
-
-                        cpStart = -1;
-                    }
-
-                    Object assignedVal = assignedFldsById.remove(fldId);
-
-                    reader.skip(len);
-
-                    if (assignedVal != REMOVED_FIELD_MARKER) {
-                        writer.writeInt(fldId);
-
-                        int lenPos = writer.reserveAndMark(4);
-
-                        serializer.writeValue(writer, assignedVal);
-
-                        writer.writeDelta(lenPos);
-                    }
-                }
-                else {
-                    if (len != 0 && PortableUtils.isPlainType(reader.readByte(0))) {
-                        if (cpStart < 0)
-                            cpStart = reader.position() - 4 - 4;
-
-                        reader.skip(len);
-                    }
-                    else {
-                        if (cpStart >= 0) {
-                            writer.write(reader.array(), cpStart, reader.position() - 4 - cpStart);
-
-                            cpStart = -1;
-                        }
-                        else
-                            writer.writeInt(fldId);
-
-                        Object val;
-
-                        if (len == 0)
-                            val = null;
-                        else if (readCache == null) {
-                            int savedPos = reader.position();
-
-                            val = reader.parseValue();
-
-                            assert reader.position() == savedPos + len;
-                        }
-                        else {
-                            val = readCache.get(fldId);
-
-                            reader.skip(len);
-                        }
-
-                        int lenPos = writer.reserveAndMark(4);
-
-                        serializer.writeValue(writer, val);
-
-                        writer.writeDelta(lenPos);
-                    }
-                }
-            }
-
-            if (cpStart >= 0)
-                writer.write(reader.array(), cpStart, reader.position() - cpStart);
-        }
-
-        if (assignedVals != null && (remainsFlds == null || !remainsFlds.isEmpty())) {
-            boolean metadataEnabled = ctx.isMetaDataEnabled(typeId);
-
-            PortableMetadata metadata = null;
-
-            if (metadataEnabled)
-                metadata = ctx.metaData(typeId);
-
-            Map<String, String> newFldsMetadata = null;
-
-            for (Map.Entry<String, Object> entry : assignedVals.entrySet()) {
-                Object val = entry.getValue();
-
-                if (val == REMOVED_FIELD_MARKER)
-                    continue;
-
-                String name = entry.getKey();
-
-                int fldId = ctx.fieldId(typeId, name);
-
-                if (remainsFlds != null && !remainsFlds.contains(fldId))
-                    continue;
-
-                writer.writeInt(fldId);
-
-                int lenPos = writer.reserveAndMark(4);
-
-                serializer.writeValue(writer, val);
-
-                writer.writeDelta(lenPos);
-
-                if (metadataEnabled) {
-                    String oldFldTypeName = metadata == null ? null : metadata.fieldTypeName(name);
-
-                    String newFldTypeName;
-
-                    if (val instanceof PortableValueWithType)
-                        newFldTypeName = ((PortableValueWithType)val).typeName();
-                    else {
-                        byte type = PortableUtils.typeByClass(val.getClass());
-
-                        newFldTypeName = CacheObjectPortableProcessorImpl.fieldTypeName(type);
-                    }
-
-                    if (oldFldTypeName == null) {
-                        // It's a new field, we have to add it to metadata.
-
-                        if (newFldsMetadata == null)
-                            newFldsMetadata = new HashMap<>();
-
-                        newFldsMetadata.put(name, newFldTypeName);
-                    }
-                    else {
-                        if (!"Object".equals(oldFldTypeName) && !oldFldTypeName.equals(newFldTypeName)) {
-                            throw new PortableException(
-                                "Wrong value has been set [" +
-                                    "typeName=" + (typeName == null ? metadata.typeName() : typeName) +
-                                    ", fieldName=" + name +
-                                    ", fieldType=" + oldFldTypeName +
-                                    ", assignedValueType=" + newFldTypeName +
-                                    ", assignedValue=" + (((PortableValueWithType)val).value()) + ']'
-                            );
-                        }
-                    }
-                }
-            }
-
-            if (newFldsMetadata != null) {
-                String typeName = this.typeName;
-
-                if (typeName == null)
-                    typeName = metadata.typeName();
-
-                ctx.updateMetaData(typeId, typeName, newFldsMetadata);
-            }
-        }
-
-        writer.writeRawOffsetIfNeeded();
-
-        if (reader != null) {
-            int rawOff = reader.readIntAbsolute(start + RAW_DATA_OFF_POS);
-            int len = reader.readIntAbsolute(start + TOTAL_LEN_POS);
-
-            if (rawOff < len)
-                writer.write(reader.array(), rawOff, len - rawOff);
-        }
-
-        writer.writeLength();
-    }
-
-    /** {@inheritDoc} */
-    @Override public PortableBuilderImpl hashCode(int hashCode) {
-        this.hashCode = hashCode;
-
-        return this;
-    }
-
-    /**
-     *
-     */
-    private void ensureReadCacheInit() {
-        if (readCache == null) {
-            Map<Integer, Object> readCache = new HashMap<>();
-
-            int pos = start + hdrLen;
-            int end = start + reader.readIntAbsolute(start + RAW_DATA_OFF_POS);
-
-            while (pos < end) {
-                int fieldId = reader.readIntAbsolute(pos);
-
-                pos += 4;
-
-                int len = reader.readIntAbsolute(pos);
-
-                pos += 4;
-
-                Object val = reader.getValueQuickly(pos, len);
-
-                readCache.put(fieldId, val);
-
-                pos += len;
-            }
-
-            this.readCache = readCache;
-        }
-    }
-
-    /** {@inheritDoc} */
-    @Override public <F> F getField(String name) {
-        Object val;
-
-        if (assignedVals != null && assignedVals.containsKey(name)) {
-            val = assignedVals.get(name);
-
-            if (val == REMOVED_FIELD_MARKER)
-                return null;
-        }
-        else {
-            ensureReadCacheInit();
-
-            int fldId = ctx.fieldId(typeId, name);
-
-            val = readCache.get(fldId);
-        }
-
-        return (F)PortableUtils.unwrapLazy(val);
-    }
-
-    /** {@inheritDoc} */
-    @Override public PortableBuilder setField(String name, Object val) {
-        GridArgumentCheck.notNull(val, "val");
-
-        if (assignedVals == null)
-            assignedVals = new LinkedHashMap<>();
-
-        Object oldVal = assignedVals.put(name, val);
-
-        if (oldVal instanceof PortableValueWithType) {
-            ((PortableValueWithType)oldVal).value(val);
-
-            assignedVals.put(name, oldVal);
-        }
-
-        return this;
-    }
-
-    /** {@inheritDoc} */
-    @Override public <T> PortableBuilder setField(String name, @Nullable T val, Class<? super T> type) {
-        if (assignedVals == null)
-            assignedVals = new LinkedHashMap<>();
-
-        //int fldId = ctx.fieldId(typeId, fldName);
-
-        assignedVals.put(name, new PortableValueWithType(PortableUtils.typeByClass(type), val));
-
-        return this;
-    }
-
-    /** {@inheritDoc} */
-    @Override public PortableBuilder setField(String name, @Nullable PortableBuilder builder) {
-        if (builder == null)
-            return setField(name, null, Object.class);
-        else
-            return setField(name, (Object)builder);
-    }
-
-    /**
-     * Removes field from portable object.
-     *
-     * @param name Field name.
-     * @return {@code this} instance for chaining.
-     */
-    @Override public PortableBuilderImpl removeField(String name) {
-        if (assignedVals == null)
-            assignedVals = new LinkedHashMap<>();
-
-        assignedVals.put(name, REMOVED_FIELD_MARKER);
-
-        return this;
-    }
-
-    /**
-     * Creates builder initialized by specified portable object.
-     *
-     * @param obj Portable object to initialize builder.
-     * @return New builder.
-     */
-    public static PortableBuilderImpl wrap(PortableObject obj) {
-        PortableObjectImpl heapObj;
-
-        if (obj instanceof PortableObjectOffheapImpl)
-            heapObj = (PortableObjectImpl)((PortableObjectOffheapImpl)obj).heapCopy();
-        else
-            heapObj = (PortableObjectImpl)obj;
-
-        return new PortableBuilderImpl(heapObj);
-    }
-
-    /**
-     * @return Object start position in source array.
-     */
-    int start() {
-        return start;
-    }
-
-    /**
-     * @return Object type id.
-     */
-    int typeId() {
-        return typeId;
-    }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/9057a4c0/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableBuilderReader.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableBuilderReader.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableBuilderReader.java
deleted file mode 100644
index 30b31f0..0000000
--- a/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableBuilderReader.java
+++ /dev/null
@@ -1,776 +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;
-
-import java.sql.Timestamp;
-import java.util.Date;
-import java.util.HashMap;
-import java.util.Map;
-import org.apache.ignite.portable.PortableException;
-
-import static java.nio.charset.StandardCharsets.UTF_8;
-import static org.apache.ignite.internal.portable.GridPortableMarshaller.NULL;
-import static org.apache.ignite.internal.portable.GridPortableMarshaller.STRING;
-
-/**
- *
- */
-class PortableBuilderReader {
-    /** */
-    private static final PortablePrimitives PRIM = PortablePrimitives.get();
-
-    /** */
-    private final Map<Integer, PortableBuilderImpl> objMap = new HashMap<>();
-
-    /** */
-    private final PortableContext ctx;
-
-    /** */
-    private final PortableReaderExImpl reader;
-
-    /** */
-    private byte[] arr;
-
-    /** */
-    private int pos;
-
-    /**
-     * @param objImpl Portable object
-     */
-    PortableBuilderReader(PortableObjectImpl objImpl) {
-        ctx = objImpl.context();
-        arr = objImpl.array();
-        pos = objImpl.start();
-
-        // TODO: IGNITE-1272 - Is class loader needed here?
-        reader = new PortableReaderExImpl(portableContext(), arr, pos, null);
-    }
-
-    /**
-     * @return Portable context.
-     */
-    public PortableContext portableContext() {
-        return ctx;
-    }
-
-    /**
-     * @param obj Mutable portable object.
-     */
-    public void registerObject(PortableBuilderImpl obj) {
-        objMap.put(obj.start(), obj);
-    }
-
-    /**
-     * @return Read int value.
-     */
-    public int readInt() {
-        int res = readInt(0);
-
-        pos += 4;
-
-        return res;
-    }
-
-    /**
-     * @return Read int value.
-     */
-    public byte readByte() {
-        return arr[pos++];
-    }
-
-    /**
-     * @return Read boolean value.
-     */
-    public boolean readBoolean() {
-        return readByte() == 1;
-    }
-
-    /**
-     * @return Read int value.
-     */
-    public byte readByte(int off) {
-        return arr[pos + off];
-    }
-
-    /**
-     * @param off Offset related to {@link #pos}
-     * @return Read int value.
-     */
-    public int readInt(int off) {
-        return PRIM.readInt(arr, pos + off);
-    }
-
-    /**
-     * @param pos Position in the source array.
-     * @return Read int value.
-     */
-    public int readIntAbsolute(int pos) {
-        return PRIM.readInt(arr, pos);
-    }
-
-    /**
-     * @return Read length of array.
-     */
-    public int readLength() {
-        return PRIM.readInt(arr, pos);
-    }
-
-    /**
-     * Read string length.
-     *
-     * @return String length.
-     */
-    public int readStringLength() {
-        boolean utf = PRIM.readBoolean(arr, pos);
-
-        int arrLen = PRIM.readInt(arr, pos + 1);
-
-        return 1 + (utf ? arrLen : arrLen << 1);
-    }
-
-    /**
-     * Reads string.
-     *
-     * @return String.
-     */
-    public String readString() {
-        byte flag = readByte();
-
-        if (flag == NULL)
-            return null;
-
-        if (flag != STRING)
-            throw new PortableException("Failed to deserialize String.");
-
-        boolean convert = readBoolean();
-        int len = readInt();
-
-        String str;
-
-        if (convert) {
-            str = new String(arr, pos, len, UTF_8);
-
-            pos += len;
-        }
-        else {
-            str = String.valueOf(PRIM.readCharArray(arr, pos, len));
-
-            pos += len << 1;
-        }
-
-        return str;
-    }
-
-    /**
-     *
-     */
-    public void skipValue() {
-        byte type = arr[pos++];
-
-        int len;
-
-        switch (type) {
-            case GridPortableMarshaller.NULL:
-                return;
-
-            case GridPortableMarshaller.OBJ:
-                pos += readInt(GridPortableMarshaller.TOTAL_LEN_POS - 1) - 1;
-
-                return;
-
-            case GridPortableMarshaller.BOOLEAN:
-            case GridPortableMarshaller.BYTE:
-                len = 1;
-                break;
-
-            case GridPortableMarshaller.CHAR:
-            case GridPortableMarshaller.SHORT:
-                len = 2;
-
-                break;
-
-            case GridPortableMarshaller.HANDLE:
-            case GridPortableMarshaller.FLOAT:
-            case GridPortableMarshaller.INT:
-                len = 4;
-
-                break;
-
-            case GridPortableMarshaller.ENUM:
-                //skipping type id and ordinal value
-                len = 8;
-
-                break;
-
-            case GridPortableMarshaller.LONG:
-            case GridPortableMarshaller.DOUBLE:
-                len = 8;
-
-                break;
-
-            case GridPortableMarshaller.BYTE_ARR:
-            case GridPortableMarshaller.BOOLEAN_ARR:
-                len = 4 + readLength();
-
-                break;
-
-            case GridPortableMarshaller.STRING:
-                len = 4 + readStringLength();
-
-                break;
-
-            case GridPortableMarshaller.DECIMAL:
-                len = /** scale */ 4  + /** mag len */ 4  + /** mag bytes count */ readInt(4);
-
-                break;
-
-            case GridPortableMarshaller.UUID:
-                len = 8 + 8;
-
-                break;
-
-            case GridPortableMarshaller.DATE:
-                len = 8 + 4;
-
-                break;
-
-            case GridPortableMarshaller.CHAR_ARR:
-            case GridPortableMarshaller.SHORT_ARR:
-                len = 4 + readLength() * 2;
-
-                break;
-
-            case GridPortableMarshaller.INT_ARR:
-            case GridPortableMarshaller.FLOAT_ARR:
-                len = 4 + readLength() * 4;
-
-                break;
-
-            case GridPortableMarshaller.LONG_ARR:
-            case GridPortableMarshaller.DOUBLE_ARR:
-                len = 4 + readLength() * 8;
-
-                break;
-
-            case GridPortableMarshaller.DECIMAL_ARR:
-            case GridPortableMarshaller.DATE_ARR:
-            case GridPortableMarshaller.OBJ_ARR:
-            case GridPortableMarshaller.ENUM_ARR:
-            case GridPortableMarshaller.UUID_ARR:
-            case GridPortableMarshaller.STRING_ARR: {
-                int size = readInt();
-
-                for (int i = 0; i < size; i++)
-                    skipValue();
-
-                return;
-            }
-
-            case GridPortableMarshaller.COL: {
-                int size = readInt();
-
-                pos++; // skip collection type
-
-                for (int i = 0; i < size; i++)
-                    skipValue();
-
-                return;
-            }
-
-            case GridPortableMarshaller.MAP: {
-                int size = readInt();
-
-                pos++; // skip collection type
-
-                for (int i = 0; i < size; i++) {
-                    skipValue(); // skip key.
-                    skipValue(); // skip value.
-                }
-
-                return;
-            }
-
-            case GridPortableMarshaller.MAP_ENTRY:
-                skipValue();
-                skipValue();
-
-                return;
-
-            case GridPortableMarshaller.PORTABLE_OBJ:
-                len = readInt() + 4;
-
-                break;
-
-            default:
-                throw new PortableException("Invalid flag value: " + type);
-        }
-
-        pos += len;
-    }
-
-    /**
-     * @param pos Position.
-     * @param len Length.
-     * @return Object.
-     */
-    public Object getValueQuickly(int pos, int len) {
-        byte type = arr[pos];
-
-        switch (type) {
-            case GridPortableMarshaller.NULL:
-                return null;
-
-            case GridPortableMarshaller.HANDLE: {
-                int objStart = pos - readIntAbsolute(pos + 1);
-
-                PortableBuilderImpl res = objMap.get(objStart);
-
-                if (res == null) {
-                    res = new PortableBuilderImpl(this, objStart);
-
-                    objMap.put(objStart, res);
-                }
-
-                return res;
-            }
-
-            case GridPortableMarshaller.OBJ: {
-                PortableBuilderImpl res = objMap.get(pos);
-
-                if (res == null) {
-                    res = new PortableBuilderImpl(this, pos);
-
-                    objMap.put(pos, res);
-                }
-
-                return res;
-            }
-
-            case GridPortableMarshaller.BYTE:
-                return arr[pos + 1];
-
-            case GridPortableMarshaller.SHORT:
-                return PRIM.readShort(arr, pos + 1);
-
-            case GridPortableMarshaller.INT:
-                return PRIM.readInt(arr, pos + 1);
-
-            case GridPortableMarshaller.LONG:
-                return PRIM.readLong(arr, pos + 1);
-
-            case GridPortableMarshaller.FLOAT:
-                return PRIM.readFloat(arr, pos + 1);
-
-            case GridPortableMarshaller.DOUBLE:
-                return PRIM.readDouble(arr, pos + 1);
-
-            case GridPortableMarshaller.CHAR:
-                return PRIM.readChar(arr, pos + 1);
-
-            case GridPortableMarshaller.BOOLEAN:
-                return arr[pos + 1] != 0;
-
-            case GridPortableMarshaller.DECIMAL:
-            case GridPortableMarshaller.STRING:
-            case GridPortableMarshaller.UUID:
-            case GridPortableMarshaller.DATE:
-            case GridPortableMarshaller.BYTE_ARR:
-            case GridPortableMarshaller.SHORT_ARR:
-            case GridPortableMarshaller.INT_ARR:
-            case GridPortableMarshaller.LONG_ARR:
-            case GridPortableMarshaller.FLOAT_ARR:
-            case GridPortableMarshaller.DOUBLE_ARR:
-            case GridPortableMarshaller.CHAR_ARR:
-            case GridPortableMarshaller.BOOLEAN_ARR:
-            case GridPortableMarshaller.DECIMAL_ARR:
-            case GridPortableMarshaller.DATE_ARR:
-            case GridPortableMarshaller.UUID_ARR:
-            case GridPortableMarshaller.STRING_ARR:
-                return new PortablePlainLazyValue(this, pos, len);
-
-            case GridPortableMarshaller.COL:
-            case GridPortableMarshaller.OBJ_ARR:
-            case GridPortableMarshaller.MAP:
-            case GridPortableMarshaller.ENUM_ARR:
-            case GridPortableMarshaller.MAP_ENTRY:
-                return new LazyCollection(pos);
-
-            case GridPortableMarshaller.ENUM: {
-                if (len == 1) {
-                    assert readByte(pos) == GridPortableMarshaller.NULL;
-
-                    return null;
-                }
-
-                int mark = position();
-                position(pos + 1);
-
-                PortableBuilderEnum builderEnum = new PortableBuilderEnum(this);
-
-                position(mark);
-
-                return builderEnum;
-            }
-
-            case GridPortableMarshaller.PORTABLE_OBJ: {
-                int size = readIntAbsolute(pos + 1);
-
-                int start = readIntAbsolute(pos + 4 + size);
-
-                PortableObjectImpl portableObj = new PortableObjectImpl(ctx, arr, pos + 4 + start);
-
-                return new PortablePlainPortableObject(portableObj);
-            }
-
-            default:
-                throw new PortableException("Invalid flag value: " + type);
-        }
-    }
-
-    /**
-     * @return Parsed value.
-     */
-    public Object parseValue() {
-        int valPos = pos;
-
-        byte type = arr[pos++];
-
-        int plainLazyValLen;
-
-        switch (type) {
-            case GridPortableMarshaller.NULL:
-                return null;
-
-            case GridPortableMarshaller.HANDLE: {
-                int objStart = pos - 1 - readInt();
-
-                PortableBuilderImpl res = objMap.get(objStart);
-
-                if (res == null) {
-                    res = new PortableBuilderImpl(this, objStart);
-
-                    objMap.put(objStart, res);
-                }
-
-                return res;
-            }
-
-            case GridPortableMarshaller.OBJ: {
-                pos--;
-
-                PortableBuilderImpl res = objMap.get(pos);
-
-                if (res == null) {
-                    res = new PortableBuilderImpl(this, pos);
-
-                    objMap.put(pos, res);
-                }
-
-                pos += readInt(GridPortableMarshaller.TOTAL_LEN_POS);
-
-                return res;
-            }
-
-            case GridPortableMarshaller.BYTE:
-                return arr[pos++];
-
-            case GridPortableMarshaller.SHORT: {
-                Object res = PRIM.readShort(arr, pos);
-                pos += 2;
-                return res;
-            }
-
-            case GridPortableMarshaller.INT:
-                return readInt();
-
-            case GridPortableMarshaller.LONG:
-                plainLazyValLen = 8;
-
-                break;
-
-            case GridPortableMarshaller.FLOAT:
-                plainLazyValLen = 4;
-
-                break;
-
-            case GridPortableMarshaller.DOUBLE:
-                plainLazyValLen = 8;
-
-                break;
-
-            case GridPortableMarshaller.CHAR:
-                plainLazyValLen = 2;
-
-                break;
-
-            case GridPortableMarshaller.BOOLEAN:
-                return arr[pos++] != 0;
-
-            case GridPortableMarshaller.DECIMAL:
-                plainLazyValLen = /** scale */ 4  + /** mag len */ 4  + /** mag bytes count */ readInt(4);
-
-                break;
-
-            case GridPortableMarshaller.STRING:
-                plainLazyValLen = 4 + readStringLength();
-
-                break;
-
-            case GridPortableMarshaller.UUID:
-                plainLazyValLen = 8 + 8;
-
-                break;
-
-            case GridPortableMarshaller.DATE:
-                plainLazyValLen = 8 + 4;
-
-                break;
-
-            case GridPortableMarshaller.BYTE_ARR:
-                plainLazyValLen = 4 + readLength();
-
-                break;
-
-            case GridPortableMarshaller.SHORT_ARR:
-                plainLazyValLen = 4 + readLength() * 2;
-
-                break;
-
-            case GridPortableMarshaller.INT_ARR:
-                plainLazyValLen = 4 + readLength() * 4;
-
-                break;
-
-            case GridPortableMarshaller.LONG_ARR:
-                plainLazyValLen = 4 + readLength() * 8;
-
-                break;
-
-            case GridPortableMarshaller.FLOAT_ARR:
-                plainLazyValLen = 4 + readLength() * 4;
-
-                break;
-
-            case GridPortableMarshaller.DOUBLE_ARR:
-                plainLazyValLen = 4 + readLength() * 8;
-
-                break;
-
-            case GridPortableMarshaller.CHAR_ARR:
-                plainLazyValLen = 4 + readLength() * 2;
-
-                break;
-
-            case GridPortableMarshaller.BOOLEAN_ARR:
-                plainLazyValLen = 4 + readLength();
-
-                break;
-
-            case GridPortableMarshaller.OBJ_ARR:
-                return new PortableObjectArrayLazyValue(this);
-
-            case GridPortableMarshaller.DATE_ARR: {
-                int size = readInt();
-
-                Date[] res = new Date[size];
-
-                for (int i = 0; i < res.length; i++) {
-                    byte flag = arr[pos++];
-
-                    if (flag == GridPortableMarshaller.NULL) continue;
-
-                    if (flag != GridPortableMarshaller.DATE)
-                        throw new PortableException("Invalid flag value: " + flag);
-
-                    long time = PRIM.readLong(arr, pos);
-
-                    pos += 8;
-
-                    if (ctx.isUseTimestamp()) {
-                        Timestamp ts = new Timestamp(time);
-
-                        ts.setNanos(ts.getNanos() + readInt());
-
-                        res[i] = ts;
-                    }
-                    else {
-                        res[i] = new Date(time);
-
-                        pos += 4;
-                    }
-                }
-
-                return res;
-            }
-
-            case GridPortableMarshaller.UUID_ARR:
-            case GridPortableMarshaller.STRING_ARR:
-            case GridPortableMarshaller.DECIMAL_ARR: {
-                int size = readInt();
-
-                for (int i = 0; i < size; i++) {
-                    byte flag = arr[pos++];
-
-                    if (flag == GridPortableMarshaller.UUID)
-                        pos += 8 + 8;
-                    else if (flag == GridPortableMarshaller.STRING)
-                        pos += 4 + readStringLength();
-                    else if (flag == GridPortableMarshaller.DECIMAL)
-                        pos += 4 + readLength();
-                    else
-                        assert flag == GridPortableMarshaller.NULL;
-                }
-
-                return new PortablePlainLazyValue(this, valPos, pos - valPos);
-            }
-
-            case GridPortableMarshaller.COL: {
-                int size = readInt();
-                byte colType = arr[pos++];
-
-                switch (colType) {
-                    case GridPortableMarshaller.USER_COL:
-                    case GridPortableMarshaller.ARR_LIST:
-                        return new PortableLazyArrayList(this, size);
-
-                    case GridPortableMarshaller.LINKED_LIST:
-                        return new PortableLazyLinkedList(this, size);
-
-                    case GridPortableMarshaller.HASH_SET:
-                    case GridPortableMarshaller.LINKED_HASH_SET:
-                    case GridPortableMarshaller.TREE_SET:
-                    case GridPortableMarshaller.CONC_SKIP_LIST_SET:
-                        return new PortableLazySet(this, size);
-                }
-
-                throw new PortableException("Unknown collection type: " + colType);
-            }
-
-            case GridPortableMarshaller.MAP:
-                return PortableLazyMap.parseMap(this);
-
-            case GridPortableMarshaller.ENUM:
-                return new PortableBuilderEnum(this);
-
-            case GridPortableMarshaller.ENUM_ARR:
-                return new PortableEnumArrayLazyValue(this);
-
-            case GridPortableMarshaller.MAP_ENTRY:
-                return new PortableLazyMapEntry(this);
-
-            case GridPortableMarshaller.PORTABLE_OBJ: {
-                int size = readInt();
-
-                pos += size;
-
-                int start = readInt();
-
-                PortableObjectImpl portableObj = new PortableObjectImpl(ctx, arr,
-                    pos - 4 - size + start);
-
-                return new PortablePlainPortableObject(portableObj);
-            }
-
-
-            default:
-                throw new PortableException("Invalid flag value: " + type);
-        }
-
-        PortablePlainLazyValue res = new PortablePlainLazyValue(this, valPos, 1 + plainLazyValLen);
-
-        pos += plainLazyValLen;
-
-        return res;
-    }
-
-    /**
-     * @return Array.
-     */
-    public byte[] array() {
-        return arr;
-    }
-
-    /**
-     * @return Position of reader.
-     */
-    public int position() {
-        return pos;
-    }
-
-    /**
-     * @param pos New pos.
-     */
-    public void position(int pos) {
-        this.pos = pos;
-    }
-
-    /**
-     * @param n Number of bytes to skip.
-     */
-    public void skip(int n) {
-        pos += n;
-    }
-
-    /**
-     * @return Reader.
-     */
-    PortableReaderExImpl reader() {
-        return reader;
-    }
-
-    /**
-     *
-     */
-    private class LazyCollection implements PortableLazyValue {
-        /** */
-        private final int valOff;
-
-        /** */
-        private Object col;
-
-        /**
-         * @param valOff Value.
-         */
-        protected LazyCollection(int valOff) {
-            this.valOff = valOff;
-        }
-
-        /**
-         * @return Object.
-         */
-        private Object wrappedCollection() {
-            if (col == null) {
-                position(valOff);
-
-                col = parseValue();
-            }
-
-            return col;
-        }
-
-        /** {@inheritDoc} */
-        @Override public void writeTo(PortableWriterExImpl writer, PortableBuilderSerializer ctx) {
-            ctx.writeValue(writer, wrappedCollection());
-        }
-
-        /** {@inheritDoc} */
-        @Override public Object value() {
-            return PortableUtils.unwrapLazy(wrappedCollection());
-        }
-    }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/9057a4c0/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableBuilderSerializationAware.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableBuilderSerializationAware.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableBuilderSerializationAware.java
deleted file mode 100644
index d75cd7b..0000000
--- a/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableBuilderSerializationAware.java
+++ /dev/null
@@ -1,29 +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;
-
-/**
- *
- */
-interface PortableBuilderSerializationAware {
-    /**
-     * @param writer Writer.
-     * @param ctx Context.
-     */
-    public void writeTo(PortableWriterExImpl writer, PortableBuilderSerializer ctx);
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/9057a4c0/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableBuilderSerializer.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableBuilderSerializer.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableBuilderSerializer.java
deleted file mode 100644
index 6ce9f4c..0000000
--- a/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableBuilderSerializer.java
+++ /dev/null
@@ -1,211 +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;
-
-import java.util.Collection;
-import java.util.IdentityHashMap;
-import java.util.Map;
-import org.apache.ignite.internal.util.GridConcurrentSkipListSet;
-import org.apache.ignite.portable.PortableObject;
-
-/**
- *
- */
-class PortableBuilderSerializer {
-    /** */
-    private final Map<PortableBuilderImpl, Integer> objToPos = new IdentityHashMap<>();
-
-    /** */
-    private Map<PortableObject, PortableBuilderImpl> portableObjToWrapper;
-
-    /**
-     * @param obj Mutable object.
-     * @param posInResArr Object position in the array.
-     */
-    public void registerObjectWriting(PortableBuilderImpl obj, int posInResArr) {
-        objToPos.put(obj, posInResArr);
-    }
-
-    /**
-     * @param writer Writer.
-     * @param val Value.
-     */
-    public void writeValue(PortableWriterExImpl writer, Object val) {
-        if (val == null) {
-            writer.writeByte(GridPortableMarshaller.NULL);
-
-            return;
-        }
-
-        if (val instanceof PortableBuilderSerializationAware) {
-            ((PortableBuilderSerializationAware)val).writeTo(writer, this);
-
-            return;
-        }
-
-        if (val instanceof PortableObjectEx) {
-            if (portableObjToWrapper == null)
-                portableObjToWrapper = new IdentityHashMap<>();
-
-            PortableBuilderImpl wrapper = portableObjToWrapper.get(val);
-
-            if (wrapper == null) {
-                wrapper = PortableBuilderImpl.wrap((PortableObject)val);
-
-                portableObjToWrapper.put((PortableObject)val, wrapper);
-            }
-
-            val = wrapper;
-        }
-
-        if (val instanceof PortableBuilderImpl) {
-            PortableBuilderImpl obj = (PortableBuilderImpl)val;
-
-            Integer posInResArr = objToPos.get(obj);
-
-            if (posInResArr == null) {
-                objToPos.put(obj, writer.outputStream().position());
-
-                obj.serializeTo(writer.newWriter(obj.typeId()), this);
-            }
-            else {
-                int handle = writer.outputStream().position() - posInResArr;
-
-                writer.writeByte(GridPortableMarshaller.HANDLE);
-                writer.writeInt(handle);
-            }
-
-            return;
-        }
-
-        if (val.getClass().isEnum()) {
-            writer.writeByte(GridPortableMarshaller.ENUM);
-            writer.writeInt(writer.context().typeId(val.getClass().getName()));
-            writer.writeInt(((Enum)val).ordinal());
-
-            return;
-        }
-
-        if (val instanceof Collection) {
-            Collection<?> c = (Collection<?>)val;
-
-            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());
-
-
-            writer.writeByte(colType);
-
-            for (Object obj : c)
-                writeValue(writer, obj);
-
-            return;
-        }
-
-        if (val instanceof Map) {
-            Map<?, ?> map = (Map<?, ?>)val;
-
-            writer.writeByte(GridPortableMarshaller.MAP);
-            writer.writeInt(map.size());
-
-            writer.writeByte(writer.context().mapType(map.getClass()));
-
-            for (Map.Entry<?, ?> entry : map.entrySet()) {
-                writeValue(writer, entry.getKey());
-                writeValue(writer, entry.getValue());
-            }
-
-            return;
-        }
-
-        Byte flag = PortableUtils.PLAIN_CLASS_TO_FLAG.get(val.getClass());
-
-        if (flag != null) {
-            PortableUtils.writePlainObject(writer, val);
-
-            return;
-        }
-
-        if (val instanceof Object[]) {
-            int compTypeId = writer.context().typeId(((Object[])val).getClass().getComponentType().getName());
-
-            if (val instanceof PortableBuilderEnum[]) {
-                writeArray(writer, GridPortableMarshaller.ENUM_ARR, (Object[])val, compTypeId);
-
-                return;
-            }
-
-            if (((Object[])val).getClass().getComponentType().isEnum()) {
-                Enum[] enumArr = (Enum[])val;
-
-                writer.writeByte(GridPortableMarshaller.ENUM_ARR);
-                writer.writeInt(compTypeId);
-                writer.writeInt(enumArr.length);
-
-                for (Enum anEnum : enumArr)
-                    writeValue(writer, anEnum);
-
-                return;
-            }
-
-            writeArray(writer, GridPortableMarshaller.OBJ_ARR, (Object[])val, compTypeId);
-
-            return;
-        }
-
-        writer.doWriteObject(val, false);
-    }
-
-    /**
-     * @param writer Writer.
-     * @param elementType Element type.
-     * @param arr The array.
-     * @param compTypeId Component type ID.
-     */
-    public void writeArray(PortableWriterExImpl writer, byte elementType, Object[] arr, int compTypeId) {
-        writer.writeByte(elementType);
-        writer.writeInt(compTypeId);
-        writer.writeInt(arr.length);
-
-        for (Object obj : arr)
-            writeValue(writer, obj);
-    }
-
-    /**
-     * @param writer Writer.
-     * @param elementType Element type.
-     * @param arr The array.
-     * @param clsName Component class name.
-     */
-    public void writeArray(PortableWriterExImpl writer, byte elementType, Object[] arr, String clsName) {
-        writer.writeByte(elementType);
-        writer.writeInt(GridPortableMarshaller.UNREGISTERED_TYPE_ID);
-        writer.writeString(clsName);
-        writer.writeInt(arr.length);
-
-        for (Object obj : arr)
-            writeValue(writer, obj);
-    }
-
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/9057a4c0/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 7e4266c..24ad5ce 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
@@ -48,7 +48,7 @@ import static java.lang.reflect.Modifier.isTransient;
 /**
  * Portable class descriptor.
  */
-class PortableClassDescriptor {
+public class PortableClassDescriptor {
     /** */
     private final PortableContext ctx;
 
@@ -123,7 +123,7 @@ class PortableClassDescriptor {
         boolean keepDeserialized
     ) throws PortableException {
         this(ctx, cls, userType, typeId, typeName, idMapper, serializer, useTs, metaDataEnabled, keepDeserialized,
-             true);
+            true);
     }
 
     /**
@@ -285,7 +285,7 @@ class PortableClassDescriptor {
     /**
      * @return Type ID.
      */
-    int typeId() {
+    public int typeId() {
         return typeId;
     }
 
@@ -399,7 +399,7 @@ class PortableClassDescriptor {
                 break;
 
             case DECIMAL:
-                writer.doWriteDecimal((BigDecimal) obj);
+                writer.doWriteDecimal((BigDecimal)obj);
 
                 break;
 
@@ -656,39 +656,32 @@ class PortableClassDescriptor {
      * @return Whether further write is needed.
      */
     private boolean writeHeader(Object obj, PortableWriterExImpl writer) {
-        int handle = writer.handle(obj);
-
-        if (handle >= 0) {
-            writer.doWriteByte(GridPortableMarshaller.HANDLE);
-            writer.doWriteInt(handle);
-
+        if (writer.tryWriteAsHandle(obj))
             return false;
-        }
-        else {
-            int pos = writer.position();
 
-            writer.doWriteByte(GridPortableMarshaller.OBJ);
-            writer.doWriteBoolean(userType);
-            writer.doWriteInt(registered ? typeId : GridPortableMarshaller.UNREGISTERED_TYPE_ID);
-            writer.doWriteInt(obj instanceof CacheObjectImpl ? 0 : obj.hashCode());
+        int pos = writer.position();
 
-            // For length and raw offset.
-            int reserved = writer.reserve(8);
+        writer.doWriteByte(GridPortableMarshaller.OBJ);
+        writer.doWriteBoolean(userType);
+        writer.doWriteInt(registered ? typeId : GridPortableMarshaller.UNREGISTERED_TYPE_ID);
+        writer.doWriteInt(obj instanceof CacheObjectImpl ? 0 : obj.hashCode());
 
-            // Class name in case if typeId registration is failed.
-            if (!registered)
-                writer.doWriteString(cls.getName());
+        // For length and raw offset.
+        int reserved = writer.reserve(8);
 
-            int current = writer.position();
-            int len = current - pos;
+        // Class name in case if typeId registration is failed.
+        if (!registered)
+            writer.doWriteString(cls.getName());
 
-            // Default raw offset (equal to header length).
-            writer.position(reserved + 4);
-            writer.doWriteInt(len);
-            writer.position(current);
+        int current = writer.position();
+        int len = current - pos;
 
-            return true;
-        }
+        // Default raw offset (equal to header length).
+        writer.position(reserved + 4);
+        writer.doWriteInt(len);
+        writer.position(current);
+
+        return true;
     }
 
     /**
@@ -787,7 +780,7 @@ class PortableClassDescriptor {
         else if (cls == PortableObjectImpl.class)
             return Mode.PORTABLE_OBJ;
         else if (PortableMarshalAware.class.isAssignableFrom(cls))
-           return Mode.PORTABLE;
+            return Mode.PORTABLE;
         else if (Externalizable.class.isAssignableFrom(cls))
             return Mode.EXTERNALIZABLE;
         else if (Map.Entry.class.isAssignableFrom(cls))
@@ -1352,4 +1345,4 @@ class PortableClassDescriptor {
             return typeName;
         }
     }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/9057a4c0/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 15b6433..db7e41e 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
@@ -794,10 +794,10 @@ public class PortableContext implements Externalizable {
      * @param typeId Type ID.
      * @return Whether meta data is enabled.
      */
-    boolean isMetaDataEnabled(int typeId) {
+    public boolean isMetaDataEnabled(int typeId) {
         Boolean enabled = metaEnabled.get(typeId);
 
-        return enabled != null ? enabled : true;
+        return enabled != null ? enabled : metaDataEnabled;
     }
 
     /**
@@ -827,7 +827,7 @@ public class PortableContext implements Externalizable {
      * @param fields Fields map.
      * @throws PortableException In case of error.
      */
-    void updateMetaData(int typeId, String typeName, Map<String, String> fields) throws PortableException {
+    public void updateMetaData(int typeId, String typeName, Map<String, String> fields) throws PortableException {
         updateMetaData(typeId, new PortableMetaDataImpl(typeName, fields, null));
     }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/9057a4c0/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableEnumArrayLazyValue.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableEnumArrayLazyValue.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableEnumArrayLazyValue.java
deleted file mode 100644
index c953bb3..0000000
--- a/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableEnumArrayLazyValue.java
+++ /dev/null
@@ -1,112 +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;
-
-import org.apache.ignite.internal.util.typedef.internal.U;
-import org.apache.ignite.portable.PortableException;
-import org.apache.ignite.portable.PortableInvalidClassException;
-
-/**
- *
- */
-class PortableEnumArrayLazyValue extends PortableAbstractLazyValue {
-    /** */
-    private final int len;
-
-    /** */
-    private final int compTypeId;
-
-    /** */
-    private final String clsName;
-
-    /**
-     * @param reader Reader.
-     */
-    protected PortableEnumArrayLazyValue(PortableBuilderReader reader) {
-        super(reader, reader.position() - 1);
-
-        int typeId = reader.readInt();
-
-        if (typeId == GridPortableMarshaller.UNREGISTERED_TYPE_ID) {
-            clsName = reader.readString();
-
-            Class cls;
-
-            try {
-                // TODO: IGNITE-1272 - Is class loader needed here?
-                cls = U.forName(reader.readString(), null);
-            }
-            catch (ClassNotFoundException e) {
-                throw new PortableInvalidClassException("Failed to load the class: " + clsName, e);
-            }
-
-            compTypeId = reader.portableContext().descriptorForClass(cls).typeId();
-        }
-        else {
-            compTypeId = typeId;
-            clsName = null;
-        }
-
-        int size = reader.readInt();
-
-        for (int i = 0; i < size; i++)
-            reader.skipValue();
-
-        len = reader.position() - valOff;
-    }
-
-    /** {@inheritDoc} */
-    @Override protected Object init() {
-        reader.position(valOff + 1);
-
-        //skipping component type id
-        reader.readInt();
-
-        int size = reader.readInt();
-
-        PortableBuilderEnum[] res = new PortableBuilderEnum[size];
-
-        for (int i = 0; i < size; i++) {
-            byte flag = reader.readByte();
-
-            if (flag == GridPortableMarshaller.NULL)
-                continue;
-
-            if (flag != GridPortableMarshaller.ENUM)
-                throw new PortableException("Invalid flag value: " + flag);
-
-            res[i] = new PortableBuilderEnum(reader);
-        }
-
-        return res;
-    }
-
-    /** {@inheritDoc} */
-    @Override public void writeTo(PortableWriterExImpl writer, PortableBuilderSerializer ctx) {
-        if (val != null) {
-            if (clsName != null)
-                ctx.writeArray(writer, GridPortableMarshaller.ENUM_ARR, (Object[])val, clsName);
-            else
-                ctx.writeArray(writer, GridPortableMarshaller.ENUM_ARR, (Object[])val, compTypeId);
-
-            return;
-        }
-
-        writer.write(reader.array(), valOff, len);
-    }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/9057a4c0/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableLazyArrayList.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableLazyArrayList.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableLazyArrayList.java
deleted file mode 100644
index 84a0c22..0000000
--- a/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableLazyArrayList.java
+++ /dev/null
@@ -1,159 +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;
-
-import java.util.AbstractList;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.List;
-
-/**
- *
- */
-class PortableLazyArrayList extends AbstractList<Object> implements PortableBuilderSerializationAware {
-    /** */
-    private final PortableBuilderReader reader;
-
-    /** */
-    private final int off;
-
-    /** */
-    private List<Object> delegate;
-
-    /**
-     * @param reader Reader.
-     * @param size Size,
-     */
-    PortableLazyArrayList(PortableBuilderReader reader, int size) {
-        this.reader = reader;
-        off = reader.position() - 1/* flag */ - 4/* size */ - 1/* col type */;
-
-        assert size >= 0;
-
-        for (int i = 0; i < size; i++)
-            reader.skipValue();
-    }
-
-    /**
-     *
-     */
-    private void ensureDelegateInit() {
-        if (delegate == null) {
-            int size = reader.readIntAbsolute(off + 1);
-
-            reader.position(off + 1/* flag */ + 4/* size */ + 1/* col type */);
-
-            delegate = new ArrayList<>(size);
-
-            for (int i = 0; i < size; i++)
-                delegate.add(reader.parseValue());
-        }
-    }
-
-    /** {@inheritDoc} */
-    @Override public Object get(int idx) {
-        ensureDelegateInit();
-
-        return PortableUtils.unwrapLazy(delegate.get(idx));
-    }
-
-    /** {@inheritDoc} */
-    @Override public boolean add(Object o) {
-        ensureDelegateInit();
-
-        return delegate.add(o);
-    }
-
-    /** {@inheritDoc} */
-    @Override public void add(int idx, Object element) {
-        ensureDelegateInit();
-
-        delegate.add(idx, element);
-    }
-
-    /** {@inheritDoc} */
-    @Override public Object set(int idx, Object element) {
-        ensureDelegateInit();
-
-        return PortableUtils.unwrapLazy(delegate.set(idx, element));
-    }
-
-    /** {@inheritDoc} */
-    @Override public Object remove(int idx) {
-        ensureDelegateInit();
-
-        return PortableUtils.unwrapLazy(delegate.remove(idx));
-    }
-
-    /** {@inheritDoc} */
-    @Override public void clear() {
-        if (delegate == null)
-            delegate = new ArrayList<>();
-        else
-            delegate.clear();
-    }
-
-    /** {@inheritDoc} */
-    @Override public boolean addAll(int idx, Collection<?> c) {
-        return delegate.addAll(idx, c);
-    }
-
-    /** {@inheritDoc} */
-    @Override protected void removeRange(int fromIdx, int toIdx) {
-        ensureDelegateInit();
-
-        delegate.subList(fromIdx, toIdx).clear();
-    }
-
-    /** {@inheritDoc} */
-    @Override public int size() {
-        if (delegate == null)
-            return reader.readIntAbsolute(off + 1);
-
-        return delegate.size();
-    }
-
-    /** {@inheritDoc} */
-    @Override public void writeTo(PortableWriterExImpl writer, PortableBuilderSerializer ctx) {
-        if (delegate == null) {
-            int size = reader.readIntAbsolute(off + 1);
-
-            int hdrSize = 1 /* flag */ + 4 /* size */ + 1 /* col type */;
-
-            writer.write(reader.array(), off, hdrSize);
-
-            reader.position(off + hdrSize);
-
-            for (int i = 0; i < size; i++) {
-                Object o = reader.parseValue();
-
-                ctx.writeValue(writer, o);
-            }
-        }
-        else {
-            writer.writeByte(GridPortableMarshaller.COL);
-            writer.writeInt(delegate.size());
-
-            byte colType = reader.array()[off + 1 /* flag */ + 4 /* size */];
-            writer.writeByte(colType);
-
-            for (Object o : delegate)
-                ctx.writeValue(writer, o);
-        }
-    }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/9057a4c0/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableLazyLinkedList.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableLazyLinkedList.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableLazyLinkedList.java
deleted file mode 100644
index 7b2e15a..0000000
--- a/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableLazyLinkedList.java
+++ /dev/null
@@ -1,215 +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;
-
-import java.util.AbstractList;
-import java.util.Collection;
-import java.util.Iterator;
-import java.util.LinkedList;
-import java.util.List;
-import java.util.ListIterator;
-
-/**
- *
- */
-class PortableLazyLinkedList extends AbstractList<Object> implements PortableBuilderSerializationAware {
-    /** */
-    private final PortableBuilderReader reader;
-
-    /** */
-    private final int off;
-
-    /** */
-    private List<Object> delegate;
-
-    /**
-     * @param reader Reader.
-     * @param size Size,
-     */
-    PortableLazyLinkedList(PortableBuilderReader reader, int size) {
-        this.reader = reader;
-        off = reader.position() - 1/* flag */ - 4/* size */ - 1/* col type */;
-
-        assert size >= 0;
-
-        for (int i = 0; i < size; i++)
-            reader.skipValue();
-    }
-
-    /**
-     *
-     */
-    private void ensureDelegateInit() {
-        if (delegate == null) {
-            int size = reader.readIntAbsolute(off + 1);
-
-            reader.position(off + 1/* flag */ + 4/* size */ + 1/* col type */);
-
-            delegate = new LinkedList<>();
-
-            for (int i = 0; i < size; i++)
-                delegate.add(reader.parseValue());
-        }
-    }
-
-    /** {@inheritDoc} */
-    @Override public Object get(int idx) {
-        ensureDelegateInit();
-
-        return PortableUtils.unwrapLazy(delegate.get(idx));
-    }
-
-    /** {@inheritDoc} */
-    @Override public boolean add(Object o) {
-        ensureDelegateInit();
-
-        return delegate.add(o);
-    }
-
-    /** {@inheritDoc} */
-    @Override public void add(int idx, Object element) {
-        ensureDelegateInit();
-
-        delegate.add(idx, element);
-    }
-
-    /** {@inheritDoc} */
-    @Override public Object set(int idx, Object element) {
-        ensureDelegateInit();
-
-        return PortableUtils.unwrapLazy(delegate.set(idx, element));
-    }
-
-    /** {@inheritDoc} */
-    @Override public Object remove(int idx) {
-        ensureDelegateInit();
-
-        return PortableUtils.unwrapLazy(delegate.remove(idx));
-    }
-
-    /** {@inheritDoc} */
-    @Override public void clear() {
-        if (delegate == null)
-            delegate = new LinkedList<>();
-        else
-            delegate.clear();
-    }
-
-    /** {@inheritDoc} */
-    @Override public boolean addAll(int idx, Collection<?> c) {
-        ensureDelegateInit();
-
-        return delegate.addAll(idx, c);
-    }
-
-    /** {@inheritDoc} */
-    @Override protected void removeRange(int fromIdx, int toIdx) {
-        ensureDelegateInit();
-
-        delegate.subList(fromIdx, toIdx).clear();
-    }
-
-    /** {@inheritDoc} */
-    @Override public int size() {
-        if (delegate == null)
-            return reader.readIntAbsolute(off + 1);
-
-        return delegate.size();
-    }
-
-    /** {@inheritDoc} */
-    @Override public ListIterator<Object> listIterator(final int idx) {
-        ensureDelegateInit();
-
-        return new ListIterator<Object>() {
-            /** */
-            private final ListIterator<Object> delegate = PortableLazyLinkedList.super.listIterator(idx);
-
-            @Override public boolean hasNext() {
-                return delegate.hasNext();
-            }
-
-            @Override public Object next() {
-                return PortableUtils.unwrapLazy(delegate.next());
-            }
-
-            @Override public boolean hasPrevious() {
-                return delegate.hasPrevious();
-            }
-
-            @Override public Object previous() {
-                return PortableUtils.unwrapLazy(delegate.previous());
-            }
-
-            @Override public int nextIndex() {
-                return delegate.nextIndex();
-            }
-
-            @Override public int previousIndex() {
-                return delegate.previousIndex();
-            }
-
-            @Override public void remove() {
-                delegate.remove();
-            }
-
-            @Override public void set(Object o) {
-                delegate.set(o);
-            }
-
-            @Override public void add(Object o) {
-                delegate.add(o);
-            }
-        };
-    }
-
-    /** {@inheritDoc} */
-    @Override public Iterator<Object> iterator() {
-        ensureDelegateInit();
-
-        return PortableUtils.unwrapLazyIterator(super.iterator());
-    }
-
-    /** {@inheritDoc} */
-    @Override public void writeTo(PortableWriterExImpl writer, PortableBuilderSerializer ctx) {
-        if (delegate == null) {
-            int size = reader.readIntAbsolute(off + 1);
-
-            int hdrSize = 1 /* flag */ + 4 /* size */ + 1 /* col type */;
-            writer.write(reader.array(), off, hdrSize);
-
-            reader.position(off + hdrSize);
-
-            for (int i = 0; i < size; i++) {
-                Object o = reader.parseValue();
-
-                ctx.writeValue(writer, o);
-            }
-        }
-        else {
-            writer.writeByte(GridPortableMarshaller.COL);
-            writer.writeInt(delegate.size());
-
-            byte colType = reader.array()[off + 1 /* flag */ + 4 /* size */];
-            writer.writeByte(colType);
-
-            for (Object o : delegate)
-                ctx.writeValue(writer, o);
-        }
-    }
-}
\ No newline at end of file


[34/39] ignite git commit: ignite-1351: moved portable API examples to Ignite except cross-platform example

Posted by ag...@apache.org.
ignite-1351: moved portable API examples to Ignite except cross-platform example


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

Branch: refs/heads/ignite-264
Commit: 6a5a48a0c9e379ecf83c8a799d0d8e2397fe5b3c
Parents: c5d303b
Author: Denis Magda <dm...@gridgain.com>
Authored: Thu Sep 3 16:59:36 2015 +0300
Committer: Denis Magda <dm...@gridgain.com>
Committed: Thu Sep 3 16:59:36 2015 +0300

----------------------------------------------------------------------
 examples/config/example-default.xml             |  76 +++++
 examples/config/example-ignite.xml              |  56 +---
 .../config/portable/example-ignite-portable.xml |  44 +++
 .../ignite/examples/portable/Address.java       |  72 +++++
 .../ignite/examples/portable/Employee.java      |  93 ++++++
 .../ignite/examples/portable/EmployeeKey.java   |  90 ++++++
 .../portable/ExamplePortableNodeStartup.java    |  36 +++
 .../ignite/examples/portable/Organization.java  |  93 ++++++
 .../examples/portable/OrganizationType.java     |  32 ++
 ...mputeClientPortableTaskExecutionExample.java | 154 +++++++++
 .../portable/computegrid/ComputeClientTask.java | 116 +++++++
 .../portable/computegrid/package-info.java      |  21 ++
 .../CacheClientPortablePutGetExample.java       | 226 +++++++++++++
 .../CacheClientPortableQueryExample.java        | 323 +++++++++++++++++++
 .../portable/datagrid/package-info.java         |  21 ++
 .../ignite/examples/portable/package-info.java  |  21 ++
 .../CacheClientPortableExampleTest.java         |  46 +++
 .../ComputeClientPortableExampleTest.java       |  37 +++
 .../testsuites/IgniteExamplesSelfTestSuite.java |   6 +
 19 files changed, 1513 insertions(+), 50 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/6a5a48a0/examples/config/example-default.xml
----------------------------------------------------------------------
diff --git a/examples/config/example-default.xml b/examples/config/example-default.xml
new file mode 100644
index 0000000..e6c359d
--- /dev/null
+++ b/examples/config/example-default.xml
@@ -0,0 +1,76 @@
+<?xml version="1.0" encoding="UTF-8"?>
+
+<!--
+  Licensed to the Apache Software Foundation (ASF) under one or more
+  contributor license agreements.  See the NOTICE file distributed with
+  this work for additional information regarding copyright ownership.
+  The ASF licenses this file to You under the Apache License, Version 2.0
+  (the "License"); you may not use this file except in compliance with
+  the License.  You may obtain a copy of the License at
+
+       http://www.apache.org/licenses/LICENSE-2.0
+
+  Unless required by applicable law or agreed to in writing, software
+  distributed under the License is distributed on an "AS IS" BASIS,
+  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  See the License for the specific language governing permissions and
+  limitations under the License.
+-->
+
+<!--
+    Ignite configuration with all defaults and enabled p2p deployment and enabled events.
+-->
+<beans xmlns="http://www.springframework.org/schema/beans"
+       xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+       xmlns:util="http://www.springframework.org/schema/util"
+       xsi:schemaLocation="
+        http://www.springframework.org/schema/beans
+        http://www.springframework.org/schema/beans/spring-beans.xsd
+        http://www.springframework.org/schema/util
+        http://www.springframework.org/schema/util/spring-util.xsd">
+    <bean abstract="true" id="ignite.cfg" class="org.apache.ignite.configuration.IgniteConfiguration">
+        <!-- Set to true to enable distributed class loading for examples, default is false. -->
+        <property name="peerClassLoadingEnabled" value="true"/>
+
+        <!-- Enable task execution events for examples. -->
+        <property name="includeEventTypes">
+            <list>
+                <!--Task execution events-->
+                <util:constant static-field="org.apache.ignite.events.EventType.EVT_TASK_STARTED"/>
+                <util:constant static-field="org.apache.ignite.events.EventType.EVT_TASK_FINISHED"/>
+                <util:constant static-field="org.apache.ignite.events.EventType.EVT_TASK_FAILED"/>
+                <util:constant static-field="org.apache.ignite.events.EventType.EVT_TASK_TIMEDOUT"/>
+                <util:constant static-field="org.apache.ignite.events.EventType.EVT_TASK_SESSION_ATTR_SET"/>
+                <util:constant static-field="org.apache.ignite.events.EventType.EVT_TASK_REDUCED"/>
+
+                <!--Cache events-->
+                <util:constant static-field="org.apache.ignite.events.EventType.EVT_CACHE_OBJECT_PUT"/>
+                <util:constant static-field="org.apache.ignite.events.EventType.EVT_CACHE_OBJECT_READ"/>
+                <util:constant static-field="org.apache.ignite.events.EventType.EVT_CACHE_OBJECT_REMOVED"/>
+            </list>
+        </property>
+
+        <!-- Explicitly configure TCP discovery SPI to provide list of initial nodes. -->
+        <property name="discoverySpi">
+            <bean class="org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi">
+                <property name="ipFinder">
+                    <!--
+                        Ignite provides several options for automatic discovery that can be used
+                        instead os static IP based discovery. For information on all options refer
+                        to our documentation: http://apacheignite.readme.io/docs/cluster-config
+                    -->
+                    <!-- Uncomment static IP finder to enable static-based discovery of initial nodes. -->
+                    <!--<bean class="org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder">-->
+                    <bean class="org.apache.ignite.spi.discovery.tcp.ipfinder.multicast.TcpDiscoveryMulticastIpFinder">
+                        <property name="addresses">
+                            <list>
+                                <!-- In distributed environment, replace with actual host IP address. -->
+                                <value>127.0.0.1:47500..47509</value>
+                            </list>
+                        </property>
+                    </bean>
+                </property>
+            </bean>
+        </property>
+    </bean>
+</beans>

http://git-wip-us.apache.org/repos/asf/ignite/blob/6a5a48a0/examples/config/example-ignite.xml
----------------------------------------------------------------------
diff --git a/examples/config/example-ignite.xml b/examples/config/example-ignite.xml
index e7adb54..d842a6d 100644
--- a/examples/config/example-ignite.xml
+++ b/examples/config/example-ignite.xml
@@ -22,62 +22,18 @@
 -->
 <beans xmlns="http://www.springframework.org/schema/beans"
        xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
-       xmlns:util="http://www.springframework.org/schema/util"
-       xsi:schemaLocation="
-        http://www.springframework.org/schema/beans
-        http://www.springframework.org/schema/beans/spring-beans.xsd
-        http://www.springframework.org/schema/util
-        http://www.springframework.org/schema/util/spring-util.xsd">
-    <bean id="ignite.cfg" class="org.apache.ignite.configuration.IgniteConfiguration">
-        <!-- Set to true to enable distributed class loading for examples, default is false. -->
-        <property name="peerClassLoadingEnabled" value="true"/>
+       xsi:schemaLocation="http://www.springframework.org/schema/beans
+        http://www.springframework.org/schema/beans/spring-beans.xsd">
+    <!-- Imports default Ignite configuration -->
+    <import resource="example-default.xml"/>
 
+    <bean parent="ignite.cfg">
+        <!-- Enabled optimized marshaller -->
         <property name="marshaller">
             <bean class="org.apache.ignite.marshaller.optimized.OptimizedMarshaller">
                 <!-- Set to false to allow non-serializable objects in examples, default is true. -->
                 <property name="requireSerializable" value="false"/>
             </bean>
         </property>
-
-        <!-- Enable task execution events for examples. -->
-        <property name="includeEventTypes">
-            <list>
-                <!--Task execution events-->
-                <util:constant static-field="org.apache.ignite.events.EventType.EVT_TASK_STARTED"/>
-                <util:constant static-field="org.apache.ignite.events.EventType.EVT_TASK_FINISHED"/>
-                <util:constant static-field="org.apache.ignite.events.EventType.EVT_TASK_FAILED"/>
-                <util:constant static-field="org.apache.ignite.events.EventType.EVT_TASK_TIMEDOUT"/>
-                <util:constant static-field="org.apache.ignite.events.EventType.EVT_TASK_SESSION_ATTR_SET"/>
-                <util:constant static-field="org.apache.ignite.events.EventType.EVT_TASK_REDUCED"/>
-
-                <!--Cache events-->
-                <util:constant static-field="org.apache.ignite.events.EventType.EVT_CACHE_OBJECT_PUT"/>
-                <util:constant static-field="org.apache.ignite.events.EventType.EVT_CACHE_OBJECT_READ"/>
-                <util:constant static-field="org.apache.ignite.events.EventType.EVT_CACHE_OBJECT_REMOVED"/>
-            </list>
-        </property>
-
-        <!-- Explicitly configure TCP discovery SPI to provide list of initial nodes. -->
-        <property name="discoverySpi">
-            <bean class="org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi">
-                <property name="ipFinder">
-                    <!--
-                        Ignite provides several options for automatic discovery that can be used
-                        instead os static IP based discovery. For information on all options refer
-                        to our documentation: http://apacheignite.readme.io/docs/cluster-config
-                    -->
-                    <!-- Uncomment static IP finder to enable static-based discovery of initial nodes. -->
-                    <!--<bean class="org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder">-->
-                    <bean class="org.apache.ignite.spi.discovery.tcp.ipfinder.multicast.TcpDiscoveryMulticastIpFinder">
-                        <property name="addresses">
-                            <list>
-                                <!-- In distributed environment, replace with actual host IP address. -->
-                                <value>127.0.0.1:47500..47509</value>
-                            </list>
-                        </property>
-                    </bean>
-                </property>
-            </bean>
-        </property>
     </bean>
 </beans>

http://git-wip-us.apache.org/repos/asf/ignite/blob/6a5a48a0/examples/config/portable/example-ignite-portable.xml
----------------------------------------------------------------------
diff --git a/examples/config/portable/example-ignite-portable.xml b/examples/config/portable/example-ignite-portable.xml
new file mode 100644
index 0000000..cde15ea
--- /dev/null
+++ b/examples/config/portable/example-ignite-portable.xml
@@ -0,0 +1,44 @@
+<?xml version="1.0" encoding="UTF-8"?>
+
+<!--
+  Licensed to the Apache Software Foundation (ASF) under one or more
+  contributor license agreements.  See the NOTICE file distributed with
+  this work for additional information regarding copyright ownership.
+  The ASF licenses this file to You under the Apache License, Version 2.0
+  (the "License"); you may not use this file except in compliance with
+  the License.  You may obtain a copy of the License at
+
+       http://www.apache.org/licenses/LICENSE-2.0
+
+  Unless required by applicable law or agreed to in writing, software
+  distributed under the License is distributed on an "AS IS" BASIS,
+  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  See the License for the specific language governing permissions and
+  limitations under the License.
+-->
+
+<!--
+    Ignite configuration with all defaults and enabled p2p deployment, events and portable marshaller.
+
+    Use this configuration file when running HTTP REST examples (see 'examples/rest' folder).
+
+    When starting a standalone node, you need to execute the following command:
+    {IGNITE_HOME}/bin/ignite.{bat|sh} examples/config/portable/example-ignite-portable.xml
+
+    When starting Ignite from Java IDE, pass path to this file to Ignition:
+    Ignition.start("examples/config/portable/example-ignite-portable.xml");
+-->
+<beans xmlns="http://www.springframework.org/schema/beans"
+       xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+       xsi:schemaLocation="http://www.springframework.org/schema/beans
+        http://www.springframework.org/schema/beans/spring-beans.xsd">
+    <!-- Imports default Ignite configuration -->
+    <import resource="../example-default.xml"/>
+
+    <bean parent="ignite.cfg">
+        <!-- Enables portable marshaller -->
+        <property name="marshaller">
+            <bean class="org.apache.ignite.marshaller.portable.PortableMarshaller"/>
+        </property>
+    </bean>
+</beans>

http://git-wip-us.apache.org/repos/asf/ignite/blob/6a5a48a0/examples/src/main/java/org/apache/ignite/examples/portable/Address.java
----------------------------------------------------------------------
diff --git a/examples/src/main/java/org/apache/ignite/examples/portable/Address.java b/examples/src/main/java/org/apache/ignite/examples/portable/Address.java
new file mode 100644
index 0000000..cb08b25
--- /dev/null
+++ b/examples/src/main/java/org/apache/ignite/examples/portable/Address.java
@@ -0,0 +1,72 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.examples.portable;
+
+import org.apache.ignite.portable.PortableException;
+import org.apache.ignite.portable.PortableMarshalAware;
+import org.apache.ignite.portable.PortableReader;
+import org.apache.ignite.portable.PortableWriter;
+
+/**
+ * Employee address.
+ * <p>
+ * This class implements {@link PortableMarshalAware} only for example purposes,
+ * in order to show how to customize serialization and deserialization of
+ * portable objects.
+ */
+public class Address implements PortableMarshalAware {
+    /** Street. */
+    private String street;
+
+    /** ZIP code. */
+    private int zip;
+
+    /**
+     * Required for portable deserialization.
+     */
+    public Address() {
+        // No-op.
+    }
+
+    /**
+     * @param street Street.
+     * @param zip ZIP code.
+     */
+    public Address(String street, int zip) {
+        this.street = street;
+        this.zip = zip;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writePortable(PortableWriter writer) throws PortableException {
+        writer.writeString("street", street);
+        writer.writeInt("zip", zip);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void readPortable(PortableReader reader) throws PortableException {
+        street = reader.readString("street");
+        zip = reader.readInt("zip");
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return "Address [street=" + street +
+            ", zip=" + zip + ']';
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/6a5a48a0/examples/src/main/java/org/apache/ignite/examples/portable/Employee.java
----------------------------------------------------------------------
diff --git a/examples/src/main/java/org/apache/ignite/examples/portable/Employee.java b/examples/src/main/java/org/apache/ignite/examples/portable/Employee.java
new file mode 100644
index 0000000..9614168
--- /dev/null
+++ b/examples/src/main/java/org/apache/ignite/examples/portable/Employee.java
@@ -0,0 +1,93 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.examples.portable;
+
+import java.util.Collection;
+
+/**
+ * This class represents employee object.
+ */
+public class Employee {
+    /** Name. */
+    private String name;
+
+    /** Salary. */
+    private long salary;
+
+    /** Address. */
+    private Address address;
+
+    /** Departments. */
+    private Collection<String> departments;
+
+    /**
+     * Required for portable deserialization.
+     */
+    public Employee() {
+        // No-op.
+    }
+
+    /**
+     * @param name Name.
+     * @param salary Salary.
+     * @param address Address.
+     * @param departments Departments.
+     */
+    public Employee(String name, long salary, Address address, Collection<String> departments) {
+        this.name = name;
+        this.salary = salary;
+        this.address = address;
+        this.departments = departments;
+    }
+
+    /**
+     * @return Name.
+     */
+    public String name() {
+        return name;
+    }
+
+    /**
+     * @return Salary.
+     */
+    public long salary() {
+        return salary;
+    }
+
+    /**
+     * @return Address.
+     */
+    public Address address() {
+        return address;
+    }
+
+    /**
+     * @return Departments.
+     */
+    public Collection<String> departments() {
+        return departments;
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return "Employee [name=" + name +
+            ", salary=" + salary +
+            ", address=" + address +
+            ", departments=" + departments + ']';
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/6a5a48a0/examples/src/main/java/org/apache/ignite/examples/portable/EmployeeKey.java
----------------------------------------------------------------------
diff --git a/examples/src/main/java/org/apache/ignite/examples/portable/EmployeeKey.java b/examples/src/main/java/org/apache/ignite/examples/portable/EmployeeKey.java
new file mode 100644
index 0000000..f322167
--- /dev/null
+++ b/examples/src/main/java/org/apache/ignite/examples/portable/EmployeeKey.java
@@ -0,0 +1,90 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.examples.portable;
+
+/**
+ * This class represents key for employee object.
+ * <p>
+ * Used in query example to collocate employees
+ * with their organizations.
+ */
+public class EmployeeKey {
+    /** ID. */
+    private int id;
+
+    /** Organization ID. */
+    private int organizationId;
+
+    /**
+     * Required for portable deserialization.
+     */
+    public EmployeeKey() {
+        // No-op.
+    }
+
+    /**
+     * @param id ID.
+     * @param organizationId Organization ID.
+     */
+    public EmployeeKey(int id, int organizationId) {
+        this.id = id;
+        this.organizationId = organizationId;
+    }
+
+    /**
+     * @return ID.
+     */
+    public int id() {
+        return id;
+    }
+
+    /**
+     * @return Organization ID.
+     */
+    public int organizationId() {
+        return organizationId;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean equals(Object o) {
+        if (this == o)
+            return true;
+
+        if (o == null || getClass() != o.getClass())
+            return false;
+
+        EmployeeKey key = (EmployeeKey)o;
+
+        return id == key.id && organizationId == key.organizationId;
+    }
+
+    /** {@inheritDoc} */
+    @Override public int hashCode() {
+        int res = id;
+
+        res = 31 * res + organizationId;
+
+        return res;
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return "EmployeeKey [id=" + id +
+            ", organizationId=" + organizationId + ']';
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/6a5a48a0/examples/src/main/java/org/apache/ignite/examples/portable/ExamplePortableNodeStartup.java
----------------------------------------------------------------------
diff --git a/examples/src/main/java/org/apache/ignite/examples/portable/ExamplePortableNodeStartup.java b/examples/src/main/java/org/apache/ignite/examples/portable/ExamplePortableNodeStartup.java
new file mode 100644
index 0000000..87a41f7
--- /dev/null
+++ b/examples/src/main/java/org/apache/ignite/examples/portable/ExamplePortableNodeStartup.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.examples.portable;
+
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.Ignition;
+
+/**
+ * Starts up an empty node with example configuration and portable marshaller enabled.
+ */
+public class ExamplePortableNodeStartup {
+    /**
+     * Start up an empty node with example configuration and portable marshaller enabled.
+     *
+     * @param args Command line arguments, none required.
+     * @throws IgniteException If failed.
+     */
+    public static void main(String[] args) throws IgniteException {
+        Ignition.start("examples/config/portable/example-ignite-portable.xml");
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/6a5a48a0/examples/src/main/java/org/apache/ignite/examples/portable/Organization.java
----------------------------------------------------------------------
diff --git a/examples/src/main/java/org/apache/ignite/examples/portable/Organization.java b/examples/src/main/java/org/apache/ignite/examples/portable/Organization.java
new file mode 100644
index 0000000..f52cac1
--- /dev/null
+++ b/examples/src/main/java/org/apache/ignite/examples/portable/Organization.java
@@ -0,0 +1,93 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.examples.portable;
+
+import java.sql.Timestamp;
+
+/**
+ * This class represents organization object.
+ */
+public class Organization {
+    /** Name. */
+    private String name;
+
+    /** Address. */
+    private Address address;
+
+    /** Type. */
+    private OrganizationType type;
+
+    /** Last update time. */
+    private Timestamp lastUpdated;
+
+    /**
+     * Required for portable deserialization.
+     */
+    public Organization() {
+        // No-op.
+    }
+
+    /**
+     * @param name Name.
+     * @param address Address.
+     * @param type Type.
+     * @param lastUpdated Last update time.
+     */
+    public Organization(String name, Address address, OrganizationType type, Timestamp lastUpdated) {
+        this.name = name;
+        this.address = address;
+        this.type = type;
+        this.lastUpdated = lastUpdated;
+    }
+
+    /**
+     * @return Name.
+     */
+    public String name() {
+        return name;
+    }
+
+    /**
+     * @return Address.
+     */
+    public Address address() {
+        return address;
+    }
+
+    /**
+     * @return Type.
+     */
+    public OrganizationType type() {
+        return type;
+    }
+
+    /**
+     * @return Last update time.
+     */
+    public Timestamp lastUpdated() {
+        return lastUpdated;
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return "Organization [name=" + name +
+            ", address=" + address +
+            ", type=" + type +
+            ", lastUpdated=" + lastUpdated + ']';
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/6a5a48a0/examples/src/main/java/org/apache/ignite/examples/portable/OrganizationType.java
----------------------------------------------------------------------
diff --git a/examples/src/main/java/org/apache/ignite/examples/portable/OrganizationType.java b/examples/src/main/java/org/apache/ignite/examples/portable/OrganizationType.java
new file mode 100644
index 0000000..c753e2d
--- /dev/null
+++ b/examples/src/main/java/org/apache/ignite/examples/portable/OrganizationType.java
@@ -0,0 +1,32 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.examples.portable;
+
+/**
+ * Organization type enum.
+ */
+public enum OrganizationType {
+    /** Non-profit organization. */
+    NON_PROFIT,
+
+    /** Private organization. */
+    PRIVATE,
+
+    /** Government organization. */
+    GOVERNMENT
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/6a5a48a0/examples/src/main/java/org/apache/ignite/examples/portable/computegrid/ComputeClientPortableTaskExecutionExample.java
----------------------------------------------------------------------
diff --git a/examples/src/main/java/org/apache/ignite/examples/portable/computegrid/ComputeClientPortableTaskExecutionExample.java b/examples/src/main/java/org/apache/ignite/examples/portable/computegrid/ComputeClientPortableTaskExecutionExample.java
new file mode 100644
index 0000000..34d9cde
--- /dev/null
+++ b/examples/src/main/java/org/apache/ignite/examples/portable/computegrid/ComputeClientPortableTaskExecutionExample.java
@@ -0,0 +1,154 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.examples.portable.computegrid;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import org.apache.ignite.Ignite;
+import org.apache.ignite.Ignition;
+import org.apache.ignite.examples.portable.Address;
+import org.apache.ignite.examples.portable.Employee;
+import org.apache.ignite.examples.portable.ExamplePortableNodeStartup;
+import org.apache.ignite.portable.PortableObject;
+
+/**
+ * This example demonstrates use of portable objects with task execution.
+ * Specifically it shows that portable objects are simple Java POJOs and do not require any special treatment.
+ * <p>
+ * The example executes map-reduce task that accepts collection of portable objects as an argument.
+ * 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 portable marshaller: {@code 'ignite.{sh|bat} examples/config/portable/example-ignite-portable.xml'}.
+ * <p>
+ * Alternatively you can run {@link ExamplePortableNodeStartup} in another JVM which will
+ * start node with {@code examples/config/portable/example-ignite-portable.xml} configuration.
+ */
+public class ComputeClientPortableTaskExecutionExample {
+    /**
+     * Executes example.
+     *
+     * @param args Command line arguments, none required.
+     */
+    public static void main(String[] args) {
+        try (Ignite ignite = Ignition.start("examples/config/portable/example-ignite-portable.xml")) {
+            System.out.println();
+            System.out.println(">>> Portable objects task execution example started.");
+
+            if (ignite.cluster().forRemotes().nodes().isEmpty()) {
+                System.out.println();
+                System.out.println(">>> This example requires remote nodes to be started.");
+                System.out.println(">>> Please start at least 1 remote node.");
+                System.out.println(">>> Refer to example's javadoc for details on configuration.");
+                System.out.println();
+
+                return;
+            }
+
+            // Generate employees to calculate average salary for.
+            Collection<Employee> employees = employees();
+
+            System.out.println();
+            System.out.println(">>> Calculating average salary for employees:");
+
+            for (Employee employee : employees)
+                System.out.println(">>>     " + employee);
+
+            // Convert collection of employees to collection of portable objects.
+            // This allows to send objects across nodes without requiring to have
+            // Employee class on classpath of these nodes.
+            Collection<PortableObject> portables = ignite.portables().toPortable(employees);
+
+            // Execute task and get average salary.
+            Long avgSalary = ignite.compute(ignite.cluster().forRemotes()).execute(new ComputeClientTask(), portables);
+
+            System.out.println();
+            System.out.println(">>> Average salary for all employees: " + avgSalary);
+            System.out.println();
+        }
+    }
+
+    /**
+     * Creates collection of employees.
+     *
+     * @return Collection of employees.
+     */
+    private static Collection<Employee> employees() {
+        Collection<Employee> employees = new ArrayList<>();
+
+        employees.add(new Employee(
+            "James Wilson",
+            12500,
+            new Address("1096 Eddy Street, San Francisco, CA", 94109),
+            Arrays.asList("Human Resources", "Customer Service")
+        ));
+
+        employees.add(new Employee(
+            "Daniel Adams",
+            11000,
+            new Address("184 Fidler Drive, San Antonio, TX", 78205),
+            Arrays.asList("Development", "QA")
+        ));
+
+        employees.add(new Employee(
+            "Cristian Moss",
+            12500,
+            new Address("667 Jerry Dove Drive, Florence, SC", 29501),
+            Arrays.asList("Logistics")
+        ));
+
+        employees.add(new Employee(
+            "Allison Mathis",
+            25300,
+            new Address("2702 Freedom Lane, Hornitos, CA", 95325),
+            Arrays.asList("Development")
+        ));
+
+        employees.add(new Employee(
+            "Breana Robbin",
+            6500,
+            new Address("3960 Sundown Lane, Austin, TX", 78758),
+            Arrays.asList("Sales")
+        ));
+
+        employees.add(new Employee(
+            "Philip Horsley",
+            19800,
+            new Address("2803 Elsie Drive, Sioux Falls, SD", 57104),
+            Arrays.asList("Sales")
+        ));
+
+        employees.add(new Employee(
+            "Brian Peters",
+            10600,
+            new Address("1407 Pearlman Avenue, Boston, MA", 12110),
+            Arrays.asList("Development", "QA")
+        ));
+
+        employees.add(new Employee(
+            "Jack Yang",
+            12900,
+            new Address("4425 Parrish Avenue Smithsons Valley, TX", 78130),
+            Arrays.asList("Sales")
+        ));
+
+        return employees;
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/6a5a48a0/examples/src/main/java/org/apache/ignite/examples/portable/computegrid/ComputeClientTask.java
----------------------------------------------------------------------
diff --git a/examples/src/main/java/org/apache/ignite/examples/portable/computegrid/ComputeClientTask.java b/examples/src/main/java/org/apache/ignite/examples/portable/computegrid/ComputeClientTask.java
new file mode 100644
index 0000000..0eee8c6
--- /dev/null
+++ b/examples/src/main/java/org/apache/ignite/examples/portable/computegrid/ComputeClientTask.java
@@ -0,0 +1,116 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.examples.portable.computegrid;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+import org.apache.ignite.compute.ComputeJob;
+import org.apache.ignite.compute.ComputeJobAdapter;
+import org.apache.ignite.compute.ComputeJobResult;
+import org.apache.ignite.compute.ComputeTaskSplitAdapter;
+import org.apache.ignite.lang.IgniteBiTuple;
+import org.apache.ignite.portable.PortableObject;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Task that is used for {@link ComputeClientPortableTaskExecutionExample} and
+ * similar examples in .NET and C++.
+ * <p>
+ * This task calculates average salary for provided collection of employees.
+ * It splits the collection into batches of size {@code 3} and creates a job
+ * for each batch. After all jobs are executed, there results are reduced to
+ * get the average salary.
+ */
+public class ComputeClientTask extends ComputeTaskSplitAdapter<Collection<PortableObject>, Long> {
+    /** {@inheritDoc} */
+    @Override protected Collection<? extends ComputeJob> split(
+        int gridSize,
+        Collection<PortableObject> arg
+    ) {
+        Collection<ComputeClientJob> jobs = new ArrayList<>();
+
+        Collection<PortableObject> employees = new ArrayList<>();
+
+        // Split provided collection into batches and
+        // create a job for each batch.
+        for (PortableObject employee : arg) {
+            employees.add(employee);
+
+            if (employees.size() == 3) {
+                jobs.add(new ComputeClientJob(employees));
+
+                employees = new ArrayList<>(3);
+            }
+        }
+
+        if (!employees.isEmpty())
+            jobs.add(new ComputeClientJob(employees));
+
+        return jobs;
+    }
+
+    /** {@inheritDoc} */
+    @Nullable @Override public Long reduce(List<ComputeJobResult> results) {
+        long sum = 0;
+        int cnt = 0;
+
+        for (ComputeJobResult res : results) {
+            IgniteBiTuple<Long, Integer> t = res.getData();
+
+            sum += t.get1();
+            cnt += t.get2();
+        }
+
+        return sum / cnt;
+    }
+
+    /**
+     * Remote job for {@link ComputeClientTask}.
+     */
+    private static class ComputeClientJob extends ComputeJobAdapter {
+        /** Collection of employees. */
+        private final Collection<PortableObject> employees;
+
+        /**
+         * @param employees Collection of employees.
+         */
+        private ComputeClientJob(Collection<PortableObject> employees) {
+            this.employees = employees;
+        }
+
+        /** {@inheritDoc} */
+        @Nullable @Override public Object execute() {
+            long sum = 0;
+            int cnt = 0;
+
+            for (PortableObject employee : employees) {
+                System.out.println(">>> Processing employee: " + employee.field("name"));
+
+                // Get salary from portable object. Note that object
+                // doesn't need to be fully deserialized.
+                long salary = employee.field("salary");
+
+                sum += salary;
+                cnt++;
+            }
+
+            return new IgniteBiTuple<>(sum, cnt);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/6a5a48a0/examples/src/main/java/org/apache/ignite/examples/portable/computegrid/package-info.java
----------------------------------------------------------------------
diff --git a/examples/src/main/java/org/apache/ignite/examples/portable/computegrid/package-info.java b/examples/src/main/java/org/apache/ignite/examples/portable/computegrid/package-info.java
new file mode 100644
index 0000000..469128c
--- /dev/null
+++ b/examples/src/main/java/org/apache/ignite/examples/portable/computegrid/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.
+ */
+
+/**
+ * Demonstrates the usage of portable objects with task execution.
+ */
+package org.apache.ignite.examples.portable.computegrid;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/6a5a48a0/examples/src/main/java/org/apache/ignite/examples/portable/datagrid/CacheClientPortablePutGetExample.java
----------------------------------------------------------------------
diff --git a/examples/src/main/java/org/apache/ignite/examples/portable/datagrid/CacheClientPortablePutGetExample.java b/examples/src/main/java/org/apache/ignite/examples/portable/datagrid/CacheClientPortablePutGetExample.java
new file mode 100644
index 0000000..19c5685
--- /dev/null
+++ b/examples/src/main/java/org/apache/ignite/examples/portable/datagrid/CacheClientPortablePutGetExample.java
@@ -0,0 +1,226 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.examples.portable.datagrid;
+
+import java.sql.Timestamp;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.Map;
+import org.apache.ignite.Ignite;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.Ignition;
+import org.apache.ignite.cache.CacheAtomicityMode;
+import org.apache.ignite.cache.CacheMode;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.examples.portable.Address;
+import org.apache.ignite.examples.portable.ExamplePortableNodeStartup;
+import org.apache.ignite.examples.portable.Organization;
+import org.apache.ignite.examples.portable.OrganizationType;
+import org.apache.ignite.portable.PortableObject;
+
+/**
+ * This example demonstrates use of portable objects with Ignite cache.
+ * Specifically it shows that portable objects are simple Java POJOs and do not require any special treatment.
+ * <p>
+ * The example executes several put-get operations on Ignite cache with portable values. Note that
+ * it demonstrates how portable object can be retrieved in fully-deserialized form or in portable object
+ * format using special cache projection.
+ * <p>
+ * Remote nodes should always be started with special configuration file which
+ * enables the portable marshaller: {@code 'ignite.{sh|bat} examples/config/portable/example-ignite-portable.xml'}.
+ * <p>
+ * Alternatively you can run {@link ExamplePortableNodeStartup} in another JVM which will
+ * start node with {@code examples/config/portable/example-ignite-portable.xml} configuration.
+ */
+public class CacheClientPortablePutGetExample {
+    /** Cache name. */
+    private static final String CACHE_NAME = CacheClientPortablePutGetExample.class.getSimpleName();
+
+    /**
+     * Executes example.
+     *
+     * @param args Command line arguments, none required.
+     */
+    public static void main(String[] args) {
+        try (Ignite ignite = Ignition.start("examples/config/portable/example-ignite-portable.xml")) {
+            System.out.println();
+            System.out.println(">>> Portable objects cache put-get example started.");
+
+            CacheConfiguration<Integer, Organization> cfg = new CacheConfiguration<>();
+
+            cfg.setCacheMode(CacheMode.PARTITIONED);
+            cfg.setName(CACHE_NAME);
+            cfg.setAtomicityMode(CacheAtomicityMode.ATOMIC);
+
+            try (IgniteCache<Integer, Organization> cache = ignite.createCache(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.");
+                    System.out.println(">>> Please start at least 1 remote cache node.");
+                    System.out.println(">>> Refer to example's javadoc for details on configuration.");
+                    System.out.println();
+
+                    return;
+                }
+
+                putGet(cache);
+                putGetPortable(cache);
+                putGetAll(cache);
+                putGetAllPortable(cache);
+
+                System.out.println();
+            }
+        }
+    }
+
+    /**
+     * Execute individual put and get.
+     *
+     * @param cache Cache.
+     */
+    private static void putGet(IgniteCache<Integer, Organization> cache) {
+        // Create new Organization portable object to store in cache.
+        Organization org = new Organization(
+            "Microsoft", // Name.
+            new Address("1096 Eddy Street, San Francisco, CA", 94109), // Address.
+            OrganizationType.PRIVATE, // Type.
+            new Timestamp(System.currentTimeMillis())); // Last update time.
+
+        // Put created data entry to cache.
+        cache.put(1, org);
+
+        // Get recently created organization as a strongly-typed fully de-serialized instance.
+        Organization orgFromCache = cache.get(1);
+
+        System.out.println();
+        System.out.println(">>> Retrieved organization instance from cache: " + orgFromCache);
+    }
+
+    /**
+     * Execute individual put and get, getting value in portable format, without de-serializing it.
+     *
+     * @param cache Cache.
+     */
+    private static void putGetPortable(IgniteCache<Integer, Organization> cache) {
+        // Create new Organization portable object to store in cache.
+        Organization org = new Organization(
+            "Microsoft", // Name.
+            new Address("1096 Eddy Street, San Francisco, CA", 94109), // Address.
+            OrganizationType.PRIVATE, // Type.
+            new Timestamp(System.currentTimeMillis())); // Last update time.
+
+        // Put created data entry to cache.
+        cache.put(1, org);
+
+        // Get cache that will get values as portable objects.
+        IgniteCache<Integer, PortableObject> portableCache = cache.withKeepPortable();
+
+        // Get recently created organization as a portable object.
+        PortableObject po = portableCache.get(1);
+
+        // Get organization's name from portable object (note that
+        // object doesn't need to be fully deserialized).
+        String name = po.field("name");
+
+        System.out.println();
+        System.out.println(">>> Retrieved organization name from portable object: " + name);
+    }
+
+    /**
+     * Execute bulk {@code putAll(...)} and {@code getAll(...)} operations.
+     *
+     * @param cache Cache.
+     */
+    private static void putGetAll(IgniteCache<Integer, Organization> cache) {
+        // Create new Organization portable objects to store in cache.
+        Organization org1 = new Organization(
+            "Microsoft", // Name.
+            new Address("1096 Eddy Street, San Francisco, CA", 94109), // Address.
+            OrganizationType.PRIVATE, // Type.
+            new Timestamp(System.currentTimeMillis())); // Last update time.
+
+        Organization org2 = new Organization(
+            "Red Cross", // Name.
+            new Address("184 Fidler Drive, San Antonio, TX", 78205), // Address.
+            OrganizationType.NON_PROFIT, // Type.
+            new Timestamp(System.currentTimeMillis())); // Last update time.
+
+        Map<Integer, Organization> map = new HashMap<>();
+
+        map.put(1, org1);
+        map.put(2, org2);
+
+        // Put created data entries to cache.
+        cache.putAll(map);
+
+        // Get recently created organizations as a strongly-typed fully de-serialized instances.
+        Map<Integer, Organization> mapFromCache = cache.getAll(map.keySet());
+
+        System.out.println();
+        System.out.println(">>> Retrieved organization instances from cache:");
+
+        for (Organization org : mapFromCache.values())
+            System.out.println(">>>     " + org);
+    }
+
+    /**
+     * Execute bulk {@code putAll(...)} and {@code getAll(...)} operations,
+     * getting values in portable format, without de-serializing it.
+     *
+     * @param cache Cache.
+     */
+    private static void putGetAllPortable(IgniteCache<Integer, Organization> cache) {
+        // Create new Organization portable objects to store in cache.
+        Organization org1 = new Organization(
+            "Microsoft", // Name.
+            new Address("1096 Eddy Street, San Francisco, CA", 94109), // Address.
+            OrganizationType.PRIVATE, // Type.
+            new Timestamp(System.currentTimeMillis())); // Last update time.
+
+        Organization org2 = new Organization(
+            "Red Cross", // Name.
+            new Address("184 Fidler Drive, San Antonio, TX", 78205), // Address.
+            OrganizationType.NON_PROFIT, // Type.
+            new Timestamp(System.currentTimeMillis())); // Last update time.
+
+        Map<Integer, Organization> map = new HashMap<>();
+
+        map.put(1, org1);
+        map.put(2, org2);
+
+        // Put created data entries to cache.
+        cache.putAll(map);
+
+        // Get cache that will get values as portable objects.
+        IgniteCache<Integer, PortableObject> portableCache = cache.withKeepPortable();
+
+        // Get recently created organizations as portable objects.
+        Map<Integer, PortableObject> poMap = portableCache.getAll(map.keySet());
+
+        Collection<String> names = new ArrayList<>();
+
+        // Get organizations' names from portable objects (note that
+        // objects don't need to be fully deserialized).
+        for (PortableObject po : poMap.values())
+            names.add(po.<String>field("name"));
+
+        System.out.println();
+        System.out.println(">>> Retrieved organization names from portable objects: " + names);
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/6a5a48a0/examples/src/main/java/org/apache/ignite/examples/portable/datagrid/CacheClientPortableQueryExample.java
----------------------------------------------------------------------
diff --git a/examples/src/main/java/org/apache/ignite/examples/portable/datagrid/CacheClientPortableQueryExample.java b/examples/src/main/java/org/apache/ignite/examples/portable/datagrid/CacheClientPortableQueryExample.java
new file mode 100644
index 0000000..1eb43b3
--- /dev/null
+++ b/examples/src/main/java/org/apache/ignite/examples/portable/datagrid/CacheClientPortableQueryExample.java
@@ -0,0 +1,323 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.examples.portable.datagrid;
+
+import java.sql.Timestamp;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import javax.cache.Cache;
+import org.apache.ignite.Ignite;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.Ignition;
+import org.apache.ignite.cache.CacheMode;
+import org.apache.ignite.cache.CacheTypeMetadata;
+import org.apache.ignite.cache.query.QueryCursor;
+import org.apache.ignite.cache.query.SqlFieldsQuery;
+import org.apache.ignite.cache.query.SqlQuery;
+import org.apache.ignite.cache.query.TextQuery;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.examples.portable.Address;
+import org.apache.ignite.examples.portable.Employee;
+import org.apache.ignite.examples.portable.EmployeeKey;
+import org.apache.ignite.examples.portable.ExamplePortableNodeStartup;
+import org.apache.ignite.examples.portable.Organization;
+import org.apache.ignite.examples.portable.OrganizationType;
+import org.apache.ignite.portable.PortableObject;
+
+/**
+ * This example demonstrates use of portable objects with cache queries.
+ * The example populates cache with sample data and runs several SQL and full text queries over this data.
+ * <p>
+ * Remote nodes should always be started with special configuration file which
+ * enables the portable marshaller: {@code 'ignite.{sh|bat} examples/config/portable/example-ignite-portable.xml'}.
+ * <p>
+ * Alternatively you can run {@link ExamplePortableNodeStartup} in another JVM which will
+ * start node with {@code examples/config/portable/example-ignite-portable.xml} configuration.
+ */
+public class CacheClientPortableQueryExample {
+    /** Organization cache name. */
+    private static final String ORGANIZATION_CACHE_NAME = CacheClientPortableQueryExample.class.getSimpleName()
+        + "Organizations";
+
+    /** Employee cache name. */
+    private static final String EMPLOYEE_CACHE_NAME = CacheClientPortableQueryExample.class.getSimpleName()
+        + "Employees";
+
+    /**
+     * Executes example.
+     *
+     * @param args Command line arguments, none required.
+     */
+    public static void main(String[] args) {
+        try (Ignite ignite = Ignition.start("examples/config/portable/example-ignite-portable.xml")) {
+            System.out.println();
+            System.out.println(">>> Portable objects cache query example started.");
+
+            CacheConfiguration<Integer, Organization> orgCacheCfg = new CacheConfiguration<>();
+
+            orgCacheCfg.setCacheMode(CacheMode.PARTITIONED);
+            orgCacheCfg.setName(ORGANIZATION_CACHE_NAME);
+
+            orgCacheCfg.setTypeMetadata(Arrays.asList(createOrganizationTypeMetadata()));
+
+            CacheConfiguration<EmployeeKey, Employee> employeeCacheCfg = new CacheConfiguration<>();
+
+            employeeCacheCfg.setCacheMode(CacheMode.PARTITIONED);
+            employeeCacheCfg.setName(EMPLOYEE_CACHE_NAME);
+
+            employeeCacheCfg.setTypeMetadata(Arrays.asList(createEmployeeTypeMetadata()));
+
+            try (IgniteCache<Integer, Organization> orgCache = ignite.createCache(orgCacheCfg);
+                 IgniteCache<EmployeeKey, Employee> employeeCache = ignite.createCache(employeeCacheCfg)
+            ) {
+                if (ignite.cluster().forDataNodes(orgCache.getName()).nodes().isEmpty()) {
+                    System.out.println();
+                    System.out.println(">>> This example requires remote cache nodes to be started.");
+                    System.out.println(">>> Please start at least 1 remote cache node.");
+                    System.out.println(">>> Refer to example's javadoc for details on configuration.");
+                    System.out.println();
+
+                    return;
+                }
+
+                // Populate cache with sample data entries.
+                populateCache(orgCache, employeeCache);
+
+                // Get cache that will work with portable objects.
+                IgniteCache<PortableObject, PortableObject> portableCache = employeeCache.withKeepPortable();
+
+                // Run SQL query example.
+                sqlQuery(portableCache);
+
+                // Run SQL query with join example.
+                sqlJoinQuery(portableCache);
+
+                // Run SQL fields query example.
+                sqlFieldsQuery(portableCache);
+
+                // Run full text query example.
+                textQuery(portableCache);
+
+                System.out.println();
+            }
+        }
+    }
+
+    /**
+     * Create cache type metadata for {@link Employee}.
+     *
+     * @return Cache type metadata.
+     */
+    private static CacheTypeMetadata createEmployeeTypeMetadata() {
+        CacheTypeMetadata employeeTypeMeta = new CacheTypeMetadata();
+
+        employeeTypeMeta.setValueType(Employee.class);
+
+        employeeTypeMeta.setKeyType(EmployeeKey.class);
+
+        Map<String, Class<?>> ascFields = new HashMap<>();
+
+        ascFields.put("name", String.class);
+        ascFields.put("salary", Long.class);
+        ascFields.put("address.zip", Integer.class);
+        ascFields.put("organizationId", Integer.class);
+
+        employeeTypeMeta.setAscendingFields(ascFields);
+
+        employeeTypeMeta.setTextFields(Arrays.asList("address.street"));
+
+        return employeeTypeMeta;
+    }
+
+    /**
+     * Create cache type metadata for {@link Organization}.
+     *
+     * @return Cache type metadata.
+     */
+    private static CacheTypeMetadata createOrganizationTypeMetadata() {
+        CacheTypeMetadata organizationTypeMeta = new CacheTypeMetadata();
+
+        organizationTypeMeta.setValueType(Organization.class);
+
+        organizationTypeMeta.setKeyType(Integer.class);
+
+        Map<String, Class<?>> ascFields = new HashMap<>();
+
+        ascFields.put("name", String.class);
+
+        Map<String, Class<?>> queryFields = new HashMap<>();
+
+        queryFields.put("address.street", String.class);
+
+        organizationTypeMeta.setAscendingFields(ascFields);
+
+        organizationTypeMeta.setQueryFields(queryFields);
+
+        return organizationTypeMeta;
+    }
+
+    /**
+     * Queries employees that have provided ZIP code in address.
+     *
+     * @param cache Ignite cache.
+     */
+    private static void sqlQuery(IgniteCache<PortableObject, PortableObject> cache) {
+        SqlQuery<PortableObject, PortableObject> query = new SqlQuery<>(Employee.class, "zip = ?");
+
+        int zip = 94109;
+
+        QueryCursor<Cache.Entry<PortableObject, PortableObject>> employees = cache.query(query.setArgs(zip));
+
+        System.out.println();
+        System.out.println(">>> Employees with zip " + zip + ':');
+
+        for (Cache.Entry<PortableObject, PortableObject> e : employees.getAll())
+            System.out.println(">>>     " + e.getValue().deserialize());
+    }
+
+    /**
+     * Queries employees that work for organization with provided name.
+     *
+     * @param cache Ignite cache.
+     */
+    private static void sqlJoinQuery(IgniteCache<PortableObject, PortableObject> cache) {
+        SqlQuery<PortableObject, PortableObject> query = new SqlQuery<>(Employee.class,
+            "from Employee, \"" + ORGANIZATION_CACHE_NAME + "\".Organization as org " +
+                "where Employee.organizationId = org._key and org.name = ?");
+
+        String organizationName = "GridGain";
+
+        QueryCursor<Cache.Entry<PortableObject, PortableObject>> employees =
+            cache.query(query.setArgs(organizationName));
+
+        System.out.println();
+        System.out.println(">>> Employees working for " + organizationName + ':');
+
+        for (Cache.Entry<PortableObject, PortableObject> e : employees.getAll())
+            System.out.println(">>>     " + e.getValue());
+    }
+
+    /**
+     * Queries names and salaries for all employees.
+     *
+     * @param cache Ignite cache.
+     */
+    private static void sqlFieldsQuery(IgniteCache<PortableObject, PortableObject> cache) {
+        SqlFieldsQuery query = new SqlFieldsQuery("select name, salary from Employee");
+
+        QueryCursor<List<?>> employees = cache.query(query);
+
+        System.out.println();
+        System.out.println(">>> Employee names and their salaries:");
+
+        for (List<?> row : employees.getAll())
+            System.out.println(">>>     [Name=" + row.get(0) + ", salary=" + row.get(1) + ']');
+    }
+
+    /**
+     * Queries employees that live in Texas using full-text query API.
+     *
+     * @param cache Ignite cache.
+     */
+    private static void textQuery(IgniteCache<PortableObject, PortableObject> cache) {
+        TextQuery<PortableObject, PortableObject> query = new TextQuery<>(Employee.class, "TX");
+
+        QueryCursor<Cache.Entry<PortableObject, PortableObject>> employees = cache.query(query);
+
+        System.out.println();
+        System.out.println(">>> Employees living in Texas:");
+
+        for (Cache.Entry<PortableObject, PortableObject> e : employees.getAll())
+            System.out.println(">>>     " + e.getValue().deserialize());
+    }
+
+    /**
+     * Populates cache with data.
+     *
+     * @param orgCache Organization cache.
+     * @param employeeCache Employee cache.
+     */
+    private static void populateCache(IgniteCache<Integer, Organization> orgCache,
+        IgniteCache<EmployeeKey, Employee> employeeCache) {
+        orgCache.put(1, new Organization(
+            "GridGain",
+            new Address("1065 East Hillsdale Blvd, Foster City, CA", 94404),
+            OrganizationType.PRIVATE,
+            new Timestamp(System.currentTimeMillis())
+        ));
+
+        orgCache.put(2, new Organization(
+            "Microsoft",
+            new Address("1096 Eddy Street, San Francisco, CA", 94109),
+            OrganizationType.PRIVATE,
+            new Timestamp(System.currentTimeMillis())
+        ));
+
+        employeeCache.put(new EmployeeKey(1, 1), new Employee(
+            "James Wilson",
+            12500,
+            new Address("1096 Eddy Street, San Francisco, CA", 94109),
+            Arrays.asList("Human Resources", "Customer Service")
+        ));
+
+        employeeCache.put(new EmployeeKey(2, 1), new Employee(
+            "Daniel Adams",
+            11000,
+            new Address("184 Fidler Drive, San Antonio, TX", 78130),
+            Arrays.asList("Development", "QA")
+        ));
+
+        employeeCache.put(new EmployeeKey(3, 1), new Employee(
+            "Cristian Moss",
+            12500,
+            new Address("667 Jerry Dove Drive, Florence, SC", 29501),
+            Arrays.asList("Logistics")
+        ));
+
+        employeeCache.put(new EmployeeKey(4, 2), new Employee(
+            "Allison Mathis",
+            25300,
+            new Address("2702 Freedom Lane, San Francisco, CA", 94109),
+            Arrays.asList("Development")
+        ));
+
+        employeeCache.put(new EmployeeKey(5, 2), new Employee(
+            "Breana Robbin",
+            6500,
+            new Address("3960 Sundown Lane, Austin, TX", 78130),
+            Arrays.asList("Sales")
+        ));
+
+        employeeCache.put(new EmployeeKey(6, 2), new Employee(
+            "Philip Horsley",
+            19800,
+            new Address("2803 Elsie Drive, Sioux Falls, SD", 57104),
+            Arrays.asList("Sales")
+        ));
+
+        employeeCache.put(new EmployeeKey(7, 2), new Employee(
+            "Brian Peters",
+            10600,
+            new Address("1407 Pearlman Avenue, Boston, MA", 12110),
+            Arrays.asList("Development", "QA")
+        ));
+    }
+}
+

http://git-wip-us.apache.org/repos/asf/ignite/blob/6a5a48a0/examples/src/main/java/org/apache/ignite/examples/portable/datagrid/package-info.java
----------------------------------------------------------------------
diff --git a/examples/src/main/java/org/apache/ignite/examples/portable/datagrid/package-info.java b/examples/src/main/java/org/apache/ignite/examples/portable/datagrid/package-info.java
new file mode 100644
index 0000000..b24f233
--- /dev/null
+++ b/examples/src/main/java/org/apache/ignite/examples/portable/datagrid/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.
+ */
+
+/**
+ * Demonstrates the usage of portable objects with cache.
+ */
+package org.apache.ignite.examples.portable.datagrid;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/6a5a48a0/examples/src/main/java/org/apache/ignite/examples/portable/package-info.java
----------------------------------------------------------------------
diff --git a/examples/src/main/java/org/apache/ignite/examples/portable/package-info.java b/examples/src/main/java/org/apache/ignite/examples/portable/package-info.java
new file mode 100644
index 0000000..4301027
--- /dev/null
+++ b/examples/src/main/java/org/apache/ignite/examples/portable/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 portable classes and examples.
+ */
+package org.apache.ignite.examples.portable;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/6a5a48a0/examples/src/test/java/org/apache/ignite/examples/CacheClientPortableExampleTest.java
----------------------------------------------------------------------
diff --git a/examples/src/test/java/org/apache/ignite/examples/CacheClientPortableExampleTest.java b/examples/src/test/java/org/apache/ignite/examples/CacheClientPortableExampleTest.java
new file mode 100644
index 0000000..6ea1484
--- /dev/null
+++ b/examples/src/test/java/org/apache/ignite/examples/CacheClientPortableExampleTest.java
@@ -0,0 +1,46 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.examples;
+
+import org.apache.ignite.examples.portable.datagrid.CacheClientPortablePutGetExample;
+import org.apache.ignite.examples.portable.datagrid.CacheClientPortableQueryExample;
+import org.apache.ignite.testframework.junits.common.GridAbstractExamplesTest;
+
+/**
+ *
+ */
+public class CacheClientPortableExampleTest extends GridAbstractExamplesTest {
+    /** {@inheritDoc} */
+    @Override protected String defaultConfig() {
+        return "examples/config/portable/example-ignite-portable.xml";
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testPortablePutGetExample() throws Exception {
+        CacheClientPortablePutGetExample.main(new String[] {});
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testPortableQueryExample() throws Exception {
+        CacheClientPortableQueryExample.main(new String[] {});
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/6a5a48a0/examples/src/test/java/org/apache/ignite/examples/ComputeClientPortableExampleTest.java
----------------------------------------------------------------------
diff --git a/examples/src/test/java/org/apache/ignite/examples/ComputeClientPortableExampleTest.java b/examples/src/test/java/org/apache/ignite/examples/ComputeClientPortableExampleTest.java
new file mode 100644
index 0000000..2223aec
--- /dev/null
+++ b/examples/src/test/java/org/apache/ignite/examples/ComputeClientPortableExampleTest.java
@@ -0,0 +1,37 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.ignite.examples;
+
+import org.apache.ignite.examples.portable.computegrid.ComputeClientPortableTaskExecutionExample;
+import org.apache.ignite.testframework.junits.common.GridAbstractExamplesTest;
+
+/**
+ *
+ */
+public class ComputeClientPortableExampleTest extends GridAbstractExamplesTest {
+    /** {@inheritDoc} */
+    @Override protected String defaultConfig() {
+        return "examples/config/portable/example-ignite-portable.xml";
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testPortableTaskExecutionExample() throws Exception {
+        ComputeClientPortableTaskExecutionExample.main(new String[] {});
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/6a5a48a0/examples/src/test/java/org/apache/ignite/testsuites/IgniteExamplesSelfTestSuite.java
----------------------------------------------------------------------
diff --git a/examples/src/test/java/org/apache/ignite/testsuites/IgniteExamplesSelfTestSuite.java b/examples/src/test/java/org/apache/ignite/testsuites/IgniteExamplesSelfTestSuite.java
index 4669ae4..baa23fc 100644
--- a/examples/src/test/java/org/apache/ignite/testsuites/IgniteExamplesSelfTestSuite.java
+++ b/examples/src/test/java/org/apache/ignite/testsuites/IgniteExamplesSelfTestSuite.java
@@ -20,10 +20,12 @@ package org.apache.ignite.testsuites;
 import junit.framework.TestSuite;
 import org.apache.ignite.examples.BasicExamplesMultiNodeSelfTest;
 import org.apache.ignite.examples.BasicExamplesSelfTest;
+import org.apache.ignite.examples.CacheClientPortableExampleTest;
 import org.apache.ignite.examples.CacheExamplesMultiNodeSelfTest;
 import org.apache.ignite.examples.CacheExamplesSelfTest;
 import org.apache.ignite.examples.CheckpointExamplesSelfTest;
 import org.apache.ignite.examples.ClusterGroupExampleSelfTest;
+import org.apache.ignite.examples.ComputeClientPortableExampleTest;
 import org.apache.ignite.examples.ContinuationExamplesMultiNodeSelfTest;
 import org.apache.ignite.examples.ContinuationExamplesSelfTest;
 import org.apache.ignite.examples.ContinuousMapperExamplesMultiNodeSelfTest;
@@ -93,6 +95,10 @@ public class IgniteExamplesSelfTestSuite extends TestSuite {
         suite.addTest(new TestSuite(MonteCarloExamplesMultiNodeSelfTest.class));
         suite.addTest(new TestSuite(HibernateL2CacheExampleMultiNodeSelfTest.class));
 
+        // Portable.
+        suite.addTest(new TestSuite(CacheClientPortableExampleTest.class));
+        suite.addTest(new TestSuite(ComputeClientPortableExampleTest.class));
+
         return suite;
     }
 }
\ No newline at end of file


[11/39] ignite git commit: Added test, more debug info for GridDhtTxPrepareFuture.

Posted by ag...@apache.org.
Added test, more debug info for GridDhtTxPrepareFuture.


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

Branch: refs/heads/ignite-264
Commit: 3aa9ea36354c76fa9bf958bdbf17b26951696ea9
Parents: ba4a582
Author: sboikov <sb...@gridgain.com>
Authored: Wed Sep 2 15:10:04 2015 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Wed Sep 2 15:10:04 2015 +0300

----------------------------------------------------------------------
 .../distributed/dht/GridDhtTxPrepareFuture.java | 15 ++++-
 .../CachePutAllFailoverAbstractTest.java        | 60 +++++++++++++++++---
 .../CachePutAllFailoverAtomicTest.java          | 30 ++++++++++
 .../IgniteCacheFailoverTestSuite2.java          |  2 +
 4 files changed, 99 insertions(+), 8 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/3aa9ea36/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 b5211ee..f1771d4 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
@@ -64,6 +64,7 @@ import org.apache.ignite.internal.util.future.GridCompoundFuture;
 import org.apache.ignite.internal.util.future.GridFutureAdapter;
 import org.apache.ignite.internal.util.tostring.GridToStringExclude;
 import org.apache.ignite.internal.util.tostring.GridToStringInclude;
+import org.apache.ignite.internal.util.typedef.C1;
 import org.apache.ignite.internal.util.typedef.CI1;
 import org.apache.ignite.internal.util.typedef.CIX1;
 import org.apache.ignite.internal.util.typedef.F;
@@ -174,6 +175,7 @@ public final class GridDhtTxPrepareFuture extends GridCompoundFuture<IgniteInter
     private Collection<IgniteTxKey> filterFailedKeys;
 
     /** Keys that should be locked. */
+    @GridToStringInclude
     private GridConcurrentHashSet<IgniteTxKey> lockKeys = new GridConcurrentHashSet<>();
 
     /** Force keys future for correct transforms. */
@@ -1249,7 +1251,18 @@ public final class GridDhtTxPrepareFuture extends GridCompoundFuture<IgniteInter
 
     /** {@inheritDoc} */
     @Override public String toString() {
-        return S.toString(GridDhtTxPrepareFuture.class, this, "xid", tx.xidVersion(), "super", super.toString());
+        Collection<String> futs = F.viewReadOnly(futures(), new C1<IgniteInternalFuture<?>, String>() {
+            @Override public String apply(IgniteInternalFuture<?> f) {
+                return "[node=" + ((MiniFuture)f).node().id() +
+                    ", loc=" + ((MiniFuture)f).node().isLocal() +
+                    ", done=" + f.isDone() + "]";
+            }
+        });
+
+        return S.toString(GridDhtTxPrepareFuture.class, this,
+            "xid", tx.xidVersion(),
+            "innerFuts", futs,
+            "super", super.toString());
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/3aa9ea36/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/CachePutAllFailoverAbstractTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/CachePutAllFailoverAbstractTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/CachePutAllFailoverAbstractTest.java
index ea81680..62fddda 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/CachePutAllFailoverAbstractTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/CachePutAllFailoverAbstractTest.java
@@ -25,10 +25,13 @@ import org.apache.ignite.internal.IgniteInternalFuture;
 import org.apache.ignite.internal.processors.cache.GridCacheAbstractSelfTest;
 import org.apache.ignite.internal.util.typedef.internal.S;
 import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.lang.IgniteFuture;
 import org.apache.ignite.testframework.GridTestUtils;
 import org.jetbrains.annotations.NotNull;
 
 import java.io.Serializable;
+import java.util.ArrayList;
+import java.util.Collection;
 import java.util.TreeMap;
 import java.util.concurrent.Callable;
 import java.util.concurrent.ThreadLocalRandom;
@@ -44,7 +47,7 @@ public abstract class CachePutAllFailoverAbstractTest extends GridCacheAbstractS
     private static final int NODE_CNT = 2;
 
     /** */
-    private static final long TEST_TIME = 2 * 60_000;
+    private static final long TEST_TIME = 60_000;
 
     /** {@inheritDoc} */
     @Override protected int gridCount() {
@@ -82,6 +85,21 @@ public abstract class CachePutAllFailoverAbstractTest extends GridCacheAbstractS
      * @throws Exception If failed.
      */
     public void testPutAllFailover() throws Exception {
+        testPutAllFailover(false);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testPutAllFailoverAsync() throws Exception {
+        testPutAllFailover(true);
+    }
+
+    /**
+     * @param async If {@code true} tests asynchronous operation.
+     * @throws Exception If failed.
+     */
+    private void testPutAllFailover(final boolean async) throws Exception {
         final AtomicBoolean finished = new AtomicBoolean();
 
         final long endTime = System.currentTimeMillis() + TEST_TIME;
@@ -103,7 +121,9 @@ public abstract class CachePutAllFailoverAbstractTest extends GridCacheAbstractS
         });
 
         try {
-            final IgniteCache<TestKey, TestValue> cache = ignite(0).cache(null);
+            IgniteCache<TestKey, TestValue> cache0 = ignite(0).cache(null);
+
+            final IgniteCache<TestKey, TestValue> cache = async ? cache0.withAsync() : cache0;
 
             GridTestUtils.runMultiThreaded(new Callable<Object>() {
                 @Override public Object call() throws Exception {
@@ -116,15 +136,41 @@ public abstract class CachePutAllFailoverAbstractTest extends GridCacheAbstractS
                     long lastInfo = 0;
 
                     while ((time = System.currentTimeMillis()) < endTime) {
-                        if (time - lastInfo > 5000)
+                        if (time - lastInfo > 5000) {
                             log.info("Do putAll [iter=" + iter + ']');
 
-                        TreeMap<TestKey, TestValue> map = new TreeMap<>();
+                            lastInfo = time;
+                        }
+
+                        if (async) {
+                            Collection<IgniteFuture<?>> futs = new ArrayList<>();
+
+                            for (int i = 0 ; i < 10; i++) {
+                                TreeMap<TestKey, TestValue> map = new TreeMap<>();
+
+                                for (int k = 0; k < 100; k++)
+                                    map.put(new TestKey(rnd.nextInt(200)), new TestValue(iter));
+
+                                cache.putAll(map);
+
+                                IgniteFuture<?> fut = cache.future();
+
+                                assertNotNull(fut);
+
+                                futs.add(fut);
+                            }
+
+                            for (IgniteFuture<?> fut : futs)
+                                fut.get();
+                        }
+                        else {
+                            TreeMap<TestKey, TestValue> map = new TreeMap<>();
 
-                        for (int k = 0; k < 100; k++)
-                            map.put(new TestKey(rnd.nextInt(200)), new TestValue(iter));
+                            for (int k = 0; k < 100; k++)
+                                map.put(new TestKey(rnd.nextInt(200)), new TestValue(iter));
 
-                        cache.putAll(map);
+                            cache.putAll(map);
+                        }
 
                         iter++;
                     }

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

http://git-wip-us.apache.org/repos/asf/ignite/blob/3aa9ea36/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheFailoverTestSuite2.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheFailoverTestSuite2.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheFailoverTestSuite2.java
index a8b2d65..20cc6b6 100644
--- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheFailoverTestSuite2.java
+++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheFailoverTestSuite2.java
@@ -21,6 +21,7 @@ import junit.framework.TestSuite;
 import org.apache.ignite.internal.processors.cache.CacheGetFromJobTest;
 import org.apache.ignite.internal.processors.cache.distributed.CacheAsyncOperationsFailoverAtomicTest;
 import org.apache.ignite.internal.processors.cache.distributed.CacheAsyncOperationsFailoverTxTest;
+import org.apache.ignite.internal.processors.cache.distributed.CachePutAllFailoverAtomicTest;
 import org.apache.ignite.internal.processors.cache.distributed.dht.GridCacheColocatedFailoverSelfTest;
 import org.apache.ignite.internal.processors.cache.distributed.dht.IgniteCacheCrossCacheTxFailoverTest;
 import org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridCacheAtomicFailoverSelfTest;
@@ -56,6 +57,7 @@ public class IgniteCacheFailoverTestSuite2 {
 
         suite.addTestSuite(CacheAsyncOperationsFailoverAtomicTest.class);
         suite.addTestSuite(CacheAsyncOperationsFailoverTxTest.class);
+        suite.addTestSuite(CachePutAllFailoverAtomicTest.class);
 
         return suite;
     }


[38/39] ignite git commit: minot javadoc fix

Posted by ag...@apache.org.
minot javadoc fix


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

Branch: refs/heads/ignite-264
Commit: 77fc969262584d82c5193ad347080876832f623f
Parents: 8901575
Author: Anton Vinogradov <av...@gridgain.com>
Authored: Thu Sep 3 23:24:21 2015 +0300
Committer: Anton Vinogradov <av...@gridgain.com>
Committed: Thu Sep 3 23:24:21 2015 +0300

----------------------------------------------------------------------
 .../configuration/CacheConfiguration.java       | 31 ++++++++------------
 1 file changed, 12 insertions(+), 19 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/77fc9692/modules/core/src/main/java/org/apache/ignite/configuration/CacheConfiguration.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/configuration/CacheConfiguration.java b/modules/core/src/main/java/org/apache/ignite/configuration/CacheConfiguration.java
index 1bbc110..85eed47 100644
--- a/modules/core/src/main/java/org/apache/ignite/configuration/CacheConfiguration.java
+++ b/modules/core/src/main/java/org/apache/ignite/configuration/CacheConfiguration.java
@@ -17,26 +17,10 @@
 
 package org.apache.ignite.configuration;
 
-import java.io.Serializable;
-import java.util.Collection;
-import javax.cache.Cache;
-import javax.cache.CacheException;
-import javax.cache.configuration.CompleteConfiguration;
-import javax.cache.configuration.Factory;
-import javax.cache.configuration.MutableConfiguration;
-import javax.cache.expiry.ExpiryPolicy;
 import org.apache.ignite.Ignite;
 import org.apache.ignite.IgniteCache;
 import org.apache.ignite.IgniteException;
-import org.apache.ignite.cache.CacheAtomicWriteOrderMode;
-import org.apache.ignite.cache.CacheAtomicityMode;
-import org.apache.ignite.cache.CacheEntryProcessor;
-import org.apache.ignite.cache.CacheInterceptor;
-import org.apache.ignite.cache.CacheMemoryMode;
-import org.apache.ignite.cache.CacheMode;
-import org.apache.ignite.cache.CacheRebalanceMode;
-import org.apache.ignite.cache.CacheTypeMetadata;
-import org.apache.ignite.cache.CacheWriteSynchronizationMode;
+import org.apache.ignite.cache.*;
 import org.apache.ignite.cache.affinity.AffinityFunction;
 import org.apache.ignite.cache.affinity.AffinityKeyMapper;
 import org.apache.ignite.cache.eviction.EvictionFilter;
@@ -52,6 +36,15 @@ import org.apache.ignite.lang.IgnitePredicate;
 import org.apache.ignite.plugin.CachePluginConfiguration;
 import org.jetbrains.annotations.Nullable;
 
+import javax.cache.Cache;
+import javax.cache.CacheException;
+import javax.cache.configuration.CompleteConfiguration;
+import javax.cache.configuration.Factory;
+import javax.cache.configuration.MutableConfiguration;
+import javax.cache.expiry.ExpiryPolicy;
+import java.io.Serializable;
+import java.util.Collection;
+
 /**
  * This class defines grid cache configuration. This configuration is passed to
  * grid via {@link IgniteConfiguration#getCacheConfiguration()} method. It defines all configuration
@@ -1806,7 +1799,7 @@ public class CacheConfiguration<K, V> extends MutableConfiguration<K, V> {
      * Following validator allows to put data only in case topology contains exactly 2 nodes:
      * <pre>{@code
      * new TopologyValidator() {
-     *    @Override public boolean validate(Collection<ClusterNode> nodes) {
+     *    public boolean validate(Collection<ClusterNode> nodes) {
      *       return nodes.size() == 2;
      *    }
      * }
@@ -1835,7 +1828,7 @@ public class CacheConfiguration<K, V> extends MutableConfiguration<K, V> {
      * Following validator allows to put data only in case topology contains exactly 2 nodes:
      * <pre>{@code
      * new TopologyValidator() {
-     *    @Override public boolean validate(Collection<ClusterNode> nodes) {
+     *    public boolean validate(Collection<ClusterNode> nodes) {
      *       return nodes.size() == 2;
      *    }
      * }


[36/39] ignite git commit: Merge remote-tracking branch 'origin/ignite-1.4' into ignite-1.4

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


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

Branch: refs/heads/ignite-264
Commit: 154f18588e01943bfde60ade49262837bb670204
Parents: 2fbf328 6a5a48a
Author: Anton Vinogradov <av...@apache.org>
Authored: Thu Sep 3 18:35:06 2015 +0300
Committer: Anton Vinogradov <av...@apache.org>
Committed: Thu Sep 3 18:35:06 2015 +0300

----------------------------------------------------------------------
 examples/config/example-default.xml             |  76 +++++
 examples/config/example-ignite.xml              |  56 +---
 .../config/portable/example-ignite-portable.xml |  44 +++
 .../ignite/examples/portable/Address.java       |  72 +++++
 .../ignite/examples/portable/Employee.java      |  93 ++++++
 .../ignite/examples/portable/EmployeeKey.java   |  90 ++++++
 .../portable/ExamplePortableNodeStartup.java    |  36 +++
 .../ignite/examples/portable/Organization.java  |  93 ++++++
 .../examples/portable/OrganizationType.java     |  32 ++
 ...mputeClientPortableTaskExecutionExample.java | 154 +++++++++
 .../portable/computegrid/ComputeClientTask.java | 116 +++++++
 .../portable/computegrid/package-info.java      |  21 ++
 .../CacheClientPortablePutGetExample.java       | 226 +++++++++++++
 .../CacheClientPortableQueryExample.java        | 323 +++++++++++++++++++
 .../portable/datagrid/package-info.java         |  21 ++
 .../ignite/examples/portable/package-info.java  |  21 ++
 .../CacheClientPortableExampleTest.java         |  46 +++
 .../ComputeClientPortableExampleTest.java       |  37 +++
 .../testsuites/IgniteExamplesSelfTestSuite.java |   6 +
 19 files changed, 1513 insertions(+), 50 deletions(-)
----------------------------------------------------------------------



[07/39] ignite git commit: Merge remote-tracking branch 'origin/ignite-1.4' into ignite-1.4

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


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

Branch: refs/heads/ignite-264
Commit: 2c236ac4063f31f316bf6d9003ad216e8a1fb124
Parents: 144bd4a da29a70
Author: sboikov <sb...@gridgain.com>
Authored: Wed Sep 2 13:23:55 2015 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Wed Sep 2 13:23:55 2015 +0300

----------------------------------------------------------------------
 .../processors/platform/lifecycle/PlatformLifecycleBean.java     | 4 +++-
 1 file changed, 3 insertions(+), 1 deletion(-)
----------------------------------------------------------------------



[21/39] ignite git commit: ignite-1273: added ability to modify arrays returned from PortableBuilder and fixed cyclic references processing for arrays and collections in PortableMarshaller

Posted by ag...@apache.org.
http://git-wip-us.apache.org/repos/asf/ignite/blob/9057a4c0/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
new file mode 100644
index 0000000..45355d7
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/PortableBuilderReader.java
@@ -0,0 +1,800 @@
+/*
+ * 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 java.sql.Timestamp;
+import java.util.Date;
+import java.util.HashMap;
+import java.util.Map;
+import org.apache.ignite.internal.portable.GridPortableMarshaller;
+import org.apache.ignite.internal.portable.PortableContext;
+import org.apache.ignite.internal.portable.PortableObjectImpl;
+import org.apache.ignite.internal.portable.PortablePrimitives;
+import org.apache.ignite.internal.portable.PortableReaderExImpl;
+import org.apache.ignite.internal.portable.PortableUtils;
+import org.apache.ignite.internal.portable.PortableWriterExImpl;
+import org.apache.ignite.portable.PortableException;
+
+import static java.nio.charset.StandardCharsets.UTF_8;
+import static org.apache.ignite.internal.portable.GridPortableMarshaller.NULL;
+import static org.apache.ignite.internal.portable.GridPortableMarshaller.STRING;
+
+/**
+ *
+ */
+class PortableBuilderReader {
+    /** */
+    private static final PortablePrimitives PRIM = PortablePrimitives.get();
+
+    /** */
+    private final Map<Integer, PortableBuilderImpl> objMap = new HashMap<>();
+
+    /** */
+    private final PortableContext ctx;
+
+    /** */
+    private final PortableReaderExImpl reader;
+
+    /** */
+    private byte[] arr;
+
+    /** */
+    private int pos;
+
+    /**
+     * @param objImpl Portable object
+     */
+    PortableBuilderReader(PortableObjectImpl objImpl) {
+        ctx = objImpl.context();
+        arr = objImpl.array();
+        pos = objImpl.start();
+
+        // TODO: IGNITE-1272 - Is class loader needed here?
+        reader = new PortableReaderExImpl(portableContext(), arr, pos, null);
+    }
+
+    /**
+     * @return Portable context.
+     */
+    public PortableContext portableContext() {
+        return ctx;
+    }
+
+    /**
+     * @param obj Mutable portable object.
+     */
+    public void registerObject(PortableBuilderImpl obj) {
+        objMap.put(obj.start(), obj);
+    }
+
+    /**
+     * @return Read int value.
+     */
+    public int readInt() {
+        int res = readInt(0);
+
+        pos += 4;
+
+        return res;
+    }
+
+    /**
+     * @return Read int value.
+     */
+    public byte readByte() {
+        return arr[pos++];
+    }
+
+    /**
+     * @return Read boolean value.
+     */
+    public boolean readBoolean() {
+        return readByte() == 1;
+    }
+
+    /**
+     * @return Read int value.
+     */
+    public byte readByte(int off) {
+        return arr[pos + off];
+    }
+
+    /**
+     * @param off Offset related to {@link #pos}
+     * @return Read int value.
+     */
+    public int readInt(int off) {
+        return PRIM.readInt(arr, pos + off);
+    }
+
+    /**
+     * @param pos Position in the source array.
+     * @return Read int value.
+     */
+    public int readIntAbsolute(int pos) {
+        return PRIM.readInt(arr, pos);
+    }
+
+    /**
+     * @return Read length of array.
+     */
+    public int readLength() {
+        return PRIM.readInt(arr, pos);
+    }
+
+    /**
+     * Read string length.
+     *
+     * @return String length.
+     */
+    public int readStringLength() {
+        boolean utf = PRIM.readBoolean(arr, pos);
+
+        int arrLen = PRIM.readInt(arr, pos + 1);
+
+        return 1 + (utf ? arrLen : arrLen << 1);
+    }
+
+    /**
+     * Reads string.
+     *
+     * @return String.
+     */
+    public String readString() {
+        byte flag = readByte();
+
+        if (flag == NULL)
+            return null;
+
+        if (flag != STRING)
+            throw new PortableException("Failed to deserialize String.");
+
+        boolean convert = readBoolean();
+        int len = readInt();
+
+        String str;
+
+        if (convert) {
+            str = new String(arr, pos, len, UTF_8);
+
+            pos += len;
+        }
+        else {
+            str = String.valueOf(PRIM.readCharArray(arr, pos, len));
+
+            pos += len << 1;
+        }
+
+        return str;
+    }
+
+    /**
+     *
+     */
+    public void skipValue() {
+        byte type = arr[pos++];
+
+        int len;
+
+        switch (type) {
+            case GridPortableMarshaller.NULL:
+                return;
+
+            case GridPortableMarshaller.OBJ:
+                pos += readInt(GridPortableMarshaller.TOTAL_LEN_POS - 1) - 1;
+
+                return;
+
+            case GridPortableMarshaller.BOOLEAN:
+            case GridPortableMarshaller.BYTE:
+                len = 1;
+                break;
+
+            case GridPortableMarshaller.CHAR:
+            case GridPortableMarshaller.SHORT:
+                len = 2;
+
+                break;
+
+            case GridPortableMarshaller.HANDLE:
+            case GridPortableMarshaller.FLOAT:
+            case GridPortableMarshaller.INT:
+                len = 4;
+
+                break;
+
+            case GridPortableMarshaller.ENUM:
+                //skipping type id and ordinal value
+                len = 8;
+
+                break;
+
+            case GridPortableMarshaller.LONG:
+            case GridPortableMarshaller.DOUBLE:
+                len = 8;
+
+                break;
+
+            case GridPortableMarshaller.BYTE_ARR:
+            case GridPortableMarshaller.BOOLEAN_ARR:
+                len = 4 + readLength();
+
+                break;
+
+            case GridPortableMarshaller.STRING:
+                len = 4 + readStringLength();
+
+                break;
+
+            case GridPortableMarshaller.DECIMAL:
+                len = /** scale */ 4  + /** mag len */ 4  + /** mag bytes count */ readInt(4);
+
+                break;
+
+            case GridPortableMarshaller.UUID:
+                len = 8 + 8;
+
+                break;
+
+            case GridPortableMarshaller.DATE:
+                len = 8 + 4;
+
+                break;
+
+            case GridPortableMarshaller.CHAR_ARR:
+            case GridPortableMarshaller.SHORT_ARR:
+                len = 4 + readLength() * 2;
+
+                break;
+
+            case GridPortableMarshaller.INT_ARR:
+            case GridPortableMarshaller.FLOAT_ARR:
+                len = 4 + readLength() * 4;
+
+                break;
+
+            case GridPortableMarshaller.LONG_ARR:
+            case GridPortableMarshaller.DOUBLE_ARR:
+                len = 4 + readLength() * 8;
+
+                break;
+
+            case GridPortableMarshaller.DECIMAL_ARR:
+            case GridPortableMarshaller.DATE_ARR:
+            case GridPortableMarshaller.OBJ_ARR:
+            case GridPortableMarshaller.ENUM_ARR:
+            case GridPortableMarshaller.UUID_ARR:
+            case GridPortableMarshaller.STRING_ARR: {
+                int size = readInt();
+
+                for (int i = 0; i < size; i++)
+                    skipValue();
+
+                return;
+            }
+
+            case GridPortableMarshaller.COL: {
+                int size = readInt();
+
+                pos++; // skip collection type
+
+                for (int i = 0; i < size; i++)
+                    skipValue();
+
+                return;
+            }
+
+            case GridPortableMarshaller.MAP: {
+                int size = readInt();
+
+                pos++; // skip collection type
+
+                for (int i = 0; i < size; i++) {
+                    skipValue(); // skip key.
+                    skipValue(); // skip value.
+                }
+
+                return;
+            }
+
+            case GridPortableMarshaller.MAP_ENTRY:
+                skipValue();
+                skipValue();
+
+                return;
+
+            case GridPortableMarshaller.PORTABLE_OBJ:
+                len = readInt() + 4;
+
+                break;
+
+            default:
+                throw new PortableException("Invalid flag value: " + type);
+        }
+
+        pos += len;
+    }
+
+    /**
+     * @param pos Position.
+     * @param len Length.
+     * @return Object.
+     */
+    public Object getValueQuickly(int pos, int len) {
+        byte type = arr[pos];
+
+        switch (type) {
+            case GridPortableMarshaller.NULL:
+                return null;
+
+            case GridPortableMarshaller.HANDLE: {
+                int objStart = pos - readIntAbsolute(pos + 1);
+
+                PortableBuilderImpl res = objMap.get(objStart);
+
+                if (res == null) {
+                    res = new PortableBuilderImpl(this, objStart);
+
+                    objMap.put(objStart, res);
+                }
+
+                return res;
+            }
+
+            case GridPortableMarshaller.OBJ: {
+                PortableBuilderImpl res = objMap.get(pos);
+
+                if (res == null) {
+                    res = new PortableBuilderImpl(this, pos);
+
+                    objMap.put(pos, res);
+                }
+
+                return res;
+            }
+
+            case GridPortableMarshaller.BYTE:
+                return arr[pos + 1];
+
+            case GridPortableMarshaller.SHORT:
+                return PRIM.readShort(arr, pos + 1);
+
+            case GridPortableMarshaller.INT:
+                return PRIM.readInt(arr, pos + 1);
+
+            case GridPortableMarshaller.LONG:
+                return PRIM.readLong(arr, pos + 1);
+
+            case GridPortableMarshaller.FLOAT:
+                return PRIM.readFloat(arr, pos + 1);
+
+            case GridPortableMarshaller.DOUBLE:
+                return PRIM.readDouble(arr, pos + 1);
+
+            case GridPortableMarshaller.CHAR:
+                return PRIM.readChar(arr, pos + 1);
+
+            case GridPortableMarshaller.BOOLEAN:
+                return arr[pos + 1] != 0;
+
+            case GridPortableMarshaller.DECIMAL:
+            case GridPortableMarshaller.STRING:
+            case GridPortableMarshaller.UUID:
+            case GridPortableMarshaller.DATE:
+                return new PortablePlainLazyValue(this, pos, len);
+
+            case GridPortableMarshaller.BYTE_ARR:
+            case GridPortableMarshaller.SHORT_ARR:
+            case GridPortableMarshaller.INT_ARR:
+            case GridPortableMarshaller.LONG_ARR:
+            case GridPortableMarshaller.FLOAT_ARR:
+            case GridPortableMarshaller.DOUBLE_ARR:
+            case GridPortableMarshaller.CHAR_ARR:
+            case GridPortableMarshaller.BOOLEAN_ARR:
+            case GridPortableMarshaller.DECIMAL_ARR:
+            case GridPortableMarshaller.DATE_ARR:
+            case GridPortableMarshaller.UUID_ARR:
+            case GridPortableMarshaller.STRING_ARR:
+            case GridPortableMarshaller.ENUM_ARR:
+            case GridPortableMarshaller.OBJ_ARR:
+            case GridPortableMarshaller.COL:
+            case GridPortableMarshaller.MAP:
+            case GridPortableMarshaller.MAP_ENTRY:
+                return new LazyCollection(pos);
+
+            case GridPortableMarshaller.ENUM: {
+                if (len == 1) {
+                    assert readByte(pos) == GridPortableMarshaller.NULL;
+
+                    return null;
+                }
+
+                int mark = position();
+                position(pos + 1);
+
+                PortableBuilderEnum builderEnum = new PortableBuilderEnum(this);
+
+                position(mark);
+
+                return builderEnum;
+            }
+
+            case GridPortableMarshaller.PORTABLE_OBJ: {
+                int size = readIntAbsolute(pos + 1);
+
+                int start = readIntAbsolute(pos + 4 + size);
+
+                PortableObjectImpl portableObj = new PortableObjectImpl(ctx, arr, pos + 4 + start);
+
+                return new PortablePlainPortableObject(portableObj);
+            }
+
+            default:
+                throw new PortableException("Invalid flag value: " + type);
+        }
+    }
+
+    /**
+     * @return Parsed value.
+     */
+    public Object parseValue() {
+        int valPos = pos;
+
+        byte type = arr[pos++];
+
+        int plainLazyValLen;
+
+        boolean modifiableLazyVal = false;
+
+        switch (type) {
+            case GridPortableMarshaller.NULL:
+                return null;
+
+            case GridPortableMarshaller.HANDLE: {
+                int objStart = pos - 1 - readInt();
+
+                PortableBuilderImpl res = objMap.get(objStart);
+
+                if (res == null) {
+                    res = new PortableBuilderImpl(this, objStart);
+
+                    objMap.put(objStart, res);
+                }
+
+                return res;
+            }
+
+            case GridPortableMarshaller.OBJ: {
+                pos--;
+
+                PortableBuilderImpl res = objMap.get(pos);
+
+                if (res == null) {
+                    res = new PortableBuilderImpl(this, pos);
+
+                    objMap.put(pos, res);
+                }
+
+                pos += readInt(GridPortableMarshaller.TOTAL_LEN_POS);
+
+                return res;
+            }
+
+            case GridPortableMarshaller.BYTE:
+                return arr[pos++];
+
+            case GridPortableMarshaller.SHORT: {
+                Object res = PRIM.readShort(arr, pos);
+                pos += 2;
+                return res;
+            }
+
+            case GridPortableMarshaller.INT:
+                return readInt();
+
+            case GridPortableMarshaller.LONG:
+                plainLazyValLen = 8;
+
+                break;
+
+            case GridPortableMarshaller.FLOAT:
+                plainLazyValLen = 4;
+
+                break;
+
+            case GridPortableMarshaller.DOUBLE:
+                plainLazyValLen = 8;
+
+                break;
+
+            case GridPortableMarshaller.CHAR:
+                plainLazyValLen = 2;
+
+                break;
+
+            case GridPortableMarshaller.BOOLEAN:
+                return arr[pos++] != 0;
+
+            case GridPortableMarshaller.DECIMAL:
+                plainLazyValLen = /** scale */ 4  + /** mag len */ 4  + /** mag bytes count */ readInt(4);
+
+                break;
+
+            case GridPortableMarshaller.STRING:
+                plainLazyValLen = 4 + readStringLength();
+
+                break;
+
+            case GridPortableMarshaller.UUID:
+                plainLazyValLen = 8 + 8;
+
+                break;
+
+            case GridPortableMarshaller.DATE:
+                plainLazyValLen = 8 + 4;
+
+                break;
+
+            case GridPortableMarshaller.BYTE_ARR:
+                plainLazyValLen = 4 + readLength();
+                modifiableLazyVal = true;
+
+                break;
+
+            case GridPortableMarshaller.SHORT_ARR:
+                plainLazyValLen = 4 + readLength() * 2;
+                modifiableLazyVal = true;
+
+                break;
+
+            case GridPortableMarshaller.INT_ARR:
+                plainLazyValLen = 4 + readLength() * 4;
+                modifiableLazyVal = true;
+
+                break;
+
+            case GridPortableMarshaller.LONG_ARR:
+                plainLazyValLen = 4 + readLength() * 8;
+                modifiableLazyVal = true;
+
+                break;
+
+            case GridPortableMarshaller.FLOAT_ARR:
+                plainLazyValLen = 4 + readLength() * 4;
+                modifiableLazyVal = true;
+
+                break;
+
+            case GridPortableMarshaller.DOUBLE_ARR:
+                plainLazyValLen = 4 + readLength() * 8;
+                modifiableLazyVal = true;
+
+                break;
+
+            case GridPortableMarshaller.CHAR_ARR:
+                plainLazyValLen = 4 + readLength() * 2;
+                modifiableLazyVal = true;
+
+                break;
+
+            case GridPortableMarshaller.BOOLEAN_ARR:
+                plainLazyValLen = 4 + readLength();
+                modifiableLazyVal = true;
+
+                break;
+
+            case GridPortableMarshaller.OBJ_ARR:
+                return new PortableObjectArrayLazyValue(this);
+
+            case GridPortableMarshaller.DATE_ARR: {
+                int size = readInt();
+
+                Date[] res = new Date[size];
+
+                for (int i = 0; i < res.length; i++) {
+                    byte flag = arr[pos++];
+
+                    if (flag == GridPortableMarshaller.NULL) continue;
+
+                    if (flag != GridPortableMarshaller.DATE)
+                        throw new PortableException("Invalid flag value: " + flag);
+
+                    long time = PRIM.readLong(arr, pos);
+
+                    pos += 8;
+
+                    if (ctx.isUseTimestamp()) {
+                        Timestamp ts = new Timestamp(time);
+
+                        ts.setNanos(ts.getNanos() + readInt());
+
+                        res[i] = ts;
+                    }
+                    else {
+                        res[i] = new Date(time);
+
+                        pos += 4;
+                    }
+                }
+
+                return res;
+            }
+
+            case GridPortableMarshaller.UUID_ARR:
+            case GridPortableMarshaller.STRING_ARR:
+            case GridPortableMarshaller.DECIMAL_ARR: {
+                int size = readInt();
+
+                for (int i = 0; i < size; i++) {
+                    byte flag = arr[pos++];
+
+                    if (flag == GridPortableMarshaller.UUID)
+                        pos += 8 + 8;
+                    else if (flag == GridPortableMarshaller.STRING)
+                        pos += 4 + readStringLength();
+                    else if (flag == GridPortableMarshaller.DECIMAL) {
+                        pos += 4; // scale value
+                        pos += 4 + readLength();
+                    }
+                    else
+                        assert flag == GridPortableMarshaller.NULL;
+                }
+
+                return new PortableModifiableLazyValue(this, valPos, pos - valPos);
+            }
+
+            case GridPortableMarshaller.COL: {
+                int size = readInt();
+                byte colType = arr[pos++];
+
+                switch (colType) {
+                    case GridPortableMarshaller.USER_COL:
+                    case GridPortableMarshaller.ARR_LIST:
+                        return new PortableLazyArrayList(this, size);
+
+                    case GridPortableMarshaller.LINKED_LIST:
+                        return new PortableLazyLinkedList(this, size);
+
+                    case GridPortableMarshaller.HASH_SET:
+                    case GridPortableMarshaller.LINKED_HASH_SET:
+                    case GridPortableMarshaller.TREE_SET:
+                    case GridPortableMarshaller.CONC_SKIP_LIST_SET:
+                        return new PortableLazySet(this, size);
+                }
+
+                throw new PortableException("Unknown collection type: " + colType);
+            }
+
+            case GridPortableMarshaller.MAP:
+                return PortableLazyMap.parseMap(this);
+
+            case GridPortableMarshaller.ENUM:
+                return new PortableBuilderEnum(this);
+
+            case GridPortableMarshaller.ENUM_ARR:
+                return new PortableEnumArrayLazyValue(this);
+
+            case GridPortableMarshaller.MAP_ENTRY:
+                return new PortableLazyMapEntry(this);
+
+            case GridPortableMarshaller.PORTABLE_OBJ: {
+                int size = readInt();
+
+                pos += size;
+
+                int start = readInt();
+
+                PortableObjectImpl portableObj = new PortableObjectImpl(ctx, arr,
+                    pos - 4 - size + start);
+
+                return new PortablePlainPortableObject(portableObj);
+            }
+
+
+            default:
+                throw new PortableException("Invalid flag value: " + type);
+        }
+
+        PortableAbstractLazyValue res;
+
+        if (modifiableLazyVal)
+            res = new PortableModifiableLazyValue(this, valPos, 1 + plainLazyValLen);
+        else
+            res = new PortablePlainLazyValue(this, valPos, 1 + plainLazyValLen);
+
+        pos += plainLazyValLen;
+
+        return res;
+    }
+
+    /**
+     * @return Array.
+     */
+    public byte[] array() {
+        return arr;
+    }
+
+    /**
+     * @return Position of reader.
+     */
+    public int position() {
+        return pos;
+    }
+
+    /**
+     * @param pos New pos.
+     */
+    public void position(int pos) {
+        this.pos = pos;
+    }
+
+    /**
+     * @param n Number of bytes to skip.
+     */
+    public void skip(int n) {
+        pos += n;
+    }
+
+    /**
+     * @return Reader.
+     */
+    PortableReaderExImpl reader() {
+        return reader;
+    }
+
+    /**
+     *
+     */
+    private class LazyCollection implements PortableLazyValue {
+        /** */
+        private final int valOff;
+
+        /** */
+        private Object col;
+
+        /**
+         * @param valOff Value.
+         */
+        protected LazyCollection(int valOff) {
+            this.valOff = valOff;
+        }
+
+        /**
+         * @return Object.
+         */
+        private Object wrappedCollection() {
+            if (col == null) {
+                position(valOff);
+
+                col = parseValue();
+            }
+
+            return col;
+        }
+
+        /** {@inheritDoc} */
+        @Override public void writeTo(PortableWriterExImpl writer, PortableBuilderSerializer ctx) {
+            ctx.writeValue(writer, wrappedCollection());
+        }
+
+        /** {@inheritDoc} */
+        @Override public Object value() {
+            return PortableUtils.unwrapLazy(wrappedCollection());
+        }
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/9057a4c0/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/PortableBuilderSerializationAware.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/PortableBuilderSerializationAware.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/PortableBuilderSerializationAware.java
new file mode 100644
index 0000000..976059a
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/PortableBuilderSerializationAware.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.portable.builder;
+
+import org.apache.ignite.internal.portable.*;
+
+/**
+ *
+ */
+interface PortableBuilderSerializationAware {
+    /**
+     * @param writer Writer.
+     * @param ctx Context.
+     */
+    public void writeTo(PortableWriterExImpl writer, PortableBuilderSerializer ctx);
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/9057a4c0/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
new file mode 100644
index 0000000..2d9c961
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/PortableBuilderSerializer.java
@@ -0,0 +1,214 @@
+/*
+ * 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.GridPortableMarshaller;
+import org.apache.ignite.internal.portable.PortableObjectEx;
+import org.apache.ignite.internal.portable.PortableUtils;
+import org.apache.ignite.internal.portable.PortableWriterExImpl;
+import org.apache.ignite.internal.util.*;
+import org.apache.ignite.portable.*;
+
+import java.util.*;
+
+/**
+ *
+ */
+class PortableBuilderSerializer {
+    /** */
+    private final Map<PortableBuilderImpl, Integer> objToPos = new IdentityHashMap<>();
+
+    /** */
+    private Map<PortableObject, PortableBuilderImpl> portableObjToWrapper;
+
+    /**
+     * @param obj Mutable object.
+     * @param posInResArr Object position in the array.
+     */
+    public void registerObjectWriting(PortableBuilderImpl obj, int posInResArr) {
+        objToPos.put(obj, posInResArr);
+    }
+
+    /**
+     * @param writer Writer.
+     * @param val Value.
+     */
+    public void writeValue(PortableWriterExImpl writer, Object val) {
+        if (val == null) {
+            writer.writeByte(GridPortableMarshaller.NULL);
+
+            return;
+        }
+
+        if (val instanceof PortableBuilderSerializationAware) {
+            ((PortableBuilderSerializationAware)val).writeTo(writer, this);
+
+            return;
+        }
+
+        if (val instanceof PortableObjectEx) {
+            if (portableObjToWrapper == null)
+                portableObjToWrapper = new IdentityHashMap<>();
+
+            PortableBuilderImpl wrapper = portableObjToWrapper.get(val);
+
+            if (wrapper == null) {
+                wrapper = PortableBuilderImpl.wrap((PortableObject)val);
+
+                portableObjToWrapper.put((PortableObject)val, wrapper);
+            }
+
+            val = wrapper;
+        }
+
+        if (val instanceof PortableBuilderImpl) {
+            PortableBuilderImpl obj = (PortableBuilderImpl)val;
+
+            Integer posInResArr = objToPos.get(obj);
+
+            if (posInResArr == null) {
+                objToPos.put(obj, writer.outputStream().position());
+
+                obj.serializeTo(writer.newWriter(obj.typeId()), this);
+            }
+            else {
+                int handle = writer.outputStream().position() - posInResArr;
+
+                writer.writeByte(GridPortableMarshaller.HANDLE);
+                writer.writeInt(handle);
+            }
+
+            return;
+        }
+
+        if (val.getClass().isEnum()) {
+            writer.writeByte(GridPortableMarshaller.ENUM);
+            writer.writeInt(writer.context().typeId(val.getClass().getName()));
+            writer.writeInt(((Enum)val).ordinal());
+
+            return;
+        }
+
+        if (val instanceof Collection) {
+            Collection<?> c = (Collection<?>)val;
+
+            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());
+
+
+            writer.writeByte(colType);
+
+            for (Object obj : c)
+                writeValue(writer, obj);
+
+            return;
+        }
+
+        if (val instanceof Map) {
+            Map<?, ?> map = (Map<?, ?>)val;
+
+            writer.writeByte(GridPortableMarshaller.MAP);
+            writer.writeInt(map.size());
+
+            writer.writeByte(writer.context().mapType(map.getClass()));
+
+            for (Map.Entry<?, ?> entry : map.entrySet()) {
+                writeValue(writer, entry.getKey());
+                writeValue(writer, entry.getValue());
+            }
+
+            return;
+        }
+
+        Byte flag = PortableUtils.PLAIN_CLASS_TO_FLAG.get(val.getClass());
+
+        if (flag != null) {
+            PortableUtils.writePlainObject(writer, val);
+
+            return;
+        }
+
+        if (val instanceof Object[]) {
+            int compTypeId = writer.context().typeId(((Object[])val).getClass().getComponentType().getName());
+
+            if (val instanceof PortableBuilderEnum[]) {
+                writeArray(writer, GridPortableMarshaller.ENUM_ARR, (Object[])val, compTypeId);
+
+                return;
+            }
+
+            if (((Object[])val).getClass().getComponentType().isEnum()) {
+                Enum[] enumArr = (Enum[])val;
+
+                writer.writeByte(GridPortableMarshaller.ENUM_ARR);
+                writer.writeInt(compTypeId);
+                writer.writeInt(enumArr.length);
+
+                for (Enum anEnum : enumArr)
+                    writeValue(writer, anEnum);
+
+                return;
+            }
+
+            writeArray(writer, GridPortableMarshaller.OBJ_ARR, (Object[])val, compTypeId);
+
+            return;
+        }
+
+        writer.doWriteObject(val, false);
+    }
+
+    /**
+     * @param writer Writer.
+     * @param elementType Element type.
+     * @param arr The array.
+     * @param compTypeId Component type ID.
+     */
+    public void writeArray(PortableWriterExImpl writer, byte elementType, Object[] arr, int compTypeId) {
+        writer.writeByte(elementType);
+        writer.writeInt(compTypeId);
+        writer.writeInt(arr.length);
+
+        for (Object obj : arr)
+            writeValue(writer, obj);
+    }
+
+    /**
+     * @param writer Writer.
+     * @param elementType Element type.
+     * @param arr The array.
+     * @param clsName Component class name.
+     */
+    public void writeArray(PortableWriterExImpl writer, byte elementType, Object[] arr, String clsName) {
+        writer.writeByte(elementType);
+        writer.writeInt(GridPortableMarshaller.UNREGISTERED_TYPE_ID);
+        writer.writeString(clsName);
+        writer.writeInt(arr.length);
+
+        for (Object obj : arr)
+            writeValue(writer, obj);
+    }
+
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/9057a4c0/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/PortableEnumArrayLazyValue.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/PortableEnumArrayLazyValue.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/PortableEnumArrayLazyValue.java
new file mode 100644
index 0000000..d864a6e
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/PortableEnumArrayLazyValue.java
@@ -0,0 +1,114 @@
+/*
+ * 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.GridPortableMarshaller;
+import org.apache.ignite.internal.portable.PortableWriterExImpl;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.portable.PortableException;
+import org.apache.ignite.portable.PortableInvalidClassException;
+
+/**
+ *
+ */
+class PortableEnumArrayLazyValue extends PortableAbstractLazyValue {
+    /** */
+    private final int len;
+
+    /** */
+    private final int compTypeId;
+
+    /** */
+    private final String clsName;
+
+    /**
+     * @param reader Reader.
+     */
+    protected PortableEnumArrayLazyValue(PortableBuilderReader reader) {
+        super(reader, reader.position() - 1);
+
+        int typeId = reader.readInt();
+
+        if (typeId == GridPortableMarshaller.UNREGISTERED_TYPE_ID) {
+            clsName = reader.readString();
+
+            Class cls;
+
+            try {
+                // TODO: IGNITE-1272 - Is class loader needed here?
+                cls = U.forName(reader.readString(), null);
+            }
+            catch (ClassNotFoundException e) {
+                throw new PortableInvalidClassException("Failed to load the class: " + clsName, e);
+            }
+
+            compTypeId = reader.portableContext().descriptorForClass(cls).typeId();
+        }
+        else {
+            compTypeId = typeId;
+            clsName = null;
+        }
+
+        int size = reader.readInt();
+
+        for (int i = 0; i < size; i++)
+            reader.skipValue();
+
+        len = reader.position() - valOff;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected Object init() {
+        reader.position(valOff + 1);
+
+        //skipping component type id
+        reader.readInt();
+
+        int size = reader.readInt();
+
+        PortableBuilderEnum[] res = new PortableBuilderEnum[size];
+
+        for (int i = 0; i < size; i++) {
+            byte flag = reader.readByte();
+
+            if (flag == GridPortableMarshaller.NULL)
+                continue;
+
+            if (flag != GridPortableMarshaller.ENUM)
+                throw new PortableException("Invalid flag value: " + flag);
+
+            res[i] = new PortableBuilderEnum(reader);
+        }
+
+        return res;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeTo(PortableWriterExImpl writer, PortableBuilderSerializer ctx) {
+        if (val != null) {
+            if (clsName != null)
+                ctx.writeArray(writer, GridPortableMarshaller.ENUM_ARR, (Object[])val, clsName);
+            else
+                ctx.writeArray(writer, GridPortableMarshaller.ENUM_ARR, (Object[])val, compTypeId);
+
+            return;
+        }
+
+        writer.write(reader.array(), valOff, len);
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/9057a4c0/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/PortableLazyArrayList.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/PortableLazyArrayList.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/PortableLazyArrayList.java
new file mode 100644
index 0000000..a08cfdd
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/PortableLazyArrayList.java
@@ -0,0 +1,166 @@
+/*
+ * 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.AbstractList;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+
+/**
+ *
+ */
+class PortableLazyArrayList extends AbstractList<Object> implements PortableBuilderSerializationAware {
+    /** */
+    private final PortableBuilderReader reader;
+
+    /** */
+    private final int off;
+
+    /** */
+    private List<Object> delegate;
+
+    /**
+     * @param reader Reader.
+     * @param size Size,
+     */
+    PortableLazyArrayList(PortableBuilderReader reader, int size) {
+        this.reader = reader;
+        off = reader.position() - 1/* flag */ - 4/* size */ - 1/* col type */;
+
+        assert size >= 0;
+
+        for (int i = 0; i < size; i++)
+            reader.skipValue();
+    }
+
+    /**
+     *
+     */
+    private void ensureDelegateInit() {
+        if (delegate == null) {
+            int size = reader.readIntAbsolute(off + 1);
+
+            reader.position(off + 1/* flag */ + 4/* size */ + 1/* col type */);
+
+            delegate = new ArrayList<>(size);
+
+            for (int i = 0; i < size; i++)
+                delegate.add(reader.parseValue());
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public Object get(int idx) {
+        ensureDelegateInit();
+
+        return PortableUtils.unwrapLazy(delegate.get(idx));
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean add(Object o) {
+        ensureDelegateInit();
+
+        return delegate.add(o);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void add(int idx, Object element) {
+        ensureDelegateInit();
+
+        delegate.add(idx, element);
+    }
+
+    /** {@inheritDoc} */
+    @Override public Object set(int idx, Object element) {
+        ensureDelegateInit();
+
+        return PortableUtils.unwrapLazy(delegate.set(idx, element));
+    }
+
+    /** {@inheritDoc} */
+    @Override public Object remove(int idx) {
+        ensureDelegateInit();
+
+        return PortableUtils.unwrapLazy(delegate.remove(idx));
+    }
+
+    /** {@inheritDoc} */
+    @Override public void clear() {
+        if (delegate == null)
+            delegate = new ArrayList<>();
+        else
+            delegate.clear();
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean addAll(int idx, Collection<?> c) {
+        return delegate.addAll(idx, c);
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void removeRange(int fromIdx, int toIdx) {
+        ensureDelegateInit();
+
+        delegate.subList(fromIdx, toIdx).clear();
+    }
+
+    /** {@inheritDoc} */
+    @Override public int size() {
+        if (delegate == null)
+            return reader.readIntAbsolute(off + 1);
+
+        return delegate.size();
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeTo(PortableWriterExImpl writer, PortableBuilderSerializer ctx) {
+        if (delegate == null) {
+            int size = reader.readIntAbsolute(off + 1);
+
+            int hdrSize = 1 /* flag */ + 4 /* size */ + 1 /* col type */;
+
+            writer.write(reader.array(), off, hdrSize);
+
+            reader.position(off + hdrSize);
+
+            for (int i = 0; i < size; i++) {
+                Object o = reader.parseValue();
+
+                ctx.writeValue(writer, o);
+            }
+        }
+        else {
+            writer.writeByte(GridPortableMarshaller.COL);
+            writer.writeInt(delegate.size());
+
+            byte colType = reader.array()[off + 1 /* flag */ + 4 /* size */];
+            writer.writeByte(colType);
+
+            int oldPos = reader.position();
+
+            for (Object o : delegate)
+                ctx.writeValue(writer, o);
+
+            // PortableBuilderImpl might have been written. It could override reader's position.
+            reader.position(oldPos);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/9057a4c0/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/PortableLazyLinkedList.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/PortableLazyLinkedList.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/PortableLazyLinkedList.java
new file mode 100644
index 0000000..f793d7a
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/PortableLazyLinkedList.java
@@ -0,0 +1,217 @@
+/*
+ * 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.AbstractList;
+import java.util.Collection;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.ListIterator;
+
+/**
+ *
+ */
+class PortableLazyLinkedList extends AbstractList<Object> implements PortableBuilderSerializationAware {
+    /** */
+    private final PortableBuilderReader reader;
+
+    /** */
+    private final int off;
+
+    /** */
+    private List<Object> delegate;
+
+    /**
+     * @param reader Reader.
+     * @param size Size,
+     */
+    PortableLazyLinkedList(PortableBuilderReader reader, int size) {
+        this.reader = reader;
+        off = reader.position() - 1/* flag */ - 4/* size */ - 1/* col type */;
+
+        assert size >= 0;
+
+        for (int i = 0; i < size; i++)
+            reader.skipValue();
+    }
+
+    /**
+     *
+     */
+    private void ensureDelegateInit() {
+        if (delegate == null) {
+            int size = reader.readIntAbsolute(off + 1);
+
+            reader.position(off + 1/* flag */ + 4/* size */ + 1/* col type */);
+
+            delegate = new LinkedList<>();
+
+            for (int i = 0; i < size; i++)
+                delegate.add(reader.parseValue());
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public Object get(int idx) {
+        ensureDelegateInit();
+
+        return PortableUtils.unwrapLazy(delegate.get(idx));
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean add(Object o) {
+        ensureDelegateInit();
+
+        return delegate.add(o);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void add(int idx, Object element) {
+        ensureDelegateInit();
+
+        delegate.add(idx, element);
+    }
+
+    /** {@inheritDoc} */
+    @Override public Object set(int idx, Object element) {
+        ensureDelegateInit();
+
+        return PortableUtils.unwrapLazy(delegate.set(idx, element));
+    }
+
+    /** {@inheritDoc} */
+    @Override public Object remove(int idx) {
+        ensureDelegateInit();
+
+        return PortableUtils.unwrapLazy(delegate.remove(idx));
+    }
+
+    /** {@inheritDoc} */
+    @Override public void clear() {
+        if (delegate == null)
+            delegate = new LinkedList<>();
+        else
+            delegate.clear();
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean addAll(int idx, Collection<?> c) {
+        ensureDelegateInit();
+
+        return delegate.addAll(idx, c);
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void removeRange(int fromIdx, int toIdx) {
+        ensureDelegateInit();
+
+        delegate.subList(fromIdx, toIdx).clear();
+    }
+
+    /** {@inheritDoc} */
+    @Override public int size() {
+        if (delegate == null)
+            return reader.readIntAbsolute(off + 1);
+
+        return delegate.size();
+    }
+
+    /** {@inheritDoc} */
+    @Override public ListIterator<Object> listIterator(final int idx) {
+        ensureDelegateInit();
+
+        return new ListIterator<Object>() {
+            /** */
+            private final ListIterator<Object> delegate = PortableLazyLinkedList.super.listIterator(idx);
+
+            @Override public boolean hasNext() {
+                return delegate.hasNext();
+            }
+
+            @Override public Object next() {
+                return PortableUtils.unwrapLazy(delegate.next());
+            }
+
+            @Override public boolean hasPrevious() {
+                return delegate.hasPrevious();
+            }
+
+            @Override public Object previous() {
+                return PortableUtils.unwrapLazy(delegate.previous());
+            }
+
+            @Override public int nextIndex() {
+                return delegate.nextIndex();
+            }
+
+            @Override public int previousIndex() {
+                return delegate.previousIndex();
+            }
+
+            @Override public void remove() {
+                delegate.remove();
+            }
+
+            @Override public void set(Object o) {
+                delegate.set(o);
+            }
+
+            @Override public void add(Object o) {
+                delegate.add(o);
+            }
+        };
+    }
+
+    /** {@inheritDoc} */
+    @Override public Iterator<Object> iterator() {
+        ensureDelegateInit();
+
+        return PortableUtils.unwrapLazyIterator(super.iterator());
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeTo(PortableWriterExImpl writer, PortableBuilderSerializer ctx) {
+        if (delegate == null) {
+            int size = reader.readIntAbsolute(off + 1);
+
+            int hdrSize = 1 /* flag */ + 4 /* size */ + 1 /* col type */;
+            writer.write(reader.array(), off, hdrSize);
+
+            reader.position(off + hdrSize);
+
+            for (int i = 0; i < size; i++) {
+                Object o = reader.parseValue();
+
+                ctx.writeValue(writer, o);
+            }
+        }
+        else {
+            writer.writeByte(GridPortableMarshaller.COL);
+            writer.writeInt(delegate.size());
+
+            byte colType = reader.array()[off + 1 /* flag */ + 4 /* size */];
+            writer.writeByte(colType);
+
+            for (Object o : delegate)
+                ctx.writeValue(writer, o);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/9057a4c0/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/PortableLazyMap.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/PortableLazyMap.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/PortableLazyMap.java
new file mode 100644
index 0000000..12cbfd6
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/PortableLazyMap.java
@@ -0,0 +1,220 @@
+/*
+ * 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.AbstractMap;
+import java.util.AbstractSet;
+import java.util.Iterator;
+import java.util.LinkedHashMap;
+import java.util.Map;
+import java.util.Set;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ *
+ */
+class PortableLazyMap extends AbstractMap<Object, Object> implements PortableBuilderSerializationAware {
+    /** */
+    private final PortableBuilderReader reader;
+
+    /** */
+    private final int off;
+
+    /** */
+    private Map<Object, Object> delegate;
+
+    /**
+     * @param reader Reader.
+     * @param off Offset.
+     */
+    private PortableLazyMap(PortableBuilderReader reader, int off) {
+        this.reader = reader;
+        this.off = off;
+    }
+
+    /**
+     * @param reader Reader.
+     * @return PortableLazyMap.
+     */
+    @Nullable public static PortableLazyMap parseMap(PortableBuilderReader reader) {
+        int off = reader.position() - 1;
+
+        int size = reader.readInt();
+
+        reader.skip(1); // map type.
+
+        for (int i = 0; i < size; i++) {
+            reader.skipValue(); // skip key
+            reader.skipValue(); // skip value
+        }
+
+        return new PortableLazyMap(reader, off);
+    }
+
+    /**
+     *
+     */
+    private void ensureDelegateInit() {
+        if (delegate == null) {
+            int size = reader.readIntAbsolute(off + 1);
+
+            reader.position(off + 1/* flag */ + 4/* size */ + 1/* col type */);
+
+            delegate = new LinkedHashMap<>();
+
+            for (int i = 0; i < size; i++)
+                delegate.put(PortableUtils.unwrapLazy(reader.parseValue()), reader.parseValue());
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeTo(PortableWriterExImpl writer, PortableBuilderSerializer ctx) {
+        if (delegate == null) {
+            int size = reader.readIntAbsolute(off + 1);
+
+            int hdrSize = 1 /* flag */ + 4 /* size */ + 1 /* col type */;
+            writer.write(reader.array(), off, hdrSize);
+
+            reader.position(off + hdrSize);
+
+            for (int i = 0; i < size; i++) {
+                ctx.writeValue(writer, reader.parseValue()); // key
+                ctx.writeValue(writer, reader.parseValue()); // value
+            }
+        }
+        else {
+            writer.writeByte(GridPortableMarshaller.MAP);
+            writer.writeInt(delegate.size());
+
+            byte colType = reader.array()[off + 1 /* flag */ + 4 /* size */];
+
+            writer.writeByte(colType);
+
+            for (Entry<Object, Object> entry : delegate.entrySet()) {
+                ctx.writeValue(writer, entry.getKey());
+                ctx.writeValue(writer, entry.getValue());
+            }
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public int size() {
+        if (delegate == null)
+            return reader.readIntAbsolute(off + 1);
+
+        return delegate.size();
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean containsKey(Object key) {
+        ensureDelegateInit();
+
+        return delegate.containsKey(key);
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean containsValue(Object val) {
+        return values().contains(val);
+    }
+
+    /** {@inheritDoc} */
+    @Override public Set<Object> keySet() {
+        ensureDelegateInit();
+
+        return delegate.keySet();
+    }
+
+    /** {@inheritDoc} */
+    @Override public void clear() {
+        if (delegate == null)
+            delegate = new LinkedHashMap<>();
+        else
+            delegate.clear();
+    }
+
+    /** {@inheritDoc} */
+    @Override public Object get(Object key) {
+        ensureDelegateInit();
+
+        return PortableUtils.unwrapLazy(delegate.get(key));
+    }
+
+    /** {@inheritDoc} */
+    @Override public Object put(Object key, Object val) {
+        ensureDelegateInit();
+
+        return PortableUtils.unwrapLazy(delegate.put(key, val));
+    }
+
+    /** {@inheritDoc} */
+    @Override public Object remove(Object key) {
+        ensureDelegateInit();
+
+        return PortableUtils.unwrapLazy(delegate.remove(key));
+    }
+
+    /** {@inheritDoc} */
+    @Override public Set<Entry<Object, Object>> entrySet() {
+        ensureDelegateInit();
+
+        return new AbstractSet<Entry<Object, Object>>() {
+            @Override public boolean contains(Object o) {
+                throw new UnsupportedOperationException();
+            }
+
+            @Override public Iterator<Entry<Object, Object>> iterator() {
+                return new Iterator<Entry<Object, Object>>() {
+                    /** */
+                    private final Iterator<Entry<Object, Object>> itr = delegate.entrySet().iterator();
+
+                    @Override public boolean hasNext() {
+                        return itr.hasNext();
+                    }
+
+                    @Override public Entry<Object, Object> next() {
+                        Entry<Object, Object> res = itr.next();
+
+                        final Object val = res.getValue();
+
+                        if (val instanceof PortableLazyValue) {
+                            return new SimpleEntry<Object, Object>(res.getKey(), val) {
+                                private static final long serialVersionUID = 0L;
+
+                                @Override public Object getValue() {
+                                    return ((PortableLazyValue)val).value();
+                                }
+                            };
+                        }
+
+                        return res;
+                    }
+
+                    @Override public void remove() {
+                        itr.remove();
+                    }
+                };
+            }
+
+            @Override public int size() {
+                return delegate.size();
+            }
+        };
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/9057a4c0/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
new file mode 100644
index 0000000..bd027f5
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/PortableLazyMapEntry.java
@@ -0,0 +1,68 @@
+/*
+ * 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(PortableWriterExImpl 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/9057a4c0/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/PortableLazySet.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/PortableLazySet.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/PortableLazySet.java
new file mode 100644
index 0000000..16772af
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/PortableLazySet.java
@@ -0,0 +1,92 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.portable.builder;
+
+import java.util.Collection;
+import java.util.Set;
+import org.apache.ignite.internal.portable.GridPortableMarshaller;
+import org.apache.ignite.internal.portable.PortableUtils;
+import org.apache.ignite.internal.portable.PortableWriterExImpl;
+import org.apache.ignite.internal.util.typedef.internal.U;
+
+/**
+ *
+ */
+class PortableLazySet extends PortableAbstractLazyValue {
+    /** */
+    private final int off;
+
+    /**
+     * @param reader Reader.
+     * @param size Size.
+     */
+    PortableLazySet(PortableBuilderReader reader, int size) {
+        super(reader, reader.position() - 1);
+
+        off = reader.position() - 1/* flag */ - 4/* size */ - 1/* col type */;
+
+        assert size >= 0;
+
+        for (int i = 0; i < size; i++)
+            reader.skipValue();
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeTo(PortableWriterExImpl writer, PortableBuilderSerializer ctx) {
+        if (val == null) {
+            int size = reader.readIntAbsolute(off + 1);
+
+            int hdrSize = 1 /* flag */ + 4 /* size */ + 1 /* col type */;
+            writer.write(reader.array(), off, hdrSize);
+
+            reader.position(off + hdrSize);
+
+            for (int i = 0; i < size; i++) {
+                Object o = reader.parseValue();
+
+                ctx.writeValue(writer, o);
+            }
+        }
+        else {
+            Collection<Object> c = (Collection<Object>)val;
+
+            writer.writeByte(GridPortableMarshaller.COL);
+            writer.writeInt(c.size());
+
+            byte colType = reader.array()[off + 1 /* flag */ + 4 /* size */];
+            writer.writeByte(colType);
+
+            for (Object o : c)
+                ctx.writeValue(writer, o);
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override protected Object init() {
+        int size = reader.readIntAbsolute(off + 1);
+
+        reader.position(off + 1/* flag */ + 4/* size */ + 1/* col type */);
+
+        Set<Object> res = U.newLinkedHashSet(size);
+
+        for (int i = 0; i < size; i++)
+            res.add(PortableUtils.unwrapLazy(reader.parseValue()));
+
+        return res;
+    }
+}
\ No newline at end of file

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

http://git-wip-us.apache.org/repos/asf/ignite/blob/9057a4c0/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/PortableModifiableLazyValue.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/PortableModifiableLazyValue.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/PortableModifiableLazyValue.java
new file mode 100644
index 0000000..09fb844
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/PortableModifiableLazyValue.java
@@ -0,0 +1,52 @@
+/*
+ * 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.*;
+
+/**
+ *
+ */
+public class PortableModifiableLazyValue extends PortableAbstractLazyValue {
+    /** */
+    protected final int len;
+
+    /**
+     * @param reader
+     * @param valOff
+     * @param len
+     */
+    public PortableModifiableLazyValue(PortableBuilderReader reader, int valOff, int len) {
+        super(reader, valOff);
+
+        this.len = len;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected Object init() {
+        return reader.reader().unmarshal(valOff);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeTo(PortableWriterExImpl writer, PortableBuilderSerializer ctx) {
+        if (val == null)
+            writer.write(reader.array(), valOff, len);
+        else
+            writer.writeObject(val);
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/9057a4c0/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/PortableObjectArrayLazyValue.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/PortableObjectArrayLazyValue.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/PortableObjectArrayLazyValue.java
new file mode 100644
index 0000000..1126a3c
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/PortableObjectArrayLazyValue.java
@@ -0,0 +1,91 @@
+/*
+ * 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.GridPortableMarshaller;
+import org.apache.ignite.internal.portable.PortableWriterExImpl;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.portable.PortableInvalidClassException;
+
+/**
+ *
+ */
+class PortableObjectArrayLazyValue extends PortableAbstractLazyValue {
+    /** */
+    private Object[] lazyValsArr;
+
+    /** */
+    private int compTypeId;
+
+    /** */
+    private String clsName;
+
+    /**
+     * @param reader Reader.
+     */
+    protected PortableObjectArrayLazyValue(PortableBuilderReader reader) {
+        super(reader, reader.position() - 1);
+
+        int typeId = reader.readInt();
+
+        if (typeId == GridPortableMarshaller.UNREGISTERED_TYPE_ID) {
+            clsName = reader.readString();
+
+            Class cls;
+
+            try {
+                // TODO: IGNITE-1272 - Is class loader needed here?
+                cls = U.forName(reader.readString(), null);
+            }
+            catch (ClassNotFoundException e) {
+                throw new PortableInvalidClassException("Failed to load the class: " + clsName, e);
+            }
+
+            compTypeId = reader.portableContext().descriptorForClass(cls).typeId();
+        }
+        else {
+            compTypeId = typeId;
+            clsName = null;
+        }
+
+        int size = reader.readInt();
+
+        lazyValsArr = new Object[size];
+
+        for (int i = 0; i < size; i++)
+            lazyValsArr[i] = reader.parseValue();
+    }
+
+    /** {@inheritDoc} */
+    @Override protected Object init() {
+        for (int i = 0; i < lazyValsArr.length; i++) {
+            if (lazyValsArr[i] instanceof PortableLazyValue)
+                lazyValsArr[i] = ((PortableLazyValue)lazyValsArr[i]).value();
+        }
+
+        return lazyValsArr;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeTo(PortableWriterExImpl writer, PortableBuilderSerializer ctx) {
+        if (clsName == null)
+            ctx.writeArray(writer, GridPortableMarshaller.OBJ_ARR, lazyValsArr, compTypeId);
+        else
+            ctx.writeArray(writer, GridPortableMarshaller.OBJ_ARR, lazyValsArr, clsName);
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/9057a4c0/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/PortablePlainLazyValue.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/PortablePlainLazyValue.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/PortablePlainLazyValue.java
new file mode 100644
index 0000000..136958a
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/PortablePlainLazyValue.java
@@ -0,0 +1,49 @@
+/*
+ * 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.*;
+
+/**
+ *
+ */
+class PortablePlainLazyValue extends PortableAbstractLazyValue {
+    /** */
+    protected final int len;
+
+    /**
+     * @param reader Reader
+     * @param valOff Offset
+     * @param len Length.
+     */
+    protected PortablePlainLazyValue(PortableBuilderReader reader, int valOff, int len) {
+        super(reader, valOff);
+
+        this.len = len;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected Object init() {
+        return reader.reader().unmarshal(valOff);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeTo(PortableWriterExImpl writer, PortableBuilderSerializer ctx) {
+        writer.write(reader.array(), valOff, len);
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/9057a4c0/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/PortablePlainPortableObject.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/PortablePlainPortableObject.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/PortablePlainPortableObject.java
new file mode 100644
index 0000000..8743fbe
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/PortablePlainPortableObject.java
@@ -0,0 +1,53 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.portable.builder;
+
+import org.apache.ignite.internal.portable.PortableObjectImpl;
+import org.apache.ignite.internal.portable.PortableObjectOffheapImpl;
+import org.apache.ignite.internal.portable.PortableWriterExImpl;
+import org.apache.ignite.portable.PortableObject;
+
+/**
+ *
+ */
+public class PortablePlainPortableObject implements PortableLazyValue {
+    /** */
+    private final PortableObject portableObj;
+
+    /**
+     * @param portableObj Portable object.
+     */
+    public PortablePlainPortableObject(PortableObject portableObj) {
+        this.portableObj = portableObj;
+    }
+
+    /** {@inheritDoc} */
+    @Override public Object value() {
+        return portableObj;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeTo(PortableWriterExImpl writer, PortableBuilderSerializer ctx) {
+        PortableObject val = portableObj;
+
+        if (val instanceof PortableObjectOffheapImpl)
+            val = ((PortableObjectOffheapImpl)val).heapCopy();
+
+        writer.doWritePortableObject((PortableObjectImpl)val);
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/9057a4c0/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/PortableValueWithType.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/PortableValueWithType.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/PortableValueWithType.java
new file mode 100644
index 0000000..2e031f0
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/PortableValueWithType.java
@@ -0,0 +1,75 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.portable.builder;
+
+import org.apache.ignite.internal.portable.PortableWriterExImpl;
+import org.apache.ignite.internal.processors.cache.portable.CacheObjectPortableProcessorImpl;
+import org.apache.ignite.internal.util.typedef.internal.S;
+
+/**
+ *
+ */
+class PortableValueWithType implements PortableLazyValue {
+    /** */
+    private byte type;
+
+    /** */
+    private Object val;
+
+    /**
+     * @param type Type
+     * @param val Value.
+     */
+    PortableValueWithType(byte type, Object val) {
+        this.type = type;
+        this.val = val;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeTo(PortableWriterExImpl writer, PortableBuilderSerializer ctx) {
+        if (val instanceof PortableBuilderSerializationAware)
+            ((PortableBuilderSerializationAware)val).writeTo(writer, ctx);
+        else
+            ctx.writeValue(writer, val);
+    }
+
+    /** {@inheritDoc} */
+    public String typeName() {
+        return CacheObjectPortableProcessorImpl.fieldTypeName(type);
+    }
+
+    /** {@inheritDoc} */
+    @Override public Object value() {
+        if (val instanceof PortableLazyValue)
+            return ((PortableLazyValue)val).value();
+
+        return val;
+    }
+
+    /**
+     * @param val New value.
+     */
+    public void value(Object val) {
+        this.val = val;
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return S.toString(PortableValueWithType.class, this);
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/9057a4c0/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/package-info.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/package-info.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/package-info.java
new file mode 100644
index 0000000..e069f3e
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/package-info.java
@@ -0,0 +1,22 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+/**
+ * <!-- Package description. -->
+ * Contains classes specific to portable builder API.
+ */
+package org.apache.ignite.internal.portable.builder;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/9057a4c0/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/portable/CacheObjectPortableProcessorImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/portable/CacheObjectPortableProcessorImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/portable/CacheObjectPortableProcessorImpl.java
index b33c643..1be5aea 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/portable/CacheObjectPortableProcessorImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/portable/CacheObjectPortableProcessorImpl.java
@@ -45,13 +45,13 @@ import org.apache.ignite.cluster.ClusterNode;
 import org.apache.ignite.configuration.CacheConfiguration;
 import org.apache.ignite.internal.GridKernalContext;
 import org.apache.ignite.internal.portable.GridPortableMarshaller;
-import org.apache.ignite.internal.portable.PortableBuilderImpl;
 import org.apache.ignite.internal.portable.PortableContext;
 import org.apache.ignite.internal.portable.PortableMetaDataHandler;
 import org.apache.ignite.internal.portable.PortableMetaDataImpl;
 import org.apache.ignite.internal.portable.PortableObjectImpl;
 import org.apache.ignite.internal.portable.PortableObjectOffheapImpl;
 import org.apache.ignite.internal.portable.PortableUtils;
+import org.apache.ignite.internal.portable.builder.PortableBuilderImpl;
 import org.apache.ignite.internal.portable.streams.PortableInputStream;
 import org.apache.ignite.internal.portable.streams.PortableOffheapInputStream;
 import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;

http://git-wip-us.apache.org/repos/asf/ignite/blob/9057a4c0/modules/core/src/main/resources/META-INF/classnames.properties
----------------------------------------------------------------------
diff --git a/modules/core/src/main/resources/META-INF/classnames.properties b/modules/core/src/main/resources/META-INF/classnames.properties
index d80cc49..dd8c3f3 100644
--- a/modules/core/src/main/resources/META-INF/classnames.properties
+++ b/modules/core/src/main/resources/META-INF/classnames.properties
@@ -281,7 +281,7 @@ org.apache.ignite.internal.managers.loadbalancer.GridLoadBalancerManager$1
 org.apache.ignite.internal.processors.platform.PlatformAwareEventFilter
 org.apache.ignite.internal.portable.PortableClassDescriptor$Mode
 org.apache.ignite.internal.portable.PortableContext
-org.apache.ignite.internal.portable.PortableLazyMap$1$1$1
+org.apache.ignite.internal.portable.builder.PortableLazyMap$1$1$1
 org.apache.ignite.internal.portable.PortableMetaDataImpl
 org.apache.ignite.internal.portable.PortableObjectEx
 org.apache.ignite.internal.portable.PortableObjectImpl


[05/39] ignite git commit: Merge branch 'ignite-993' into ignite-1.3.3-p7

Posted by ag...@apache.org.
Merge branch 'ignite-993' into ignite-1.3.3-p7


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

Branch: refs/heads/ignite-264
Commit: 81e0c401fb5d7a1731621d60e60d5ac4b03435d9
Parents: dadf4bd 3321645
Author: Valentin Kulichenko <va...@gmail.com>
Authored: Tue Sep 1 14:44:47 2015 -0700
Committer: Valentin Kulichenko <va...@gmail.com>
Committed: Tue Sep 1 14:44:47 2015 -0700

----------------------------------------------------------------------
 .../discovery/GridDiscoveryManager.java         |  50 +++--
 .../cache/DynamicCacheChangeRequest.java        |  17 ++
 .../cache/DynamicCacheDescriptor.java           |  14 --
 .../GridCachePartitionExchangeManager.java      |   2 +-
 .../processors/cache/GridCacheProcessor.java    | 187 +++++++++----------
 .../cache/IgniteDynamicCacheStartSelfTest.java  |   2 -
 6 files changed, 137 insertions(+), 135 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/81e0c401/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java
----------------------------------------------------------------------


[13/39] ignite git commit: Javadoc fix (GG-10706) (cherry picked from commit 1225751)

Posted by ag...@apache.org.
Javadoc fix (GG-10706)
(cherry picked from commit 1225751)


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

Branch: refs/heads/ignite-264
Commit: f940db1f04c812b7c9d536ffb35ebf37a3e45420
Parents: 6769956
Author: Anton Vinogradov <av...@apache.org>
Authored: Wed Sep 2 16:26:59 2015 +0300
Committer: Anton Vinogradov <av...@apache.org>
Committed: Wed Sep 2 16:45:49 2015 +0300

----------------------------------------------------------------------
 .../ignite/internal/processors/platform/cache/PlatformCache.java   | 2 +-
 parent/pom.xml                                                     | 1 +
 2 files changed, 2 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/f940db1f/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/cache/PlatformCache.java
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/cache/PlatformCache.java b/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/cache/PlatformCache.java
index 184aa33..27af344 100644
--- a/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/cache/PlatformCache.java
+++ b/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/cache/PlatformCache.java
@@ -704,7 +704,7 @@ public class PlatformCache extends PlatformAbstractTarget {
 
     /**
      * Clears the contents of the cache, without notifying listeners or
-     * {@link javax.cache.integration.CacheWriter}s.
+     * {@ignitelink javax.cache.integration.CacheWriter}s.
      *
      * @throws IllegalStateException if the cache is closed.
      * @throws javax.cache.CacheException if there is a problem during the clear

http://git-wip-us.apache.org/repos/asf/ignite/blob/f940db1f/parent/pom.xml
----------------------------------------------------------------------
diff --git a/parent/pom.xml b/parent/pom.xml
index 2380b08..f56b064 100644
--- a/parent/pom.xml
+++ b/parent/pom.xml
@@ -42,6 +42,7 @@
         <doxygen.exec>doxygen</doxygen.exec>
         <git.exec>git</git.exec>
         <jetty.version>9.2.11.v20150529</jetty.version>
+        <javadoc.opts>-XDenableSunApiLintControl</javadoc.opts>
     </properties>
 
     <groupId>org.apache.ignite</groupId>


[31/39] ignite git commit: Merge remote-tracking branch 'origin/ignite-1.4' into ignite-1.4

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


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

Branch: refs/heads/ignite-264
Commit: 7f4c9ace076ba4e0be5a97bc61a655b8ec11c363
Parents: 5463df2 0cba13c
Author: vozerov-gridgain <vo...@gridgain.com>
Authored: Thu Sep 3 14:28:33 2015 +0300
Committer: vozerov-gridgain <vo...@gridgain.com>
Committed: Thu Sep 3 14:28:33 2015 +0300

----------------------------------------------------------------------
 .../java8/datagrid/CacheAffinityExample.java         | 15 +++++++++++++++
 .../examples/java8/messaging/MessagingExample.java   | 11 ++++++++++-
 .../ignite/java8/examples/BasicExamplesSelfTest.java | 10 +++++++---
 .../ignite/java8/examples/CacheExamplesSelfTest.java |  8 +++++---
 .../java8/examples/CheckpointExamplesSelfTest.java   |  8 ++++----
 .../java8/examples/ClusterGroupExampleSelfTest.java  |  4 +++-
 .../java8/examples/ContinuationExamplesSelfTest.java |  8 +++++---
 .../examples/ContinuousMapperExamplesSelfTest.java   |  8 +++++---
 .../java8/examples/DeploymentExamplesSelfTest.java   |  6 ++++--
 .../java8/examples/EventsExamplesSelfTest.java       |  5 +++--
 .../examples/HibernateL2CacheExampleSelfTest.java    |  8 +++++---
 .../ignite/java8/examples/IgfsExamplesSelfTest.java  |  6 ++++--
 .../java8/examples/LifecycleExamplesSelfTest.java    |  8 +++++---
 .../java8/examples/MemcacheRestExamplesSelfTest.java |  4 +++-
 .../java8/examples/MessagingExamplesSelfTest.java    |  6 ++++--
 .../java8/examples/MonteCarloExamplesSelfTest.java   |  8 +++++---
 .../java8/examples/SpringBeanExamplesSelfTest.java   |  8 +++++---
 .../ignite/java8/examples/TaskExamplesSelfTest.java  |  4 +++-
 .../testsuites/IgniteExamplesJ8SelfTestSuite.java    | 12 ++++++++++--
 19 files changed, 105 insertions(+), 42 deletions(-)
----------------------------------------------------------------------



[29/39] ignite git commit: ignite-1344: Fixed imports at java8-examples

Posted by ag...@apache.org.
ignite-1344: Fixed imports at java8-examples


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

Branch: refs/heads/ignite-264
Commit: 0cba13cc0bfcada072925a15e7cdf79752c6b783
Parents: 27cd615
Author: ashutak <as...@gridgain.com>
Authored: Thu Sep 3 13:54:22 2015 +0300
Committer: ashutak <as...@gridgain.com>
Committed: Thu Sep 3 13:54:22 2015 +0300

----------------------------------------------------------------------
 .../java8/datagrid/CacheAffinityExample.java         | 15 +++++++++++++++
 .../examples/java8/messaging/MessagingExample.java   | 11 ++++++++++-
 .../ignite/java8/examples/BasicExamplesSelfTest.java | 10 +++++++---
 .../ignite/java8/examples/CacheExamplesSelfTest.java |  8 +++++---
 .../java8/examples/CheckpointExamplesSelfTest.java   |  8 ++++----
 .../java8/examples/ClusterGroupExampleSelfTest.java  |  4 +++-
 .../java8/examples/ContinuationExamplesSelfTest.java |  8 +++++---
 .../examples/ContinuousMapperExamplesSelfTest.java   |  8 +++++---
 .../java8/examples/DeploymentExamplesSelfTest.java   |  6 ++++--
 .../java8/examples/EventsExamplesSelfTest.java       |  5 +++--
 .../examples/HibernateL2CacheExampleSelfTest.java    |  8 +++++---
 .../ignite/java8/examples/IgfsExamplesSelfTest.java  |  6 ++++--
 .../java8/examples/LifecycleExamplesSelfTest.java    |  8 +++++---
 .../java8/examples/MemcacheRestExamplesSelfTest.java |  4 +++-
 .../java8/examples/MessagingExamplesSelfTest.java    |  6 ++++--
 .../java8/examples/MonteCarloExamplesSelfTest.java   |  8 +++++---
 .../java8/examples/SpringBeanExamplesSelfTest.java   |  8 +++++---
 .../ignite/java8/examples/TaskExamplesSelfTest.java  |  4 +++-
 .../testsuites/IgniteExamplesJ8SelfTestSuite.java    | 12 ++++++++++--
 19 files changed, 105 insertions(+), 42 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/0cba13cc/examples/src/main/java8/org/apache/ignite/examples/java8/datagrid/CacheAffinityExample.java
----------------------------------------------------------------------
diff --git a/examples/src/main/java8/org/apache/ignite/examples/java8/datagrid/CacheAffinityExample.java b/examples/src/main/java8/org/apache/ignite/examples/java8/datagrid/CacheAffinityExample.java
index 6635e7f..9d9bacc 100644
--- a/examples/src/main/java8/org/apache/ignite/examples/java8/datagrid/CacheAffinityExample.java
+++ b/examples/src/main/java8/org/apache/ignite/examples/java8/datagrid/CacheAffinityExample.java
@@ -17,6 +17,21 @@
 
 package org.apache.ignite.examples.java8.datagrid;
 
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Map;
+import org.apache.ignite.Ignite;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.IgniteCluster;
+import org.apache.ignite.IgniteCompute;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.Ignition;
+import org.apache.ignite.cache.CacheMode;
+import org.apache.ignite.cluster.ClusterNode;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.examples.ExampleNodeStartup;
+import org.apache.ignite.lang.IgniteRunnable;
+
 /**
  * This example demonstrates the simplest code that populates the distributed cache
  * and co-locates simple closure execution with each key. The goal of this particular

http://git-wip-us.apache.org/repos/asf/ignite/blob/0cba13cc/examples/src/main/java8/org/apache/ignite/examples/java8/messaging/MessagingExample.java
----------------------------------------------------------------------
diff --git a/examples/src/main/java8/org/apache/ignite/examples/java8/messaging/MessagingExample.java b/examples/src/main/java8/org/apache/ignite/examples/java8/messaging/MessagingExample.java
index f8f9242..97ec58e 100644
--- a/examples/src/main/java8/org/apache/ignite/examples/java8/messaging/MessagingExample.java
+++ b/examples/src/main/java8/org/apache/ignite/examples/java8/messaging/MessagingExample.java
@@ -17,6 +17,15 @@
 
 package org.apache.ignite.examples.java8.messaging;
 
+import java.util.concurrent.CountDownLatch;
+import org.apache.ignite.Ignite;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.IgniteMessaging;
+import org.apache.ignite.Ignition;
+import org.apache.ignite.cluster.ClusterGroup;
+import org.apache.ignite.examples.ExampleNodeStartup;
+import org.apache.ignite.examples.ExamplesUtils;
+
 /**
  * Example that demonstrates how to exchange messages between nodes. Use such
  * functionality for cases when you need to communicate to other nodes outside
@@ -154,4 +163,4 @@ public final class MessagingExample {
             return unorderedLatch.getCount() > 0;
         });
     }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/0cba13cc/examples/src/test/java8/org/apache/ignite/java8/examples/BasicExamplesSelfTest.java
----------------------------------------------------------------------
diff --git a/examples/src/test/java8/org/apache/ignite/java8/examples/BasicExamplesSelfTest.java b/examples/src/test/java8/org/apache/ignite/java8/examples/BasicExamplesSelfTest.java
index a6bd7c9..cc74cef 100644
--- a/examples/src/test/java8/org/apache/ignite/java8/examples/BasicExamplesSelfTest.java
+++ b/examples/src/test/java8/org/apache/ignite/java8/examples/BasicExamplesSelfTest.java
@@ -17,8 +17,12 @@
 
 package org.apache.ignite.java8.examples;
 
-import org.apache.ignite.examples.java8.computegrid.*;
-import org.apache.ignite.examples.java8.datastructures.*;
+import org.apache.ignite.examples.java8.computegrid.ComputeBroadcastExample;
+import org.apache.ignite.examples.java8.computegrid.ComputeCallableExample;
+import org.apache.ignite.examples.java8.computegrid.ComputeClosureExample;
+import org.apache.ignite.examples.java8.computegrid.ComputeRunnableExample;
+import org.apache.ignite.examples.java8.datastructures.IgniteExecutorServiceExample;
+import org.apache.ignite.testframework.junits.common.GridAbstractExamplesTest;
 
 /**
  * Closure examples self test.
@@ -82,4 +86,4 @@ public class BasicExamplesSelfTest extends GridAbstractExamplesTest {
 //    public void testTaskSplitExample() throws Exception {
 //        ComputeTaskSplitExample.main(EMPTY_ARGS);
 //    }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/0cba13cc/examples/src/test/java8/org/apache/ignite/java8/examples/CacheExamplesSelfTest.java
----------------------------------------------------------------------
diff --git a/examples/src/test/java8/org/apache/ignite/java8/examples/CacheExamplesSelfTest.java b/examples/src/test/java8/org/apache/ignite/java8/examples/CacheExamplesSelfTest.java
index 823210b..4446521 100644
--- a/examples/src/test/java8/org/apache/ignite/java8/examples/CacheExamplesSelfTest.java
+++ b/examples/src/test/java8/org/apache/ignite/java8/examples/CacheExamplesSelfTest.java
@@ -17,7 +17,9 @@
 
 package org.apache.ignite.java8.examples;
 
-import org.apache.ignite.examples.java8.datagrid.*;
+import org.apache.ignite.examples.java8.datagrid.CacheAffinityExample;
+import org.apache.ignite.examples.java8.datagrid.CacheApiExample;
+import org.apache.ignite.testframework.junits.common.GridAbstractExamplesTest;
 
 //import org.apache.ignite.examples.java8.datagrid.starschema.*;
 //import org.apache.ignite.examples.java8.datagrid.store.dummy.*;
@@ -34,7 +36,7 @@ public class CacheExamplesSelfTest extends GridAbstractExamplesTest {
         CacheAffinityExample.main(EMPTY_ARGS);
     }
 
-//    TODO: IGNITE-711 next example(s) should be implemented for java 8 
+//    TODO: IGNITE-711 next example(s) should be implemented for java 8
 //    or testing method(s) should be removed if example(s) does not applicable for java 8.
 //    /**
 //     * @throws Exception If failed.
@@ -142,4 +144,4 @@ public class CacheExamplesSelfTest extends GridAbstractExamplesTest {
 //    public void testCacheContinuousQueryExample() throws Exception {
 //        CacheContinuousQueryExample.main(EMPTY_ARGS);
 //    }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/0cba13cc/examples/src/test/java8/org/apache/ignite/java8/examples/CheckpointExamplesSelfTest.java
----------------------------------------------------------------------
diff --git a/examples/src/test/java8/org/apache/ignite/java8/examples/CheckpointExamplesSelfTest.java b/examples/src/test/java8/org/apache/ignite/java8/examples/CheckpointExamplesSelfTest.java
index d1c5f16..b39fe7f 100644
--- a/examples/src/test/java8/org/apache/ignite/java8/examples/CheckpointExamplesSelfTest.java
+++ b/examples/src/test/java8/org/apache/ignite/java8/examples/CheckpointExamplesSelfTest.java
@@ -17,16 +17,16 @@
 
 package org.apache.ignite.java8.examples;
 
-//import org.apache.ignite.examples.java8.computegrid.failover.*;
+import org.apache.ignite.testframework.junits.common.GridAbstractExamplesTest;
 
 /**
  * Checkpoint examples self test.
  */
 public class CheckpointExamplesSelfTest extends GridAbstractExamplesTest {
     /**
-     * TODO: IGNITE-711 next example(s) should be implemented for java 8 
+     * TODO: IGNITE-711 next example(s) should be implemented for java 8
      * or testing method(s) should be removed if example(s) does not applicable for java 8.
-     *  
+     *
      * Starts remote nodes before each test.
      *
      * Note: using beforeTestsStarted() to start nodes only once won't work.
@@ -44,4 +44,4 @@ public class CheckpointExamplesSelfTest extends GridAbstractExamplesTest {
 //    public void testCheckpointExample() throws Exception {
 //        ComputeFailoverExample.main(EMPTY_ARGS);
 //    }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/0cba13cc/examples/src/test/java8/org/apache/ignite/java8/examples/ClusterGroupExampleSelfTest.java
----------------------------------------------------------------------
diff --git a/examples/src/test/java8/org/apache/ignite/java8/examples/ClusterGroupExampleSelfTest.java b/examples/src/test/java8/org/apache/ignite/java8/examples/ClusterGroupExampleSelfTest.java
index 3dea6ca..34ef68d 100644
--- a/examples/src/test/java8/org/apache/ignite/java8/examples/ClusterGroupExampleSelfTest.java
+++ b/examples/src/test/java8/org/apache/ignite/java8/examples/ClusterGroupExampleSelfTest.java
@@ -17,6 +17,8 @@
 
 package org.apache.ignite.java8.examples;
 
+import org.apache.ignite.testframework.junits.common.GridAbstractExamplesTest;
+
 /**
  *
  */
@@ -35,4 +37,4 @@ public class ClusterGroupExampleSelfTest extends GridAbstractExamplesTest {
 //    public void testComputeClusterGroupsExample() throws Exception {
 //        ClusterGroupExample.main(EMPTY_ARGS);
 //    }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/0cba13cc/examples/src/test/java8/org/apache/ignite/java8/examples/ContinuationExamplesSelfTest.java
----------------------------------------------------------------------
diff --git a/examples/src/test/java8/org/apache/ignite/java8/examples/ContinuationExamplesSelfTest.java b/examples/src/test/java8/org/apache/ignite/java8/examples/ContinuationExamplesSelfTest.java
index 4b4387c..bea6bb6 100644
--- a/examples/src/test/java8/org/apache/ignite/java8/examples/ContinuationExamplesSelfTest.java
+++ b/examples/src/test/java8/org/apache/ignite/java8/examples/ContinuationExamplesSelfTest.java
@@ -19,17 +19,19 @@ package org.apache.ignite.java8.examples;
 
 //import org.apache.ignite.examples.java8.computegrid.*;
 
+import org.apache.ignite.testframework.junits.common.GridAbstractExamplesTest;
+
 /**
  * Continuation example self test.
  */
 public class ContinuationExamplesSelfTest extends GridAbstractExamplesTest {
     /**
-     * TODO: IGNITE-711 next example(s) should be implemented for java 8 
+     * TODO: IGNITE-711 next example(s) should be implemented for java 8
      * or testing method(s) should be removed if example(s) does not applicable for java 8.
-     *  
+     *
      * @throws Exception If failed.
      */
 //    public void testContinuationExample() throws Exception {
 //        ComputeFibonacciContinuationExample.main(EMPTY_ARGS);
 //    }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/0cba13cc/examples/src/test/java8/org/apache/ignite/java8/examples/ContinuousMapperExamplesSelfTest.java
----------------------------------------------------------------------
diff --git a/examples/src/test/java8/org/apache/ignite/java8/examples/ContinuousMapperExamplesSelfTest.java b/examples/src/test/java8/org/apache/ignite/java8/examples/ContinuousMapperExamplesSelfTest.java
index afa1262..f23b6fa 100644
--- a/examples/src/test/java8/org/apache/ignite/java8/examples/ContinuousMapperExamplesSelfTest.java
+++ b/examples/src/test/java8/org/apache/ignite/java8/examples/ContinuousMapperExamplesSelfTest.java
@@ -17,17 +17,19 @@
 
 package org.apache.ignite.java8.examples;
 
+import org.apache.ignite.testframework.junits.common.GridAbstractExamplesTest;
+
 /**
  * ContinuousMapperExample self test.
  */
 public class ContinuousMapperExamplesSelfTest extends GridAbstractExamplesTest {
     /**
-     * TODO: IGNITE-711 next example(s) should be implemented for java 8 
+     * TODO: IGNITE-711 next example(s) should be implemented for java 8
      * or testing method(s) should be removed if example(s) does not applicable for java 8.
-     *  
+     *
      * @throws Exception If failed.
      */
 //    public void testContinuousMapperExample() throws Exception {
 //        ComputeContinuousMapperExample.main(EMPTY_ARGS);
 //    }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/0cba13cc/examples/src/test/java8/org/apache/ignite/java8/examples/DeploymentExamplesSelfTest.java
----------------------------------------------------------------------
diff --git a/examples/src/test/java8/org/apache/ignite/java8/examples/DeploymentExamplesSelfTest.java b/examples/src/test/java8/org/apache/ignite/java8/examples/DeploymentExamplesSelfTest.java
index f56630d..70bddd5 100644
--- a/examples/src/test/java8/org/apache/ignite/java8/examples/DeploymentExamplesSelfTest.java
+++ b/examples/src/test/java8/org/apache/ignite/java8/examples/DeploymentExamplesSelfTest.java
@@ -19,11 +19,13 @@ package org.apache.ignite.java8.examples;
 
 //import org.apache.ignite.examples.java8.misc.deployment.*;
 
+import org.apache.ignite.testframework.junits.common.GridAbstractExamplesTest;
+
 /**
  * Deployment examples self test.
  */
 public class DeploymentExamplesSelfTest extends GridAbstractExamplesTest {
-    // TODO: IGNITE-711 next example(s) should be implemented for java 8 
+    // TODO: IGNITE-711 next example(s) should be implemented for java 8
     // or testing method(s) should be removed if example(s) does not applicable for java 8.
     /**
      * @throws Exception If failed.
@@ -31,4 +33,4 @@ public class DeploymentExamplesSelfTest extends GridAbstractExamplesTest {
 //    public void testDeploymentExample() throws Exception {
 //        DeploymentExample.main(EMPTY_ARGS);
 //    }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/0cba13cc/examples/src/test/java8/org/apache/ignite/java8/examples/EventsExamplesSelfTest.java
----------------------------------------------------------------------
diff --git a/examples/src/test/java8/org/apache/ignite/java8/examples/EventsExamplesSelfTest.java b/examples/src/test/java8/org/apache/ignite/java8/examples/EventsExamplesSelfTest.java
index e7b45a1..09784fe 100644
--- a/examples/src/test/java8/org/apache/ignite/java8/examples/EventsExamplesSelfTest.java
+++ b/examples/src/test/java8/org/apache/ignite/java8/examples/EventsExamplesSelfTest.java
@@ -17,7 +17,8 @@
 
 package org.apache.ignite.java8.examples;
 
-import org.apache.ignite.examples.java8.events.*;
+import org.apache.ignite.examples.java8.events.EventsExample;
+import org.apache.ignite.testframework.junits.common.GridAbstractExamplesTest;
 
 /**
  * Events examples self test.
@@ -29,4 +30,4 @@ public class EventsExamplesSelfTest extends GridAbstractExamplesTest {
     public void testEventsExample() throws Exception {
         EventsExample.main(EMPTY_ARGS);
     }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/0cba13cc/examples/src/test/java8/org/apache/ignite/java8/examples/HibernateL2CacheExampleSelfTest.java
----------------------------------------------------------------------
diff --git a/examples/src/test/java8/org/apache/ignite/java8/examples/HibernateL2CacheExampleSelfTest.java b/examples/src/test/java8/org/apache/ignite/java8/examples/HibernateL2CacheExampleSelfTest.java
index f3c36ab..8c7a2de 100644
--- a/examples/src/test/java8/org/apache/ignite/java8/examples/HibernateL2CacheExampleSelfTest.java
+++ b/examples/src/test/java8/org/apache/ignite/java8/examples/HibernateL2CacheExampleSelfTest.java
@@ -19,17 +19,19 @@ package org.apache.ignite.java8.examples;
 
 //import org.apache.ignite.examples.java8.datagrid.hibernate.*;
 
+import org.apache.ignite.testframework.junits.common.GridAbstractExamplesTest;
+
 /**
  * Tests the {@link org.apache.ignite.examples.java8.datagrid.hibernate.HibernateL2CacheExample}.
  */
 public class HibernateL2CacheExampleSelfTest extends GridAbstractExamplesTest {
     /**
-     * TODO: IGNITE-711 next example(s) should be implemented for java 8 
+     * TODO: IGNITE-711 next example(s) should be implemented for java 8
      * or testing method(s) should be removed if example(s) does not applicable for java 8.
-     * 
+     *
      * @throws Exception If failed.
      */
 //    public void testHibernateL2CacheExample() throws Exception {
 //        HibernateL2CacheExample.main(EMPTY_ARGS);
 //    }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/0cba13cc/examples/src/test/java8/org/apache/ignite/java8/examples/IgfsExamplesSelfTest.java
----------------------------------------------------------------------
diff --git a/examples/src/test/java8/org/apache/ignite/java8/examples/IgfsExamplesSelfTest.java b/examples/src/test/java8/org/apache/ignite/java8/examples/IgfsExamplesSelfTest.java
index 424c197..aed11b3 100644
--- a/examples/src/test/java8/org/apache/ignite/java8/examples/IgfsExamplesSelfTest.java
+++ b/examples/src/test/java8/org/apache/ignite/java8/examples/IgfsExamplesSelfTest.java
@@ -20,6 +20,8 @@ package org.apache.ignite.java8.examples;
 //import org.apache.ignite.examples.java8.igfs.*;
 //import org.apache.ignite.internal.util.typedef.internal.*;
 
+import org.apache.ignite.testframework.junits.common.GridAbstractExamplesTest;
+
 /**
  * IGFS examples self test.
  */
@@ -31,7 +33,7 @@ public class IgfsExamplesSelfTest extends GridAbstractExamplesTest {
     private static final String IGFS_LOOPBACK_CFG = "modules/core/src/test/config/igfs-loopback.xml";
 
     /**
-     * TODO: IGNITE-711 next example(s) should be implemented for java 8 
+     * TODO: IGNITE-711 next example(s) should be implemented for java 8
      * or testing method(s) should be removed if example(s) does not applicable for java 8.
      *
      * @throws Exception If failed.
@@ -50,4 +52,4 @@ public class IgfsExamplesSelfTest extends GridAbstractExamplesTest {
 //            stopAllGrids();
 //        }
 //    }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/0cba13cc/examples/src/test/java8/org/apache/ignite/java8/examples/LifecycleExamplesSelfTest.java
----------------------------------------------------------------------
diff --git a/examples/src/test/java8/org/apache/ignite/java8/examples/LifecycleExamplesSelfTest.java b/examples/src/test/java8/org/apache/ignite/java8/examples/LifecycleExamplesSelfTest.java
index f132e64..ac60659 100644
--- a/examples/src/test/java8/org/apache/ignite/java8/examples/LifecycleExamplesSelfTest.java
+++ b/examples/src/test/java8/org/apache/ignite/java8/examples/LifecycleExamplesSelfTest.java
@@ -19,17 +19,19 @@ package org.apache.ignite.java8.examples;
 
 //import org.apache.ignite.examples.java8.misc.lifecycle.*;
 
+import org.apache.ignite.testframework.junits.common.GridAbstractExamplesTest;
+
 /**
  * LifecycleExample self test.
  */
 public class LifecycleExamplesSelfTest extends GridAbstractExamplesTest {
     /**
-     * TODO: IGNITE-711 next example(s) should be implemented for java 8 
+     * TODO: IGNITE-711 next example(s) should be implemented for java 8
      * or testing method(s) should be removed if example(s) does not applicable for java 8.
-     * 
+     *
      * @throws Exception If failed.
      */
 //    public void testLifecycleExample() throws Exception {
 //        LifecycleExample.main(EMPTY_ARGS);
 //    }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/0cba13cc/examples/src/test/java8/org/apache/ignite/java8/examples/MemcacheRestExamplesSelfTest.java
----------------------------------------------------------------------
diff --git a/examples/src/test/java8/org/apache/ignite/java8/examples/MemcacheRestExamplesSelfTest.java b/examples/src/test/java8/org/apache/ignite/java8/examples/MemcacheRestExamplesSelfTest.java
index 398451e..80caf1f 100644
--- a/examples/src/test/java8/org/apache/ignite/java8/examples/MemcacheRestExamplesSelfTest.java
+++ b/examples/src/test/java8/org/apache/ignite/java8/examples/MemcacheRestExamplesSelfTest.java
@@ -19,6 +19,8 @@ package org.apache.ignite.java8.examples;
 
 //import org.apache.ignite.examples.java8.misc.client.memcache.*;
 
+import org.apache.ignite.testframework.junits.common.GridAbstractExamplesTest;
+
 /**
  * MemcacheRestExample self test.
  */
@@ -39,4 +41,4 @@ public class MemcacheRestExamplesSelfTest extends GridAbstractExamplesTest {
 //    public void testMemcacheRestExample() throws Exception {
 //        MemcacheRestExample.main(EMPTY_ARGS);
 //    }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/0cba13cc/examples/src/test/java8/org/apache/ignite/java8/examples/MessagingExamplesSelfTest.java
----------------------------------------------------------------------
diff --git a/examples/src/test/java8/org/apache/ignite/java8/examples/MessagingExamplesSelfTest.java b/examples/src/test/java8/org/apache/ignite/java8/examples/MessagingExamplesSelfTest.java
index 92b2190..0948a2a 100644
--- a/examples/src/test/java8/org/apache/ignite/java8/examples/MessagingExamplesSelfTest.java
+++ b/examples/src/test/java8/org/apache/ignite/java8/examples/MessagingExamplesSelfTest.java
@@ -17,7 +17,9 @@
 
 package org.apache.ignite.java8.examples;
 
-import org.apache.ignite.examples.java8.messaging.*;
+import org.apache.ignite.examples.java8.messaging.MessagingExample;
+import org.apache.ignite.examples.java8.messaging.MessagingPingPongExample;
+import org.apache.ignite.testframework.junits.common.GridAbstractExamplesTest;
 
 /**
  * Messaging examples self test.
@@ -51,4 +53,4 @@ public class MessagingExamplesSelfTest extends GridAbstractExamplesTest {
 //    public void testMessagingPingPongListenActorExample() throws Exception {
 //        MessagingPingPongListenActorExample.main(EMPTY_ARGS);
 //    }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/0cba13cc/examples/src/test/java8/org/apache/ignite/java8/examples/MonteCarloExamplesSelfTest.java
----------------------------------------------------------------------
diff --git a/examples/src/test/java8/org/apache/ignite/java8/examples/MonteCarloExamplesSelfTest.java b/examples/src/test/java8/org/apache/ignite/java8/examples/MonteCarloExamplesSelfTest.java
index 2173bd6..c6161de 100644
--- a/examples/src/test/java8/org/apache/ignite/java8/examples/MonteCarloExamplesSelfTest.java
+++ b/examples/src/test/java8/org/apache/ignite/java8/examples/MonteCarloExamplesSelfTest.java
@@ -19,6 +19,8 @@ package org.apache.ignite.java8.examples;
 
 //import org.apache.ignite.examples.java8.computegrid.montecarlo.*;
 
+import org.apache.ignite.testframework.junits.common.GridAbstractExamplesTest;
+
 /**
  * Ignite examples self test. Excludes Ignite Spring tests.
  *
@@ -65,12 +67,12 @@ package org.apache.ignite.java8.examples;
  */
 public class MonteCarloExamplesSelfTest extends GridAbstractExamplesTest {
     /**
-     * TODO: IGNITE-711 next example(s) should be implemented for java 8 
+     * TODO: IGNITE-711 next example(s) should be implemented for java 8
      * or testing method(s) should be removed if example(s) does not applicable for java 8.
-     *  
+     *
      * @throws Exception If failed.
      */
 //    public void testCreditRiskExample() throws Exception {
 //        CreditRiskExample.main(EMPTY_ARGS);
 //    }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/0cba13cc/examples/src/test/java8/org/apache/ignite/java8/examples/SpringBeanExamplesSelfTest.java
----------------------------------------------------------------------
diff --git a/examples/src/test/java8/org/apache/ignite/java8/examples/SpringBeanExamplesSelfTest.java b/examples/src/test/java8/org/apache/ignite/java8/examples/SpringBeanExamplesSelfTest.java
index 859eb0a..1ea2e22 100644
--- a/examples/src/test/java8/org/apache/ignite/java8/examples/SpringBeanExamplesSelfTest.java
+++ b/examples/src/test/java8/org/apache/ignite/java8/examples/SpringBeanExamplesSelfTest.java
@@ -19,17 +19,19 @@ package org.apache.ignite.java8.examples;
 
 //import org.apache.ignite.examples.java8.misc.springbean.*;
 
+import org.apache.ignite.testframework.junits.common.GridAbstractExamplesTest;
+
 /**
  * Spring bean examples self test.
  */
 public class SpringBeanExamplesSelfTest extends GridAbstractExamplesTest {
     /**
-     * TODO: IGNITE-711 next example(s) should be implemented for java 8 
+     * TODO: IGNITE-711 next example(s) should be implemented for java 8
      * or testing method(s) should be removed if example(s) does not applicable for java 8.
-     *  
+     *
      * @throws Exception If failed.
      */
 //    public void testSpringBeanHelloWorldExample() throws Exception {
 //        SpringBeanExample.main(EMPTY_ARGS);
 //    }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/0cba13cc/examples/src/test/java8/org/apache/ignite/java8/examples/TaskExamplesSelfTest.java
----------------------------------------------------------------------
diff --git a/examples/src/test/java8/org/apache/ignite/java8/examples/TaskExamplesSelfTest.java b/examples/src/test/java8/org/apache/ignite/java8/examples/TaskExamplesSelfTest.java
index a520292..37520c1 100644
--- a/examples/src/test/java8/org/apache/ignite/java8/examples/TaskExamplesSelfTest.java
+++ b/examples/src/test/java8/org/apache/ignite/java8/examples/TaskExamplesSelfTest.java
@@ -19,6 +19,8 @@ package org.apache.ignite.java8.examples;
 
 //import org.apache.ignite.examples.java8.computegrid.*;
 
+import org.apache.ignite.testframework.junits.common.GridAbstractExamplesTest;
+
 /**
  * Hello world examples self test.
  */
@@ -38,4 +40,4 @@ public class TaskExamplesSelfTest extends GridAbstractExamplesTest {
 //    public void testTaskMapExample() throws Exception {
 //        ComputeTaskMapExample.main(EMPTY_ARGS);
 //    }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/0cba13cc/examples/src/test/java8/org/apache/ignite/java8/testsuites/IgniteExamplesJ8SelfTestSuite.java
----------------------------------------------------------------------
diff --git a/examples/src/test/java8/org/apache/ignite/java8/testsuites/IgniteExamplesJ8SelfTestSuite.java b/examples/src/test/java8/org/apache/ignite/java8/testsuites/IgniteExamplesJ8SelfTestSuite.java
index 1373abe..57e48f9 100644
--- a/examples/src/test/java8/org/apache/ignite/java8/testsuites/IgniteExamplesJ8SelfTestSuite.java
+++ b/examples/src/test/java8/org/apache/ignite/java8/testsuites/IgniteExamplesJ8SelfTestSuite.java
@@ -17,7 +17,15 @@
 
 package org.apache.ignite.java8.testsuites;
 
-import org.apache.ignite.java8.examples.*;
+import junit.framework.TestSuite;
+import org.apache.ignite.java8.examples.BasicExamplesMultiNodeSelfTest;
+import org.apache.ignite.java8.examples.BasicExamplesSelfTest;
+import org.apache.ignite.java8.examples.CacheExamplesMultiNodeSelfTest;
+import org.apache.ignite.java8.examples.CacheExamplesSelfTest;
+import org.apache.ignite.java8.examples.EventsExamplesMultiNodeSelfTest;
+import org.apache.ignite.java8.examples.EventsExamplesSelfTest;
+import org.apache.ignite.java8.examples.MessagingExamplesSelfTest;
+import org.apache.ignite.testframework.GridTestUtils;
 
 import static org.apache.ignite.IgniteSystemProperties.IGNITE_OVERRIDE_MCAST_GRP;
 
@@ -68,4 +76,4 @@ public class IgniteExamplesJ8SelfTestSuite extends TestSuite {
 
         return suite;
     }
-}
\ No newline at end of file
+}


[14/39] ignite git commit: IGNITE-1083 (cherry picked from commit 0214ea9)

Posted by ag...@apache.org.
IGNITE-1083
(cherry picked from commit 0214ea9)


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

Branch: refs/heads/ignite-264
Commit: 2a4839c52c87ae25024fc7a7b971136eeb698307
Parents: f940db1
Author: Anton Vinogradov <av...@apache.org>
Authored: Wed Sep 2 19:51:41 2015 +0300
Committer: Anton Vinogradov <av...@apache.org>
Committed: Wed Sep 2 20:08:43 2015 +0300

----------------------------------------------------------------------
 modules/aop/pom.xml                | 1 +
 modules/apache-license-gen/pom.xml | 1 +
 modules/aws/pom.xml                | 1 +
 modules/clients/pom.xml            | 1 +
 modules/cloud/pom.xml              | 1 +
 modules/codegen/pom.xml            | 1 +
 modules/core/pom.xml               | 1 +
 modules/gce/pom.xml                | 1 +
 modules/geospatial/pom.xml         | 1 +
 modules/hadoop/pom.xml             | 1 +
 modules/hibernate/pom.xml          | 1 +
 modules/indexing/pom.xml           | 1 +
 modules/jcl/pom.xml                | 1 +
 modules/jms11/pom.xml              | 1 +
 modules/jta/pom.xml                | 1 +
 modules/kafka/pom.xml              | 1 +
 modules/log4j/pom.xml              | 1 +
 modules/log4j2/pom.xml             | 1 +
 modules/mesos/pom.xml              | 1 +
 modules/platform/pom.xml           | 1 +
 modules/rest-http/pom.xml          | 1 +
 modules/scalar-2.10/pom.xml        | 1 +
 modules/scalar/pom.xml             | 1 +
 modules/schedule/pom.xml           | 1 +
 modules/schema-import/pom.xml      | 1 +
 modules/slf4j/pom.xml              | 1 +
 modules/spark-2.10/pom.xml         | 1 +
 modules/spark/pom.xml              | 1 +
 modules/spring/pom.xml             | 1 +
 modules/ssh/pom.xml                | 1 +
 modules/tools/pom.xml              | 1 +
 modules/urideploy/pom.xml          | 1 +
 modules/visor-console-2.10/pom.xml | 1 +
 modules/visor-console/pom.xml      | 1 +
 modules/visor-plugins/pom.xml      | 1 +
 modules/web/pom.xml                | 1 +
 modules/yardstick/pom.xml          | 1 +
 modules/yarn/pom.xml               | 1 +
 modules/zookeeper/pom.xml          | 1 +
 39 files changed, 39 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/2a4839c5/modules/aop/pom.xml
----------------------------------------------------------------------
diff --git a/modules/aop/pom.xml b/modules/aop/pom.xml
index ce5cd10..a080b57 100644
--- a/modules/aop/pom.xml
+++ b/modules/aop/pom.xml
@@ -32,6 +32,7 @@
 
     <artifactId>ignite-aop</artifactId>
     <version>1.4.1-SNAPSHOT</version>
+    <url>http://ignite.apache.org</url>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/ignite/blob/2a4839c5/modules/apache-license-gen/pom.xml
----------------------------------------------------------------------
diff --git a/modules/apache-license-gen/pom.xml b/modules/apache-license-gen/pom.xml
index 4054004..2d15ad4 100644
--- a/modules/apache-license-gen/pom.xml
+++ b/modules/apache-license-gen/pom.xml
@@ -33,6 +33,7 @@
     <groupId>org.apache.ignite</groupId>
     <artifactId>ignite-apache-license-gen</artifactId>
     <version>1.4.1-SNAPSHOT</version>
+    <url>http://ignite.apache.org</url>
 
     <build>
         <plugins>

http://git-wip-us.apache.org/repos/asf/ignite/blob/2a4839c5/modules/aws/pom.xml
----------------------------------------------------------------------
diff --git a/modules/aws/pom.xml b/modules/aws/pom.xml
index 8c660ab..63d454d 100644
--- a/modules/aws/pom.xml
+++ b/modules/aws/pom.xml
@@ -32,6 +32,7 @@
 
     <artifactId>ignite-aws</artifactId>
     <version>1.4.1-SNAPSHOT</version>
+    <url>http://ignite.apache.org</url>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/ignite/blob/2a4839c5/modules/clients/pom.xml
----------------------------------------------------------------------
diff --git a/modules/clients/pom.xml b/modules/clients/pom.xml
index 0374bdf..303a274 100644
--- a/modules/clients/pom.xml
+++ b/modules/clients/pom.xml
@@ -32,6 +32,7 @@
 
     <artifactId>ignite-clients</artifactId>
     <version>1.4.1-SNAPSHOT</version>
+    <url>http://ignite.apache.org</url>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/ignite/blob/2a4839c5/modules/cloud/pom.xml
----------------------------------------------------------------------
diff --git a/modules/cloud/pom.xml b/modules/cloud/pom.xml
index 47aedd5..e735804 100644
--- a/modules/cloud/pom.xml
+++ b/modules/cloud/pom.xml
@@ -30,6 +30,7 @@
 
     <artifactId>ignite-cloud</artifactId>
     <version>1.4.1-SNAPSHOT</version>
+    <url>http://ignite.apache.org</url>
 
     <properties>
         <jcloud.version>1.9.0</jcloud.version>

http://git-wip-us.apache.org/repos/asf/ignite/blob/2a4839c5/modules/codegen/pom.xml
----------------------------------------------------------------------
diff --git a/modules/codegen/pom.xml b/modules/codegen/pom.xml
index 261e588..a39da2f 100644
--- a/modules/codegen/pom.xml
+++ b/modules/codegen/pom.xml
@@ -32,6 +32,7 @@
 
     <artifactId>ignite-codegen</artifactId>
     <version>1.4.1-SNAPSHOT</version>
+    <url>http://ignite.apache.org</url>
 
     <properties>
         <project.build.sourceEncoding>UTF-8</project.build.sourceEncoding>

http://git-wip-us.apache.org/repos/asf/ignite/blob/2a4839c5/modules/core/pom.xml
----------------------------------------------------------------------
diff --git a/modules/core/pom.xml b/modules/core/pom.xml
index 698606e..57abba0 100644
--- a/modules/core/pom.xml
+++ b/modules/core/pom.xml
@@ -32,6 +32,7 @@
 
     <artifactId>ignite-core</artifactId>
     <version>1.4.1-SNAPSHOT</version>
+    <url>http://ignite.apache.org</url>
 
     <repositories>
         <repository>

http://git-wip-us.apache.org/repos/asf/ignite/blob/2a4839c5/modules/gce/pom.xml
----------------------------------------------------------------------
diff --git a/modules/gce/pom.xml b/modules/gce/pom.xml
index 0e545bf..b3fbb32 100644
--- a/modules/gce/pom.xml
+++ b/modules/gce/pom.xml
@@ -32,6 +32,7 @@
 
     <artifactId>ignite-gce</artifactId>
     <version>1.4.1-SNAPSHOT</version>
+    <url>http://ignite.apache.org</url>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/ignite/blob/2a4839c5/modules/geospatial/pom.xml
----------------------------------------------------------------------
diff --git a/modules/geospatial/pom.xml b/modules/geospatial/pom.xml
index ba29dcf..1330cf5 100644
--- a/modules/geospatial/pom.xml
+++ b/modules/geospatial/pom.xml
@@ -32,6 +32,7 @@
 
     <artifactId>ignite-geospatial</artifactId>
     <version>1.4.1-SNAPSHOT</version>
+    <url>http://ignite.apache.org</url>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/ignite/blob/2a4839c5/modules/hadoop/pom.xml
----------------------------------------------------------------------
diff --git a/modules/hadoop/pom.xml b/modules/hadoop/pom.xml
index d7f5ac8..9fdfd99 100644
--- a/modules/hadoop/pom.xml
+++ b/modules/hadoop/pom.xml
@@ -32,6 +32,7 @@
 
     <artifactId>ignite-hadoop</artifactId>
     <version>1.4.1-SNAPSHOT</version>
+    <url>http://ignite.apache.org</url>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/ignite/blob/2a4839c5/modules/hibernate/pom.xml
----------------------------------------------------------------------
diff --git a/modules/hibernate/pom.xml b/modules/hibernate/pom.xml
index f8f915b..2d6d893 100644
--- a/modules/hibernate/pom.xml
+++ b/modules/hibernate/pom.xml
@@ -32,6 +32,7 @@
 
     <artifactId>ignite-hibernate</artifactId>
     <version>1.4.1-SNAPSHOT</version>
+    <url>http://ignite.apache.org</url>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/ignite/blob/2a4839c5/modules/indexing/pom.xml
----------------------------------------------------------------------
diff --git a/modules/indexing/pom.xml b/modules/indexing/pom.xml
index ddbd1e3..4bcadc8 100644
--- a/modules/indexing/pom.xml
+++ b/modules/indexing/pom.xml
@@ -32,6 +32,7 @@
 
     <artifactId>ignite-indexing</artifactId>
     <version>1.4.1-SNAPSHOT</version>
+    <url>http://ignite.apache.org</url>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/ignite/blob/2a4839c5/modules/jcl/pom.xml
----------------------------------------------------------------------
diff --git a/modules/jcl/pom.xml b/modules/jcl/pom.xml
index 0c405ee..0b2c48b 100644
--- a/modules/jcl/pom.xml
+++ b/modules/jcl/pom.xml
@@ -32,6 +32,7 @@
 
     <artifactId>ignite-jcl</artifactId>
     <version>1.4.1-SNAPSHOT</version>
+    <url>http://ignite.apache.org</url>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/ignite/blob/2a4839c5/modules/jms11/pom.xml
----------------------------------------------------------------------
diff --git a/modules/jms11/pom.xml b/modules/jms11/pom.xml
index c93ea9a..ff80007 100644
--- a/modules/jms11/pom.xml
+++ b/modules/jms11/pom.xml
@@ -33,6 +33,7 @@
 
     <artifactId>ignite-jms11</artifactId>
     <version>1.4.1-SNAPSHOT</version>
+    <url>http://ignite.apache.org</url>
 
     <properties>
         <activemq.version>5.11.1</activemq.version>

http://git-wip-us.apache.org/repos/asf/ignite/blob/2a4839c5/modules/jta/pom.xml
----------------------------------------------------------------------
diff --git a/modules/jta/pom.xml b/modules/jta/pom.xml
index 2df9bda..7965dd9 100644
--- a/modules/jta/pom.xml
+++ b/modules/jta/pom.xml
@@ -32,6 +32,7 @@
 
     <artifactId>ignite-jta</artifactId>
     <version>1.4.1-SNAPSHOT</version>
+    <url>http://ignite.apache.org</url>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/ignite/blob/2a4839c5/modules/kafka/pom.xml
----------------------------------------------------------------------
diff --git a/modules/kafka/pom.xml b/modules/kafka/pom.xml
index 9b9d95c..ed192ab 100644
--- a/modules/kafka/pom.xml
+++ b/modules/kafka/pom.xml
@@ -32,6 +32,7 @@
 
     <artifactId>ignite-kafka</artifactId>
     <version>1.4.1-SNAPSHOT</version>
+    <url>http://ignite.apache.org</url>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/ignite/blob/2a4839c5/modules/log4j/pom.xml
----------------------------------------------------------------------
diff --git a/modules/log4j/pom.xml b/modules/log4j/pom.xml
index fbba149..c3c4a84 100644
--- a/modules/log4j/pom.xml
+++ b/modules/log4j/pom.xml
@@ -32,6 +32,7 @@
 
     <artifactId>ignite-log4j</artifactId>
     <version>1.4.1-SNAPSHOT</version>
+    <url>http://ignite.apache.org</url>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/ignite/blob/2a4839c5/modules/log4j2/pom.xml
----------------------------------------------------------------------
diff --git a/modules/log4j2/pom.xml b/modules/log4j2/pom.xml
index 58dc05e..0628b47 100644
--- a/modules/log4j2/pom.xml
+++ b/modules/log4j2/pom.xml
@@ -32,6 +32,7 @@
 
     <artifactId>ignite-log4j2</artifactId>
     <version>1.4.1-SNAPSHOT</version>
+    <url>http://ignite.apache.org</url>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/ignite/blob/2a4839c5/modules/mesos/pom.xml
----------------------------------------------------------------------
diff --git a/modules/mesos/pom.xml b/modules/mesos/pom.xml
index 525137f..07bd13d 100644
--- a/modules/mesos/pom.xml
+++ b/modules/mesos/pom.xml
@@ -32,6 +32,7 @@
 
     <artifactId>ignite-mesos</artifactId>
     <version>1.4.1-SNAPSHOT</version>
+    <url>http://ignite.apache.org</url>
 
     <properties>
         <mesos.version>0.22.0</mesos.version>

http://git-wip-us.apache.org/repos/asf/ignite/blob/2a4839c5/modules/platform/pom.xml
----------------------------------------------------------------------
diff --git a/modules/platform/pom.xml b/modules/platform/pom.xml
index e583f6a..4bff370 100644
--- a/modules/platform/pom.xml
+++ b/modules/platform/pom.xml
@@ -32,6 +32,7 @@
 
     <artifactId>ignite-platform</artifactId>
     <version>1.4.1-SNAPSHOT</version>
+    <url>http://ignite.apache.org</url>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/ignite/blob/2a4839c5/modules/rest-http/pom.xml
----------------------------------------------------------------------
diff --git a/modules/rest-http/pom.xml b/modules/rest-http/pom.xml
index 4ce41ae..8156887 100644
--- a/modules/rest-http/pom.xml
+++ b/modules/rest-http/pom.xml
@@ -32,6 +32,7 @@
 
     <artifactId>ignite-rest-http</artifactId>
     <version>1.4.1-SNAPSHOT</version>
+    <url>http://ignite.apache.org</url>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/ignite/blob/2a4839c5/modules/scalar-2.10/pom.xml
----------------------------------------------------------------------
diff --git a/modules/scalar-2.10/pom.xml b/modules/scalar-2.10/pom.xml
index e21ba47..42fb9b9 100644
--- a/modules/scalar-2.10/pom.xml
+++ b/modules/scalar-2.10/pom.xml
@@ -32,6 +32,7 @@
 
     <artifactId>ignite-scalar_2.10</artifactId>
     <version>1.4.1-SNAPSHOT</version>
+    <url>http://ignite.apache.org</url>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/ignite/blob/2a4839c5/modules/scalar/pom.xml
----------------------------------------------------------------------
diff --git a/modules/scalar/pom.xml b/modules/scalar/pom.xml
index 81f4427..982fcaf 100644
--- a/modules/scalar/pom.xml
+++ b/modules/scalar/pom.xml
@@ -32,6 +32,7 @@
 
     <artifactId>ignite-scalar</artifactId>
     <version>1.4.1-SNAPSHOT</version>
+    <url>http://ignite.apache.org</url>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/ignite/blob/2a4839c5/modules/schedule/pom.xml
----------------------------------------------------------------------
diff --git a/modules/schedule/pom.xml b/modules/schedule/pom.xml
index b3e1c18..e64059a 100644
--- a/modules/schedule/pom.xml
+++ b/modules/schedule/pom.xml
@@ -32,6 +32,7 @@
 
     <artifactId>ignite-schedule</artifactId>
     <version>1.4.1-SNAPSHOT</version>
+    <url>http://ignite.apache.org</url>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/ignite/blob/2a4839c5/modules/schema-import/pom.xml
----------------------------------------------------------------------
diff --git a/modules/schema-import/pom.xml b/modules/schema-import/pom.xml
index 7364e1e..0ba8597 100644
--- a/modules/schema-import/pom.xml
+++ b/modules/schema-import/pom.xml
@@ -32,6 +32,7 @@
 
     <artifactId>ignite-schema-import</artifactId>
     <version>1.4.1-SNAPSHOT</version>
+    <url>http://ignite.apache.org</url>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/ignite/blob/2a4839c5/modules/slf4j/pom.xml
----------------------------------------------------------------------
diff --git a/modules/slf4j/pom.xml b/modules/slf4j/pom.xml
index cf98419..5cdf791 100644
--- a/modules/slf4j/pom.xml
+++ b/modules/slf4j/pom.xml
@@ -32,6 +32,7 @@
 
     <artifactId>ignite-slf4j</artifactId>
     <version>1.4.1-SNAPSHOT</version>
+    <url>http://ignite.apache.org</url>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/ignite/blob/2a4839c5/modules/spark-2.10/pom.xml
----------------------------------------------------------------------
diff --git a/modules/spark-2.10/pom.xml b/modules/spark-2.10/pom.xml
index 848d550..6c44006 100644
--- a/modules/spark-2.10/pom.xml
+++ b/modules/spark-2.10/pom.xml
@@ -32,6 +32,7 @@
 
     <artifactId>ignite-spark_2.10</artifactId>
     <version>1.4.1-SNAPSHOT</version>
+    <url>http://ignite.apache.org</url>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/ignite/blob/2a4839c5/modules/spark/pom.xml
----------------------------------------------------------------------
diff --git a/modules/spark/pom.xml b/modules/spark/pom.xml
index 3783a40..1fb2753 100644
--- a/modules/spark/pom.xml
+++ b/modules/spark/pom.xml
@@ -32,6 +32,7 @@
 
     <artifactId>ignite-spark</artifactId>
     <version>1.4.1-SNAPSHOT</version>
+    <url>http://ignite.apache.org</url>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/ignite/blob/2a4839c5/modules/spring/pom.xml
----------------------------------------------------------------------
diff --git a/modules/spring/pom.xml b/modules/spring/pom.xml
index f5563f5..4ed1fd7 100644
--- a/modules/spring/pom.xml
+++ b/modules/spring/pom.xml
@@ -32,6 +32,7 @@
 
     <artifactId>ignite-spring</artifactId>
     <version>1.4.1-SNAPSHOT</version>
+    <url>http://ignite.apache.org</url>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/ignite/blob/2a4839c5/modules/ssh/pom.xml
----------------------------------------------------------------------
diff --git a/modules/ssh/pom.xml b/modules/ssh/pom.xml
index 2d77c80..7922f53 100644
--- a/modules/ssh/pom.xml
+++ b/modules/ssh/pom.xml
@@ -32,6 +32,7 @@
 
     <artifactId>ignite-ssh</artifactId>
     <version>1.4.1-SNAPSHOT</version>
+    <url>http://ignite.apache.org</url>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/ignite/blob/2a4839c5/modules/tools/pom.xml
----------------------------------------------------------------------
diff --git a/modules/tools/pom.xml b/modules/tools/pom.xml
index 30b1a3e..5e3465e 100644
--- a/modules/tools/pom.xml
+++ b/modules/tools/pom.xml
@@ -32,6 +32,7 @@
 
     <artifactId>ignite-tools</artifactId>
     <version>1.4.1-SNAPSHOT</version>
+    <url>http://ignite.apache.org</url>
 
     <properties>
         <project.build.sourceEncoding>UTF-8</project.build.sourceEncoding>

http://git-wip-us.apache.org/repos/asf/ignite/blob/2a4839c5/modules/urideploy/pom.xml
----------------------------------------------------------------------
diff --git a/modules/urideploy/pom.xml b/modules/urideploy/pom.xml
index 94dc956..7dd0431 100644
--- a/modules/urideploy/pom.xml
+++ b/modules/urideploy/pom.xml
@@ -32,6 +32,7 @@
 
     <artifactId>ignite-urideploy</artifactId>
     <version>1.4.1-SNAPSHOT</version>
+    <url>http://ignite.apache.org</url>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/ignite/blob/2a4839c5/modules/visor-console-2.10/pom.xml
----------------------------------------------------------------------
diff --git a/modules/visor-console-2.10/pom.xml b/modules/visor-console-2.10/pom.xml
index 167708b..a0edeaf 100644
--- a/modules/visor-console-2.10/pom.xml
+++ b/modules/visor-console-2.10/pom.xml
@@ -32,6 +32,7 @@
 
     <artifactId>ignite-visor-console_2.10</artifactId>
     <version>1.4.1-SNAPSHOT</version>
+    <url>http://ignite.apache.org</url>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/ignite/blob/2a4839c5/modules/visor-console/pom.xml
----------------------------------------------------------------------
diff --git a/modules/visor-console/pom.xml b/modules/visor-console/pom.xml
index 9cadedb..d6f373a 100644
--- a/modules/visor-console/pom.xml
+++ b/modules/visor-console/pom.xml
@@ -32,6 +32,7 @@
 
     <artifactId>ignite-visor-console</artifactId>
     <version>1.4.1-SNAPSHOT</version>
+    <url>http://ignite.apache.org</url>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/ignite/blob/2a4839c5/modules/visor-plugins/pom.xml
----------------------------------------------------------------------
diff --git a/modules/visor-plugins/pom.xml b/modules/visor-plugins/pom.xml
index ae038cd..04142fa 100644
--- a/modules/visor-plugins/pom.xml
+++ b/modules/visor-plugins/pom.xml
@@ -32,6 +32,7 @@
 
     <artifactId>ignite-visor-plugins</artifactId>
     <version>1.4.1-SNAPSHOT</version>
+    <url>http://ignite.apache.org</url>
 
     <dependencies>
         <!-- Ignite dependencies -->

http://git-wip-us.apache.org/repos/asf/ignite/blob/2a4839c5/modules/web/pom.xml
----------------------------------------------------------------------
diff --git a/modules/web/pom.xml b/modules/web/pom.xml
index a682210..4432d81 100644
--- a/modules/web/pom.xml
+++ b/modules/web/pom.xml
@@ -32,6 +32,7 @@
 
     <artifactId>ignite-web</artifactId>
     <version>1.4.1-SNAPSHOT</version>
+    <url>http://ignite.apache.org</url>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/ignite/blob/2a4839c5/modules/yardstick/pom.xml
----------------------------------------------------------------------
diff --git a/modules/yardstick/pom.xml b/modules/yardstick/pom.xml
index dc4a033..3f3774a 100644
--- a/modules/yardstick/pom.xml
+++ b/modules/yardstick/pom.xml
@@ -32,6 +32,7 @@
 
     <artifactId>ignite-yardstick</artifactId>
     <version>1.4.1-SNAPSHOT</version>
+    <url>http://ignite.apache.org</url>
 
     <properties>
         <yardstick.version>0.7.0</yardstick.version>

http://git-wip-us.apache.org/repos/asf/ignite/blob/2a4839c5/modules/yarn/pom.xml
----------------------------------------------------------------------
diff --git a/modules/yarn/pom.xml b/modules/yarn/pom.xml
index ad17f6b..079a173 100644
--- a/modules/yarn/pom.xml
+++ b/modules/yarn/pom.xml
@@ -32,6 +32,7 @@
 
     <artifactId>ignite-yarn</artifactId>
     <version>1.4.1-SNAPSHOT</version>
+    <url>http://ignite.apache.org</url>
 
     <properties>
         <hadoop.version>2.7.0</hadoop.version>

http://git-wip-us.apache.org/repos/asf/ignite/blob/2a4839c5/modules/zookeeper/pom.xml
----------------------------------------------------------------------
diff --git a/modules/zookeeper/pom.xml b/modules/zookeeper/pom.xml
index bf0c578..ba10101 100644
--- a/modules/zookeeper/pom.xml
+++ b/modules/zookeeper/pom.xml
@@ -32,6 +32,7 @@
 
     <artifactId>ignite-zookeeper</artifactId>
     <version>1.4.1-SNAPSHOT</version>
+    <url>http://ignite.apache.org</url>
 
     <properties>
         <curator.version>2.8.0</curator.version>


[18/39] ignite git commit: ignite-1273: fixed cyclic references processing by PortableMarshaller and ability to modify array fields returned by PortableBuilder

Posted by ag...@apache.org.
http://git-wip-us.apache.org/repos/asf/ignite/blob/6ac9557e/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableLazyMap.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableLazyMap.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableLazyMap.java
deleted file mode 100644
index eed8121..0000000
--- a/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableLazyMap.java
+++ /dev/null
@@ -1,218 +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;
-
-import java.util.AbstractMap;
-import java.util.AbstractSet;
-import java.util.Iterator;
-import java.util.LinkedHashMap;
-import java.util.Map;
-import java.util.Set;
-import org.jetbrains.annotations.Nullable;
-
-/**
- *
- */
-class PortableLazyMap extends AbstractMap<Object, Object> implements PortableBuilderSerializationAware {
-    /** */
-    private final PortableBuilderReader reader;
-
-    /** */
-    private final int off;
-
-    /** */
-    private Map<Object, Object> delegate;
-
-    /**
-     * @param reader Reader.
-     * @param off Offset.
-     */
-    private PortableLazyMap(PortableBuilderReader reader, int off) {
-        this.reader = reader;
-        this.off = off;
-    }
-
-    /**
-     * @param reader Reader.
-     * @return PortableLazyMap.
-     */
-    @Nullable public static PortableLazyMap parseMap(PortableBuilderReader reader) {
-        int off = reader.position() - 1;
-
-        int size = reader.readInt();
-
-        reader.skip(1); // map type.
-
-        for (int i = 0; i < size; i++) {
-            reader.skipValue(); // skip key
-            reader.skipValue(); // skip value
-        }
-
-        return new PortableLazyMap(reader, off);
-    }
-
-    /**
-     *
-     */
-    private void ensureDelegateInit() {
-        if (delegate == null) {
-            int size = reader.readIntAbsolute(off + 1);
-
-            reader.position(off + 1/* flag */ + 4/* size */ + 1/* col type */);
-
-            delegate = new LinkedHashMap<>();
-
-            for (int i = 0; i < size; i++)
-                delegate.put(PortableUtils.unwrapLazy(reader.parseValue()), reader.parseValue());
-        }
-    }
-
-    /** {@inheritDoc} */
-    @Override public void writeTo(PortableWriterExImpl writer, PortableBuilderSerializer ctx) {
-        if (delegate == null) {
-            int size = reader.readIntAbsolute(off + 1);
-
-            int hdrSize = 1 /* flag */ + 4 /* size */ + 1 /* col type */;
-            writer.write(reader.array(), off, hdrSize);
-
-            reader.position(off + hdrSize);
-
-            for (int i = 0; i < size; i++) {
-                ctx.writeValue(writer, reader.parseValue()); // key
-                ctx.writeValue(writer, reader.parseValue()); // value
-            }
-        }
-        else {
-            writer.writeByte(GridPortableMarshaller.MAP);
-            writer.writeInt(delegate.size());
-
-            byte colType = reader.array()[off + 1 /* flag */ + 4 /* size */];
-
-            writer.writeByte(colType);
-
-            for (Entry<Object, Object> entry : delegate.entrySet()) {
-                ctx.writeValue(writer, entry.getKey());
-                ctx.writeValue(writer, entry.getValue());
-            }
-        }
-    }
-
-    /** {@inheritDoc} */
-    @Override public int size() {
-        if (delegate == null)
-            return reader.readIntAbsolute(off + 1);
-
-        return delegate.size();
-    }
-
-    /** {@inheritDoc} */
-    @Override public boolean containsKey(Object key) {
-        ensureDelegateInit();
-
-        return delegate.containsKey(key);
-    }
-
-    /** {@inheritDoc} */
-    @Override public boolean containsValue(Object val) {
-        return values().contains(val);
-    }
-
-    /** {@inheritDoc} */
-    @Override public Set<Object> keySet() {
-        ensureDelegateInit();
-
-        return delegate.keySet();
-    }
-
-    /** {@inheritDoc} */
-    @Override public void clear() {
-        if (delegate == null)
-            delegate = new LinkedHashMap<>();
-        else
-            delegate.clear();
-    }
-
-    /** {@inheritDoc} */
-    @Override public Object get(Object key) {
-        ensureDelegateInit();
-
-        return PortableUtils.unwrapLazy(delegate.get(key));
-    }
-
-    /** {@inheritDoc} */
-    @Override public Object put(Object key, Object val) {
-        ensureDelegateInit();
-
-        return PortableUtils.unwrapLazy(delegate.put(key, val));
-    }
-
-    /** {@inheritDoc} */
-    @Override public Object remove(Object key) {
-        ensureDelegateInit();
-
-        return PortableUtils.unwrapLazy(delegate.remove(key));
-    }
-
-    /** {@inheritDoc} */
-    @Override public Set<Entry<Object, Object>> entrySet() {
-        ensureDelegateInit();
-
-        return new AbstractSet<Entry<Object, Object>>() {
-            @Override public boolean contains(Object o) {
-                throw new UnsupportedOperationException();
-            }
-
-            @Override public Iterator<Entry<Object, Object>> iterator() {
-                return new Iterator<Entry<Object, Object>>() {
-                    /** */
-                    private final Iterator<Entry<Object, Object>> itr = delegate.entrySet().iterator();
-
-                    @Override public boolean hasNext() {
-                        return itr.hasNext();
-                    }
-
-                    @Override public Entry<Object, Object> next() {
-                        Entry<Object, Object> res = itr.next();
-
-                        final Object val = res.getValue();
-
-                        if (val instanceof PortableLazyValue) {
-                            return new SimpleEntry<Object, Object>(res.getKey(), val) {
-                                private static final long serialVersionUID = 0L;
-
-                                @Override public Object getValue() {
-                                    return ((PortableLazyValue)val).value();
-                                }
-                            };
-                        }
-
-                        return res;
-                    }
-
-                    @Override public void remove() {
-                        itr.remove();
-                    }
-                };
-            }
-
-            @Override public int size() {
-                return delegate.size();
-            }
-        };
-    }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/6ac9557e/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableLazyMapEntry.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableLazyMapEntry.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableLazyMapEntry.java
deleted file mode 100644
index 1970d21..0000000
--- a/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableLazyMapEntry.java
+++ /dev/null
@@ -1,66 +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;
-
-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(PortableWriterExImpl writer, PortableBuilderSerializer ctx) {
-        writer.writeByte(GridPortableMarshaller.MAP_ENTRY);
-
-        ctx.writeValue(writer, key);
-        ctx.writeValue(writer, val);
-    }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/6ac9557e/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableLazySet.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableLazySet.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableLazySet.java
deleted file mode 100644
index 3e1dc92..0000000
--- a/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableLazySet.java
+++ /dev/null
@@ -1,89 +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;
-
-import java.util.Collection;
-import java.util.Set;
-import org.apache.ignite.internal.util.typedef.internal.U;
-
-/**
- *
- */
-class PortableLazySet extends PortableAbstractLazyValue {
-    /** */
-    private final int off;
-
-    /**
-     * @param reader Reader.
-     * @param size Size.
-     */
-    PortableLazySet(PortableBuilderReader reader, int size) {
-        super(reader, reader.position() - 1);
-
-        off = reader.position() - 1/* flag */ - 4/* size */ - 1/* col type */;
-
-        assert size >= 0;
-
-        for (int i = 0; i < size; i++)
-            reader.skipValue();
-    }
-
-    /** {@inheritDoc} */
-    @Override public void writeTo(PortableWriterExImpl writer, PortableBuilderSerializer ctx) {
-        if (val == null) {
-            int size = reader.readIntAbsolute(off + 1);
-
-            int hdrSize = 1 /* flag */ + 4 /* size */ + 1 /* col type */;
-            writer.write(reader.array(), off, hdrSize);
-
-            reader.position(off + hdrSize);
-
-            for (int i = 0; i < size; i++) {
-                Object o = reader.parseValue();
-
-                ctx.writeValue(writer, o);
-            }
-        }
-        else {
-            Collection<Object> c = (Collection<Object>)val;
-
-            writer.writeByte(GridPortableMarshaller.COL);
-            writer.writeInt(c.size());
-
-            byte colType = reader.array()[off + 1 /* flag */ + 4 /* size */];
-            writer.writeByte(colType);
-
-            for (Object o : c)
-                ctx.writeValue(writer, o);
-        }
-    }
-
-    /** {@inheritDoc} */
-    @Override protected Object init() {
-        int size = reader.readIntAbsolute(off + 1);
-
-        reader.position(off + 1/* flag */ + 4/* size */ + 1/* col type */);
-
-        Set<Object> res = U.newLinkedHashSet(size);
-
-        for (int i = 0; i < size; i++)
-            res.add(PortableUtils.unwrapLazy(reader.parseValue()));
-
-        return res;
-    }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/6ac9557e/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableLazyValue.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableLazyValue.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableLazyValue.java
deleted file mode 100644
index 43728b7..0000000
--- a/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableLazyValue.java
+++ /dev/null
@@ -1,28 +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;
-
-/**
- *
- */
-interface PortableLazyValue extends PortableBuilderSerializationAware {
-    /**
-     * @return Value.
-     */
-    public Object value();
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/6ac9557e/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableObjectArrayLazyValue.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableObjectArrayLazyValue.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableObjectArrayLazyValue.java
deleted file mode 100644
index 897e8e8..0000000
--- a/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableObjectArrayLazyValue.java
+++ /dev/null
@@ -1,89 +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;
-
-import org.apache.ignite.internal.util.typedef.internal.U;
-import org.apache.ignite.portable.PortableInvalidClassException;
-
-/**
- *
- */
-class PortableObjectArrayLazyValue extends PortableAbstractLazyValue {
-    /** */
-    private Object[] lazyValsArr;
-
-    /** */
-    private int compTypeId;
-
-    /** */
-    private String clsName;
-
-    /**
-     * @param reader Reader.
-     */
-    protected PortableObjectArrayLazyValue(PortableBuilderReader reader) {
-        super(reader, reader.position() - 1);
-
-        int typeId = reader.readInt();
-
-        if (typeId == GridPortableMarshaller.UNREGISTERED_TYPE_ID) {
-            clsName = reader.readString();
-
-            Class cls;
-
-            try {
-                // TODO: IGNITE-1272 - Is class loader needed here?
-                cls = U.forName(reader.readString(), null);
-            }
-            catch (ClassNotFoundException e) {
-                throw new PortableInvalidClassException("Failed to load the class: " + clsName, e);
-            }
-
-            compTypeId = reader.portableContext().descriptorForClass(cls).typeId();
-        }
-        else {
-            compTypeId = typeId;
-            clsName = null;
-        }
-
-        int size = reader.readInt();
-
-        lazyValsArr = new Object[size];
-
-        for (int i = 0; i < size; i++)
-            lazyValsArr[i] = reader.parseValue();
-    }
-
-    /** {@inheritDoc} */
-    @Override protected Object init() {
-        for (int i = 0; i < lazyValsArr.length; i++) {
-            if (lazyValsArr[i] instanceof PortableLazyValue)
-                lazyValsArr[i] = ((PortableLazyValue)lazyValsArr[i]).value();
-        }
-
-        return lazyValsArr;
-    }
-
-    /** {@inheritDoc} */
-    @Override public void writeTo(PortableWriterExImpl writer, PortableBuilderSerializer ctx) {
-        if (clsName == null)
-            ctx.writeArray(writer, GridPortableMarshaller.OBJ_ARR, lazyValsArr, compTypeId);
-        else
-            ctx.writeArray(writer, GridPortableMarshaller.OBJ_ARR, lazyValsArr, clsName);
-    }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/6ac9557e/modules/core/src/main/java/org/apache/ignite/internal/portable/PortablePlainLazyValue.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/PortablePlainLazyValue.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/PortablePlainLazyValue.java
deleted file mode 100644
index d08d09b..0000000
--- a/modules/core/src/main/java/org/apache/ignite/internal/portable/PortablePlainLazyValue.java
+++ /dev/null
@@ -1,47 +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;
-
-/**
- *
- */
-class PortablePlainLazyValue extends PortableAbstractLazyValue {
-    /** */
-    protected final int len;
-
-    /**
-     * @param reader Reader
-     * @param valOff Offset
-     * @param len Length.
-     */
-    protected PortablePlainLazyValue(PortableBuilderReader reader, int valOff, int len) {
-        super(reader, valOff);
-
-        this.len = len;
-    }
-
-    /** {@inheritDoc} */
-    @Override protected Object init() {
-        return reader.reader().unmarshal(valOff);
-    }
-
-    /** {@inheritDoc} */
-    @Override public void writeTo(PortableWriterExImpl writer, PortableBuilderSerializer ctx) {
-        writer.write(reader.array(), valOff, len);
-    }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/6ac9557e/modules/core/src/main/java/org/apache/ignite/internal/portable/PortablePlainPortableObject.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/PortablePlainPortableObject.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/PortablePlainPortableObject.java
deleted file mode 100644
index cfaa04f..0000000
--- a/modules/core/src/main/java/org/apache/ignite/internal/portable/PortablePlainPortableObject.java
+++ /dev/null
@@ -1,50 +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;
-
-import org.apache.ignite.portable.PortableObject;
-
-/**
- *
- */
-public class PortablePlainPortableObject implements PortableLazyValue {
-    /** */
-    private final PortableObject portableObj;
-
-    /**
-     * @param portableObj Portable object.
-     */
-    public PortablePlainPortableObject(PortableObject portableObj) {
-        this.portableObj = portableObj;
-    }
-
-    /** {@inheritDoc} */
-    @Override public Object value() {
-        return portableObj;
-    }
-
-    /** {@inheritDoc} */
-    @Override public void writeTo(PortableWriterExImpl writer, PortableBuilderSerializer ctx) {
-        PortableObject val = portableObj;
-
-        if (val instanceof PortableObjectOffheapImpl)
-            val = ((PortableObjectOffheapImpl)val).heapCopy();
-
-        writer.doWritePortableObject((PortableObjectImpl)val);
-    }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/6ac9557e/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableReaderExImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableReaderExImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableReaderExImpl.java
index f702e06..83ccb65 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableReaderExImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableReaderExImpl.java
@@ -156,7 +156,7 @@ public class PortableReaderExImpl implements PortableReader, PortableRawReaderEx
      * @param start Start.
      * @param ldr Class loader.
      */
-    PortableReaderExImpl(PortableContext ctx, byte[] arr, int start, ClassLoader ldr) {
+    public PortableReaderExImpl(PortableContext ctx, byte[] arr, int start, ClassLoader ldr) {
         this(ctx, new PortableHeapInputStream(arr), start, ldr, new PortableReaderContext());
     }
 
@@ -256,7 +256,7 @@ public class PortableReaderExImpl implements PortableReader, PortableRawReaderEx
      * @return Unmarshalled value.
      * @throws PortableException In case of error.
      */
-    Object unmarshal(int offset) throws PortableException {
+    public Object unmarshal(int offset) throws PortableException {
         off = offset;
 
         return off >= 0 ? unmarshal(false) : null;
@@ -586,6 +586,9 @@ public class PortableReaderExImpl implements PortableReader, PortableRawReaderEx
             if (flag == NULL)
                 return null;
 
+            if (flag == HANDLE)
+                return readHandleField();
+
             if (flag != BYTE_ARR)
                 throw new PortableException("Invalid flag value: " + flag);
 
@@ -609,6 +612,9 @@ public class PortableReaderExImpl implements PortableReader, PortableRawReaderEx
             if (flag == NULL)
                 return null;
 
+            if (flag == HANDLE)
+                return readHandleField();
+
             if (flag != SHORT_ARR)
                 throw new PortableException("Invalid flag value: " + flag);
 
@@ -632,6 +638,9 @@ public class PortableReaderExImpl implements PortableReader, PortableRawReaderEx
             if (flag == NULL)
                 return null;
 
+            if (flag == HANDLE)
+                return readHandleField();
+
             if (flag != INT_ARR)
                 throw new PortableException("Invalid flag value: " + flag);
 
@@ -655,6 +664,9 @@ public class PortableReaderExImpl implements PortableReader, PortableRawReaderEx
             if (flag == NULL)
                 return null;
 
+            if (flag == HANDLE)
+                return readHandleField();
+
             if (flag != LONG_ARR)
                 throw new PortableException("Invalid flag value: " + flag);
 
@@ -678,6 +690,9 @@ public class PortableReaderExImpl implements PortableReader, PortableRawReaderEx
             if (flag == NULL)
                 return null;
 
+            if (flag == HANDLE)
+                return readHandleField();
+
             if (flag != FLOAT_ARR)
                 throw new PortableException("Invalid flag value: " + flag);
 
@@ -701,6 +716,9 @@ public class PortableReaderExImpl implements PortableReader, PortableRawReaderEx
             if (flag == NULL)
                 return null;
 
+            if (flag == HANDLE)
+                return readHandleField();
+
             if (flag != DOUBLE_ARR)
                 throw new PortableException("Invalid flag value: " + flag);
 
@@ -724,6 +742,9 @@ public class PortableReaderExImpl implements PortableReader, PortableRawReaderEx
             if (flag == NULL)
                 return null;
 
+            if (flag == HANDLE)
+                return readHandleField();
+
             if (flag != CHAR_ARR)
                 throw new PortableException("Invalid flag value: " + flag);
 
@@ -747,6 +768,9 @@ public class PortableReaderExImpl implements PortableReader, PortableRawReaderEx
             if (flag == NULL)
                 return null;
 
+            if (flag == HANDLE)
+                return readHandleField();
+
             if (flag != BOOLEAN_ARR)
                 throw new PortableException("Invalid flag value: " + flag);
 
@@ -770,6 +794,9 @@ public class PortableReaderExImpl implements PortableReader, PortableRawReaderEx
             if (flag == NULL)
                 return null;
 
+            if (flag == HANDLE)
+                return readHandleField();
+
             if (flag != DECIMAL_ARR)
                 throw new PortableException("Invalid flag value: " + flag);
 
@@ -793,6 +820,9 @@ public class PortableReaderExImpl implements PortableReader, PortableRawReaderEx
             if (flag == NULL)
                 return null;
 
+            if (flag == HANDLE)
+                return readHandleField();
+
             if (flag != STRING_ARR)
                 throw new PortableException("Invalid flag value: " + flag);
 
@@ -816,6 +846,9 @@ public class PortableReaderExImpl implements PortableReader, PortableRawReaderEx
             if (flag == NULL)
                 return null;
 
+            if (flag == HANDLE)
+                return readHandleField();
+
             if (flag != UUID_ARR)
                 throw new PortableException("Invalid flag value: " + flag);
 
@@ -839,6 +872,9 @@ public class PortableReaderExImpl implements PortableReader, PortableRawReaderEx
             if (flag == NULL)
                 return null;
 
+            if (flag == HANDLE)
+                return readHandleField();
+
             if (flag != DATE_ARR)
                 throw new PortableException("Invalid flag value: " + flag);
 
@@ -862,6 +898,9 @@ public class PortableReaderExImpl implements PortableReader, PortableRawReaderEx
             if (flag == NULL)
                 return null;
 
+            if (flag == HANDLE)
+                return readHandleField();
+
             if (flag != OBJ_ARR)
                 throw new PortableException("Invalid flag value: " + flag);
 
@@ -887,6 +926,9 @@ public class PortableReaderExImpl implements PortableReader, PortableRawReaderEx
             if (flag == NULL)
                 return null;
 
+            if (flag == HANDLE)
+                return readHandleField();
+
             if (flag != COL)
                 throw new PortableException("Invalid flag value: " + flag);
 
@@ -912,6 +954,9 @@ public class PortableReaderExImpl implements PortableReader, PortableRawReaderEx
             if (flag == NULL)
                 return null;
 
+            if (flag == HANDLE)
+                return readHandleField();
+
             if (flag != MAP)
                 throw new PortableException("Invalid flag value: " + flag);
 
@@ -935,10 +980,13 @@ public class PortableReaderExImpl implements PortableReader, PortableRawReaderEx
             if (flag == NULL)
                 return null;
 
+            if (flag == HANDLE)
+                return readHandleField();
+
             if (flag != MAP_ENTRY)
                 throw new PortableException("Invalid flag value: " + flag);
 
-            return new GridMapEntry<>(doReadObject(false), doReadObject(false));
+            return doReadMapEntry(false, true);
         }
         else
             return null;
@@ -1059,6 +1107,33 @@ public class PortableReaderExImpl implements PortableReader, PortableRawReaderEx
         rCtx.setObjectHandler(start, obj);
     }
 
+    /**
+     * @param obj Object.
+     * @param pos Position.
+     */
+    void setHandler(Object obj, int pos) {
+        rCtx.setObjectHandler(pos, obj);
+    }
+
+    /**
+     * Recreating field value from a handle.
+     *
+     * @param <T> Field type.
+     * @return Field.
+     */
+    private <T> T readHandleField() {
+        int handle = (off - 1) - doReadInt(false);
+
+        Object obj = rCtx.getObjectByHandle(handle);
+
+        if (obj == null) {
+            off = handle;
+
+            obj = doReadObject(false);
+        }
+
+        return (T)obj;
+    }
     /** {@inheritDoc} */
     @Override public byte readByte(String fieldName) throws PortableException {
         Byte val = readByte(fieldId(fieldName));
@@ -1676,7 +1751,7 @@ public class PortableReaderExImpl implements PortableReader, PortableRawReaderEx
                 else
                     po = in.offheapPointer() > 0
                         ? new PortableObjectOffheapImpl(ctx, in.offheapPointer(), start,
-                                                            in.remaining() + in.position())
+                        in.remaining() + in.position())
                         : new PortableObjectImpl(ctx, in.array(), start);
 
                 rCtx.setPortableHandler(start, po);
@@ -1805,7 +1880,6 @@ public class PortableReaderExImpl implements PortableReader, PortableRawReaderEx
 
                 return obj;
 
-
             default:
                 throw new PortableException("Invalid flag value: " + flag);
         }
@@ -2306,12 +2380,16 @@ public class PortableReaderExImpl implements PortableReader, PortableRawReaderEx
      * @return Value.
      */
     private byte[] doReadByteArray(boolean raw) {
+        int hPos = (raw ? rawOff : off) - 1;
+
         int len = doReadInt(raw);
 
         in.position(raw ? rawOff : off);
 
         byte[] arr = in.readByteArray(len);
 
+        setHandler(arr, hPos);
+
         if (raw)
             rawOff += len;
         else
@@ -2325,12 +2403,16 @@ public class PortableReaderExImpl implements PortableReader, PortableRawReaderEx
      * @return Value.
      */
     private short[] doReadShortArray(boolean raw) {
+        int hPos = (raw ? rawOff : off) - 1;
+
         int len = doReadInt(raw);
 
         in.position(raw ? rawOff : off);
 
         short[] arr = in.readShortArray(len);
 
+        setHandler(arr, hPos);
+
         int bytes = len << 1;
 
         if (raw)
@@ -2346,12 +2428,16 @@ public class PortableReaderExImpl implements PortableReader, PortableRawReaderEx
      * @return Value.
      */
     private int[] doReadIntArray(boolean raw) {
+        int hPos = (raw ? rawOff : off) - 1;
+
         int len = doReadInt(raw);
 
         in.position(raw ? rawOff : off);
 
         int[] arr = in.readIntArray(len);
 
+        setHandler(arr, hPos);
+
         int bytes = len << 2;
 
         if (raw)
@@ -2367,12 +2453,16 @@ public class PortableReaderExImpl implements PortableReader, PortableRawReaderEx
      * @return Value.
      */
     private long[] doReadLongArray(boolean raw) {
+        int hPos = (raw ? rawOff : off) - 1;
+
         int len = doReadInt(raw);
 
         in.position(raw ? rawOff : off);
 
         long[] arr = in.readLongArray(len);
 
+        setHandler(arr, hPos);
+
         int bytes = len << 3;
 
         if (raw)
@@ -2388,12 +2478,16 @@ public class PortableReaderExImpl implements PortableReader, PortableRawReaderEx
      * @return Value.
      */
     private float[] doReadFloatArray(boolean raw) {
+        int hPos = (raw ? rawOff : off) - 1;
+
         int len = doReadInt(raw);
 
         in.position(raw ? rawOff : off);
 
         float[] arr = in.readFloatArray(len);
 
+        setHandler(arr, hPos);
+
         int bytes = len << 2;
 
         if (raw)
@@ -2409,12 +2503,16 @@ public class PortableReaderExImpl implements PortableReader, PortableRawReaderEx
      * @return Value.
      */
     private double[] doReadDoubleArray(boolean raw) {
+        int hPos = (raw ? rawOff : off) - 1;
+
         int len = doReadInt(raw);
 
         in.position(raw ? rawOff : off);
 
         double[] arr = in.readDoubleArray(len);
 
+        setHandler(arr, hPos);
+
         int bytes = len << 3;
 
         if (raw)
@@ -2430,12 +2528,16 @@ public class PortableReaderExImpl implements PortableReader, PortableRawReaderEx
      * @return Value.
      */
     private char[] doReadCharArray(boolean raw) {
+        int hPos = (raw ? rawOff : off) - 1;
+
         int len = doReadInt(raw);
 
         in.position(raw ? rawOff : off);
 
         char[] arr = in.readCharArray(len);
 
+        setHandler(arr, hPos);
+
         int bytes = len << 1;
 
         if (raw)
@@ -2451,12 +2553,16 @@ public class PortableReaderExImpl implements PortableReader, PortableRawReaderEx
      * @return Value.
      */
     private boolean[] doReadBooleanArray(boolean raw) {
+        int hPos = (raw ? rawOff : off) - 1;
+
         int len = doReadInt(raw);
 
         in.position(raw ? rawOff : off);
 
         boolean[] arr = in.readBooleanArray(len);
 
+        setHandler(arr, hPos);
+
         if (raw)
             rawOff += len;
         else
@@ -2471,10 +2577,14 @@ public class PortableReaderExImpl implements PortableReader, PortableRawReaderEx
      * @throws PortableException In case of error.
      */
     private BigDecimal[] doReadDecimalArray(boolean raw) throws PortableException {
+        int hPos = (raw ? rawOff : off) - 1;
+
         int len = doReadInt(raw);
 
         BigDecimal[] arr = new BigDecimal[len];
 
+        setHandler(arr, hPos);
+
         for (int i = 0; i < len; i++) {
             byte flag = doReadByte(raw);
 
@@ -2497,10 +2607,14 @@ public class PortableReaderExImpl implements PortableReader, PortableRawReaderEx
      * @throws PortableException In case of error.
      */
     private String[] doReadStringArray(boolean raw) throws PortableException {
+        int hPos = (raw ? rawOff : off) - 1;
+
         int len = doReadInt(raw);
 
         String[] arr = new String[len];
 
+        setHandler(arr, hPos);
+
         for (int i = 0; i < len; i++) {
             byte flag = doReadByte(raw);
 
@@ -2523,10 +2637,14 @@ public class PortableReaderExImpl implements PortableReader, PortableRawReaderEx
      * @throws PortableException In case of error.
      */
     private UUID[] doReadUuidArray(boolean raw) throws PortableException {
+        int hPos = (raw ? rawOff : off) - 1;
+
         int len = doReadInt(raw);
 
         UUID[] arr = new UUID[len];
 
+        setHandler(arr, hPos);
+
         for (int i = 0; i < len; i++) {
             byte flag = doReadByte(raw);
 
@@ -2549,10 +2667,14 @@ public class PortableReaderExImpl implements PortableReader, PortableRawReaderEx
      * @throws PortableException In case of error.
      */
     private Date[] doReadDateArray(boolean raw) throws PortableException {
+        int hPos = (raw ? rawOff : off) - 1;
+
         int len = doReadInt(raw);
 
         Date[] arr = new Date[len];
 
+        setHandler(arr, hPos);
+
         for (int i = 0; i < len; i++) {
             byte flag = doReadByte(raw);
 
@@ -2576,12 +2698,16 @@ public class PortableReaderExImpl implements PortableReader, PortableRawReaderEx
      * @throws PortableException In case of error.
      */
     private Object[] doReadObjectArray(boolean raw, boolean deep) throws PortableException {
+        int hPos = (raw ? rawOff : off) - 1;
+
         Class compType = doReadClass(raw);
 
         int len = doReadInt(raw);
 
         Object[] arr = deep ? (Object[])Array.newInstance(compType, len) : new Object[len];
 
+        setHandler(arr, hPos);
+
         for (int i = 0; i < len; i++)
             arr[i] = deep ? doReadObject(raw) : unmarshal(raw);
 
@@ -2598,6 +2724,8 @@ public class PortableReaderExImpl implements PortableReader, PortableRawReaderEx
     @SuppressWarnings("unchecked")
     private Collection<?> doReadCollection(boolean raw, boolean deep, @Nullable Class<? extends Collection> cls)
         throws PortableException {
+        int hPos = (raw ? rawOff : off) - 1;
+
         int size = doReadInt(raw);
 
         assert size >= 0;
@@ -2667,6 +2795,8 @@ public class PortableReaderExImpl implements PortableReader, PortableRawReaderEx
             }
         }
 
+        setHandler(col, hPos);
+
         for (int i = 0; i < size; i++)
             col.add(deep ? doReadObject(raw) : unmarshal(raw));
 
@@ -2683,6 +2813,8 @@ public class PortableReaderExImpl implements PortableReader, PortableRawReaderEx
     @SuppressWarnings("unchecked")
     private Map<?, ?> doReadMap(boolean raw, boolean deep, @Nullable Class<? extends Map> cls)
         throws PortableException {
+        int hPos = (raw ? rawOff : off) - 1;
+
         int size = doReadInt(raw);
 
         assert size >= 0;
@@ -2742,6 +2874,8 @@ public class PortableReaderExImpl implements PortableReader, PortableRawReaderEx
             }
         }
 
+        setHandler(map, hPos);
+
         for (int i = 0; i < size; i++)
             map.put(deep ? doReadObject(raw) : unmarshal(raw), deep ? doReadObject(raw) : unmarshal(raw));
 
@@ -2755,10 +2889,16 @@ public class PortableReaderExImpl implements PortableReader, PortableRawReaderEx
      * @throws PortableException In case of error.
      */
     private Map.Entry<?, ?> doReadMapEntry(boolean raw, boolean deep) throws PortableException {
+        int hPos = (raw ? rawOff : off) - 1;
+
         Object val1 = deep ? doReadObject(raw) : unmarshal(raw);
         Object val2 = deep ? doReadObject(raw) : unmarshal(raw);
 
-        return new GridMapEntry<>(val1, val2);
+        GridMapEntry entry = new GridMapEntry<>(val1, val2);
+
+        setHandler(entry, hPos);
+
+        return entry;
     }
 
     /**
@@ -3017,4 +3157,4 @@ public class PortableReaderExImpl implements PortableReader, PortableRawReaderEx
     @Override public void close() throws IOException {
         // No-op.
     }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/6ac9557e/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 ce77783..7259cc9 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
@@ -33,6 +33,7 @@ import java.util.TreeSet;
 import java.util.UUID;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.ConcurrentSkipListSet;
+import org.apache.ignite.internal.portable.builder.PortableLazyValue;
 import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.portable.PortableObject;
 import org.jetbrains.annotations.Nullable;
@@ -359,6 +360,16 @@ public class PortableUtils {
     }
 
     /**
+     * Checks whether an array type values can or can not contain references to other object.
+     *
+     * @param type Array type.
+     * @return {@code true} if content of serialized array value cannot contain references to other object.
+     */
+    public static boolean isPlainArrayType(int type) {
+        return type >= BYTE_ARR && type <= DATE_ARR;
+    }
+
+    /**
      * @param cls Class.
      * @return Portable field type.
      */

http://git-wip-us.apache.org/repos/asf/ignite/blob/6ac9557e/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableValueWithType.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableValueWithType.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableValueWithType.java
deleted file mode 100644
index ebc68c1..0000000
--- a/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableValueWithType.java
+++ /dev/null
@@ -1,74 +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;
-
-import org.apache.ignite.internal.processors.cache.portable.CacheObjectPortableProcessorImpl;
-import org.apache.ignite.internal.util.typedef.internal.S;
-
-/**
- *
- */
-class PortableValueWithType implements PortableLazyValue {
-    /** */
-    private byte type;
-
-    /** */
-    private Object val;
-
-    /**
-     * @param type Type
-     * @param val Value.
-     */
-    PortableValueWithType(byte type, Object val) {
-        this.type = type;
-        this.val = val;
-    }
-
-    /** {@inheritDoc} */
-    @Override public void writeTo(PortableWriterExImpl writer, PortableBuilderSerializer ctx) {
-        if (val instanceof PortableBuilderSerializationAware)
-            ((PortableBuilderSerializationAware)val).writeTo(writer, ctx);
-        else
-            ctx.writeValue(writer, val);
-    }
-
-    /** {@inheritDoc} */
-    public String typeName() {
-        return CacheObjectPortableProcessorImpl.fieldTypeName(type);
-    }
-
-    /** {@inheritDoc} */
-    @Override public Object value() {
-        if (val instanceof PortableLazyValue)
-            return ((PortableLazyValue)val).value();
-
-        return val;
-    }
-
-    /**
-     * @param val New value.
-     */
-    public void value(Object val) {
-        this.val = val;
-    }
-
-    /** {@inheritDoc} */
-    @Override public String toString() {
-        return S.toString(PortableValueWithType.class, this);
-    }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/6ac9557e/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableWriterExImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableWriterExImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableWriterExImpl.java
index 6bcce2b..364d5f8 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableWriterExImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableWriterExImpl.java
@@ -156,7 +156,7 @@ public class PortableWriterExImpl implements PortableWriter, PortableRawWriterEx
      * @param off Start offset.
      * @param typeId Type ID.
      */
-    PortableWriterExImpl(PortableContext ctx, int off, int typeId, boolean metaEnabled) {
+    public PortableWriterExImpl(PortableContext ctx, int off, int typeId, boolean metaEnabled) {
         this(ctx, off);
 
         this.typeId = typeId;
@@ -320,14 +320,14 @@ public class PortableWriterExImpl implements PortableWriter, PortableRawWriterEx
     /**
      * @return Array.
      */
-    byte[] array() {
+    public byte[] array() {
         return wCtx.out.arrayCopy();
     }
 
     /**
      * @return Output stream.
      */
-    PortableOutputStream outputStream() {
+    public PortableOutputStream outputStream() {
         return wCtx.out;
     }
 
@@ -351,7 +351,7 @@ public class PortableWriterExImpl implements PortableWriter, PortableRawWriterEx
      * @param bytes Number of bytes to reserve.
      * @return Offset.
      */
-    int reserve(int bytes) {
+    public int reserve(int bytes) {
         int pos = wCtx.out.position();
 
         wCtx.out.position(pos + bytes);
@@ -363,7 +363,7 @@ public class PortableWriterExImpl implements PortableWriter, PortableRawWriterEx
      * @param bytes Number of bytes to reserve.
      * @return Offset.
      */
-    int reserveAndMark(int bytes) {
+    public int reserveAndMark(int bytes) {
         int off0 = reserve(bytes);
 
         mark = wCtx.out.position();
@@ -374,21 +374,21 @@ public class PortableWriterExImpl implements PortableWriter, PortableRawWriterEx
     /**
      * @param off Offset.
      */
-    void writeDelta(int off) {
+    public void writeDelta(int off) {
         wCtx.out.writeInt(off, wCtx.out.position() - mark);
     }
 
     /**
      *
      */
-    void writeLength() {
+    public void writeLength() {
         wCtx.out.writeInt(start + TOTAL_LEN_POS, wCtx.out.position() - start);
     }
 
     /**
      *
      */
-    void writeRawOffsetIfNeeded() {
+    public void writeRawOffsetIfNeeded() {
         if (allowFields)
             wCtx.out.writeInt(start + RAW_DATA_OFF_POS, wCtx.out.position() - start);
     }
@@ -416,63 +416,63 @@ public class PortableWriterExImpl implements PortableWriter, PortableRawWriterEx
     /**
      * @param val Value.
      */
-    void doWriteByte(byte val) {
+    public void doWriteByte(byte val) {
         wCtx.out.writeByte(val);
     }
 
     /**
      * @param val Value.
      */
-    void doWriteShort(short val) {
+    public void doWriteShort(short val) {
         wCtx.out.writeShort(val);
     }
 
     /**
      * @param val Value.
      */
-    void doWriteInt(int val) {
+    public void doWriteInt(int val) {
         wCtx.out.writeInt(val);
     }
 
     /**
      * @param val Value.
      */
-    void doWriteLong(long val) {
+    public void doWriteLong(long val) {
         wCtx.out.writeLong(val);
     }
 
     /**
      * @param val Value.
      */
-    void doWriteFloat(float val) {
+    public void doWriteFloat(float val) {
         wCtx.out.writeFloat(val);
     }
 
     /**
      * @param val Value.
      */
-    void doWriteDouble(double val) {
+    public void doWriteDouble(double val) {
         wCtx.out.writeDouble(val);
     }
 
     /**
      * @param val Value.
      */
-    void doWriteChar(char val) {
+    public void doWriteChar(char val) {
         wCtx.out.writeChar(val);
     }
 
     /**
      * @param val Value.
      */
-    void doWriteBoolean(boolean val) {
+    public void doWriteBoolean(boolean val) {
         wCtx.out.writeBoolean(val);
     }
 
     /**
      * @param val String value.
      */
-    void doWriteDecimal(@Nullable BigDecimal val) {
+    public void doWriteDecimal(@Nullable BigDecimal val) {
         if (val == null)
             doWriteByte(NULL);
         else {
@@ -498,7 +498,7 @@ public class PortableWriterExImpl implements PortableWriter, PortableRawWriterEx
     /**
      * @param val String value.
      */
-    void doWriteString(@Nullable String val) {
+    public void doWriteString(@Nullable String val) {
         if (val == null)
             doWriteByte(NULL);
         else {
@@ -528,7 +528,7 @@ public class PortableWriterExImpl implements PortableWriter, PortableRawWriterEx
     /**
      * @param uuid UUID.
      */
-    void doWriteUuid(@Nullable UUID uuid) {
+    public void doWriteUuid(@Nullable UUID uuid) {
         if (uuid == null)
             doWriteByte(NULL);
         else {
@@ -541,7 +541,7 @@ public class PortableWriterExImpl implements PortableWriter, PortableRawWriterEx
     /**
      * @param date Date.
      */
-    void doWriteDate(@Nullable Date date) {
+    public void doWriteDate(@Nullable Date date) {
         if (date == null)
             doWriteByte(NULL);
         else {
@@ -554,7 +554,7 @@ public class PortableWriterExImpl implements PortableWriter, PortableRawWriterEx
     /**
      * @param ts Timestamp.
      */
-    void doWriteTimestamp(@Nullable Timestamp ts) {
+    public void doWriteTimestamp(@Nullable Timestamp ts) {
         if (ts == null)
             doWriteByte(NULL);
         else {
@@ -569,7 +569,7 @@ public class PortableWriterExImpl implements PortableWriter, PortableRawWriterEx
      * @param detached Detached or not.
      * @throws PortableException In case of error.
      */
-    void doWriteObject(@Nullable Object obj, boolean detached) throws PortableException {
+    public void doWriteObject(@Nullable Object obj, boolean detached) throws PortableException {
         if (obj == null)
             doWriteByte(NULL);
         else {
@@ -591,6 +591,9 @@ public class PortableWriterExImpl implements PortableWriter, PortableRawWriterEx
         if (val == null)
             doWriteByte(NULL);
         else {
+            if (tryWriteAsHandle(val))
+                return;
+
             doWriteByte(BYTE_ARR);
             doWriteInt(val.length);
 
@@ -605,6 +608,9 @@ public class PortableWriterExImpl implements PortableWriter, PortableRawWriterEx
         if (val == null)
             doWriteByte(NULL);
         else {
+            if (tryWriteAsHandle(val))
+                return;
+
             doWriteByte(SHORT_ARR);
             doWriteInt(val.length);
 
@@ -619,6 +625,9 @@ public class PortableWriterExImpl implements PortableWriter, PortableRawWriterEx
         if (val == null)
             doWriteByte(NULL);
         else {
+            if (tryWriteAsHandle(val))
+                return;
+
             doWriteByte(INT_ARR);
             doWriteInt(val.length);
 
@@ -633,6 +642,9 @@ public class PortableWriterExImpl implements PortableWriter, PortableRawWriterEx
         if (val == null)
             doWriteByte(NULL);
         else {
+            if (tryWriteAsHandle(val))
+                return;
+
             doWriteByte(LONG_ARR);
             doWriteInt(val.length);
 
@@ -647,6 +659,9 @@ public class PortableWriterExImpl implements PortableWriter, PortableRawWriterEx
         if (val == null)
             doWriteByte(NULL);
         else {
+            if (tryWriteAsHandle(val))
+                return;
+
             doWriteByte(FLOAT_ARR);
             doWriteInt(val.length);
 
@@ -661,6 +676,9 @@ public class PortableWriterExImpl implements PortableWriter, PortableRawWriterEx
         if (val == null)
             doWriteByte(NULL);
         else {
+            if (tryWriteAsHandle(val))
+                return;
+
             doWriteByte(DOUBLE_ARR);
             doWriteInt(val.length);
 
@@ -675,6 +693,9 @@ public class PortableWriterExImpl implements PortableWriter, PortableRawWriterEx
         if (val == null)
             doWriteByte(NULL);
         else {
+            if (tryWriteAsHandle(val))
+                return;
+
             doWriteByte(CHAR_ARR);
             doWriteInt(val.length);
 
@@ -689,6 +710,9 @@ public class PortableWriterExImpl implements PortableWriter, PortableRawWriterEx
         if (val == null)
             doWriteByte(NULL);
         else {
+            if (tryWriteAsHandle(val))
+                return;
+
             doWriteByte(BOOLEAN_ARR);
             doWriteInt(val.length);
 
@@ -703,6 +727,9 @@ public class PortableWriterExImpl implements PortableWriter, PortableRawWriterEx
         if (val == null)
             doWriteByte(NULL);
         else {
+            if (tryWriteAsHandle(val))
+                return;
+
             doWriteByte(DECIMAL_ARR);
             doWriteInt(val.length);
 
@@ -718,6 +745,9 @@ public class PortableWriterExImpl implements PortableWriter, PortableRawWriterEx
         if (val == null)
             doWriteByte(NULL);
         else {
+            if (tryWriteAsHandle(val))
+                return;
+
             doWriteByte(STRING_ARR);
             doWriteInt(val.length);
 
@@ -733,6 +763,9 @@ public class PortableWriterExImpl implements PortableWriter, PortableRawWriterEx
         if (val == null)
             doWriteByte(NULL);
         else {
+            if (tryWriteAsHandle(val))
+                return;
+
             doWriteByte(UUID_ARR);
             doWriteInt(val.length);
 
@@ -748,6 +781,9 @@ public class PortableWriterExImpl implements PortableWriter, PortableRawWriterEx
         if (val == null)
             doWriteByte(NULL);
         else {
+            if (tryWriteAsHandle(val))
+                return;
+
             doWriteByte(DATE_ARR);
             doWriteInt(val.length);
 
@@ -764,6 +800,9 @@ public class PortableWriterExImpl implements PortableWriter, PortableRawWriterEx
         if (val == null)
             doWriteByte(NULL);
         else {
+            if (tryWriteAsHandle(val))
+                return;
+
             PortableContext.Type type = ctx.typeId(val.getClass().getComponentType());
 
             doWriteByte(OBJ_ARR);
@@ -790,6 +829,9 @@ public class PortableWriterExImpl implements PortableWriter, PortableRawWriterEx
         if (col == null)
             doWriteByte(NULL);
         else {
+            if (tryWriteAsHandle(col))
+                return;
+
             doWriteByte(COL);
             doWriteInt(col.size());
             doWriteByte(ctx.collectionType(col.getClass()));
@@ -807,6 +849,9 @@ public class PortableWriterExImpl implements PortableWriter, PortableRawWriterEx
         if (map == null)
             doWriteByte(NULL);
         else {
+            if (tryWriteAsHandle(map))
+                return;
+
             doWriteByte(MAP);
             doWriteInt(map.size());
             doWriteByte(ctx.mapType(map.getClass()));
@@ -826,6 +871,9 @@ public class PortableWriterExImpl implements PortableWriter, PortableRawWriterEx
         if (e == null)
             doWriteByte(NULL);
         else {
+            if (tryWriteAsHandle(e))
+                return;
+
             doWriteByte(MAP_ENTRY);
             doWriteObject(e.getKey(), false);
             doWriteObject(e.getValue(), false);
@@ -905,7 +953,7 @@ public class PortableWriterExImpl implements PortableWriter, PortableRawWriterEx
     /**
      * @param po Portable object.
      */
-    void doWritePortableObject(@Nullable PortableObjectImpl po) {
+    public void doWritePortableObject(@Nullable PortableObjectImpl po) {
         if (po == null)
             doWriteByte(NULL);
         else {
@@ -1106,64 +1154,88 @@ public class PortableWriterExImpl implements PortableWriter, PortableRawWriterEx
      * @param val Value.
      */
     void writeByteArrayField(@Nullable byte[] val) {
-        doWriteInt(val != null ? 5 + val.length : 1);
+        int lenPos = reserveAndMark(4);
+
         doWriteByteArray(val);
+
+        writeDelta(lenPos);
     }
 
     /**
      * @param val Value.
      */
     void writeShortArrayField(@Nullable short[] val) {
-        doWriteInt(val != null ? 5 + (val.length << 1) : 1);
+        int lenPos = reserveAndMark(4);
+
         doWriteShortArray(val);
+
+        writeDelta(lenPos);
     }
 
     /**
      * @param val Value.
      */
     void writeIntArrayField(@Nullable int[] val) {
-        doWriteInt(val != null ? 5 + (val.length << 2) : 1);
+        int lenPos = reserveAndMark(4);
+
         doWriteIntArray(val);
+
+        writeDelta(lenPos);
     }
 
     /**
      * @param val Value.
      */
     void writeLongArrayField(@Nullable long[] val) {
-        doWriteInt(val != null ? 5 + (val.length << 3) : 1);
+        int lenPos = reserveAndMark(4);
+
         doWriteLongArray(val);
+
+        writeDelta(lenPos);
     }
 
     /**
      * @param val Value.
      */
     void writeFloatArrayField(@Nullable float[] val) {
-        doWriteInt(val != null ? 5 + (val.length << 2) : 1);
+        int lenPos = reserveAndMark(4);
+
         doWriteFloatArray(val);
+
+        writeDelta(lenPos);
     }
 
     /**
      * @param val Value.
      */
     void writeDoubleArrayField(@Nullable double[] val) {
-        doWriteInt(val != null ? 5 + (val.length << 3) : 1);
+        int lenPos = reserveAndMark(4);
+
         doWriteDoubleArray(val);
+
+        writeDelta(lenPos);
     }
 
     /**
      * @param val Value.
      */
     void writeCharArrayField(@Nullable char[] val) {
-        doWriteInt(val != null ? 5 + (val.length << 1) : 1);
+        int lenPos = reserveAndMark(4);
+
         doWriteCharArray(val);
+
+        writeDelta(lenPos);
     }
 
     /**
      * @param val Value.
      */
     void writeBooleanArrayField(@Nullable boolean[] val) {
-        doWriteInt(val != null ? 5 + val.length : 1);
+        int lenPos = reserveAndMark(4);
+
         doWriteBooleanArray(val);
+
+        writeDelta(lenPos);
     }
 
     /**
@@ -1739,12 +1811,31 @@ public class PortableWriterExImpl implements PortableWriter, PortableRawWriterEx
         doWriteInt(id);
     }
 
+     /**
+      * Attempts to write the object as a handle.
+      *
+      * @param obj Object to write.
+      * @return {@code true} if the object has been written as a handle.
+      */
+     boolean tryWriteAsHandle(Object obj) {
+         int handle = handle(obj);
+
+         if (handle >= 0) {
+             doWriteByte(GridPortableMarshaller.HANDLE);
+             doWriteInt(handle);
+
+             return true;
+         }
+
+         return false;
+     }
+
     /**
      * Create new writer with same context.
      * @param typeId type
      * @return New writer.
      */
-    PortableWriterExImpl newWriter(int typeId) {
+    public PortableWriterExImpl newWriter(int typeId) {
         PortableWriterExImpl res = new PortableWriterExImpl(ctx, wCtx);
 
         res.typeId = typeId;
@@ -1755,7 +1846,7 @@ public class PortableWriterExImpl implements PortableWriter, PortableRawWriterEx
     /**
      * @return Portable context.
      */
-    PortableContext context() {
+    public PortableContext context() {
         return ctx;
     }
 
@@ -1803,4 +1894,4 @@ public class PortableWriterExImpl implements PortableWriter, PortableRawWriterEx
             handles = new IdentityHashMap<>();
         }
     }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/6ac9557e/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/PortableAbstractLazyValue.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/PortableAbstractLazyValue.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/PortableAbstractLazyValue.java
new file mode 100644
index 0000000..1f521ac
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/PortableAbstractLazyValue.java
@@ -0,0 +1,57 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.portable.builder;
+
+/**
+ *
+ */
+abstract class PortableAbstractLazyValue implements PortableLazyValue {
+    /** */
+    protected Object val;
+
+    /** */
+    protected final PortableBuilderReader reader;
+
+    /** */
+    protected final int valOff;
+
+    /**
+     * @param reader Reader.
+     * @param valOff Value.
+     */
+    protected PortableAbstractLazyValue(PortableBuilderReader reader, int valOff) {
+        this.reader = reader;
+        this.valOff = valOff;
+    }
+
+    /**
+     * @return Value.
+     */
+    protected abstract Object init();
+
+    /** {@inheritDoc} */
+    @Override public Object value() {
+        if (val == null) {
+            val = init();
+
+            assert val != null;
+        }
+
+        return val;
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/6ac9557e/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/PortableBuilderEnum.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/PortableBuilderEnum.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/PortableBuilderEnum.java
new file mode 100644
index 0000000..1472d56
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/PortableBuilderEnum.java
@@ -0,0 +1,116 @@
+/*
+ * 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.GridPortableMarshaller;
+import org.apache.ignite.internal.portable.PortableWriterExImpl;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.portable.PortableInvalidClassException;
+
+/**
+ *
+ */
+public class PortableBuilderEnum implements PortableBuilderSerializationAware {
+    /** */
+    private final int ordinal;
+
+    /** */
+    private final int typeId;
+
+    /** */
+    private final String clsName;
+
+    /**
+     * @param typeId Type ID.
+     * @param anEnum Enum instance.
+     */
+    public PortableBuilderEnum(int typeId, Enum anEnum) {
+        ordinal = anEnum.ordinal();
+        this.typeId = typeId;
+        clsName = null;
+    }
+
+    /**
+     * @param reader PortableBuilderReader.
+     */
+    public PortableBuilderEnum(PortableBuilderReader reader) {
+        int typeId = reader.readInt();
+
+        if (typeId == GridPortableMarshaller.UNREGISTERED_TYPE_ID) {
+            clsName = reader.readString();
+
+            Class cls;
+
+            try {
+                // TODO: IGNITE-1272 - Is class loader needed here?
+                cls = U.forName(reader.readString(), null);
+            }
+            catch (ClassNotFoundException e) {
+                throw new PortableInvalidClassException("Failed to load the class: " + clsName, e);
+            }
+
+            this.typeId = reader.portableContext().descriptorForClass(cls).typeId();
+        }
+        else {
+            this.typeId = typeId;
+            this.clsName = null;
+        }
+
+        ordinal = reader.readInt();
+    }
+
+    /**
+     * @return Ordinal.
+     */
+    public int getOrdinal() {
+        return ordinal;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeTo(PortableWriterExImpl writer, PortableBuilderSerializer ctx) {
+        writer.writeByte(GridPortableMarshaller.ENUM);
+
+        if (typeId == GridPortableMarshaller.UNREGISTERED_TYPE_ID) {
+            writer.writeInt(GridPortableMarshaller.UNREGISTERED_TYPE_ID);
+            writer.writeString(clsName);
+        }
+        else
+            writer.writeInt(typeId);
+
+        writer.writeInt(ordinal);
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean equals(Object o) {
+        if (o == null || getClass() != o.getClass())
+            return false;
+
+        PortableBuilderEnum that = (PortableBuilderEnum)o;
+
+        return ordinal == that.ordinal && typeId == that.typeId;
+    }
+
+    /** {@inheritDoc} */
+    @Override public int hashCode() {
+        int result = ordinal;
+
+        result = 31 * result + typeId;
+
+        return result;
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/6ac9557e/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/PortableBuilderImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/PortableBuilderImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/PortableBuilderImpl.java
new file mode 100644
index 0000000..b2e4c0d
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/PortableBuilderImpl.java
@@ -0,0 +1,537 @@
+/*
+ * 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 java.util.Collections;
+import java.util.HashMap;
+import java.util.LinkedHashMap;
+import java.util.Map;
+import java.util.Set;
+import org.apache.ignite.internal.processors.cache.portable.CacheObjectPortableProcessorImpl;
+import org.apache.ignite.internal.util.GridArgumentCheck;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.portable.PortableBuilder;
+import org.apache.ignite.portable.PortableException;
+import org.apache.ignite.portable.PortableInvalidClassException;
+import org.apache.ignite.portable.PortableMetadata;
+import org.apache.ignite.portable.PortableObject;
+import org.jetbrains.annotations.Nullable;
+import org.apache.ignite.internal.portable.*;
+import org.apache.ignite.internal.processors.cache.portable.*;
+import org.apache.ignite.internal.util.*;
+import org.apache.ignite.internal.util.typedef.internal.*;
+import org.apache.ignite.portable.*;
+
+import static org.apache.ignite.internal.portable.GridPortableMarshaller.CLS_NAME_POS;
+import static org.apache.ignite.internal.portable.GridPortableMarshaller.DFLT_HDR_LEN;
+import static org.apache.ignite.internal.portable.GridPortableMarshaller.HASH_CODE_POS;
+import static org.apache.ignite.internal.portable.GridPortableMarshaller.RAW_DATA_OFF_POS;
+import static org.apache.ignite.internal.portable.GridPortableMarshaller.TOTAL_LEN_POS;
+import static org.apache.ignite.internal.portable.GridPortableMarshaller.TYPE_ID_POS;
+import static org.apache.ignite.internal.portable.GridPortableMarshaller.UNREGISTERED_TYPE_ID;
+
+/**
+ *
+ */
+public class PortableBuilderImpl implements PortableBuilder {
+    /** */
+    private static final Object REMOVED_FIELD_MARKER = new Object();
+
+    /** */
+    private final PortableContext ctx;
+
+    /** */
+    private final int typeId;
+
+    /** May be null. */
+    private String typeName;
+
+    /** May be null. */
+    private String clsNameToWrite;
+
+    /** */
+    private boolean registeredType = true;
+
+    /** */
+    private Map<String, Object> assignedVals;
+
+    /** */
+    private Map<Integer, Object> readCache;
+
+    /** Position of object in source array, or -1 if object is not created from PortableObject. */
+    private final int start;
+
+    /** Total header length */
+    private final int hdrLen;
+
+    /**
+     * Context of PortableObject reading process. Or {@code null} if object is not created from PortableObject.
+     */
+    private final PortableBuilderReader reader;
+
+    /** */
+    private int hashCode;
+
+    /**
+     * @param clsName Class name.
+     * @param ctx Portable context.
+     */
+    public PortableBuilderImpl(PortableContext ctx, String clsName) {
+        this(ctx, ctx.typeId(clsName), PortableContext.typeName(clsName));
+    }
+
+    /**
+     * @param typeId Type ID.
+     * @param ctx Portable context.
+     */
+    public PortableBuilderImpl(PortableContext ctx, int typeId) {
+        this(ctx, typeId, null);
+    }
+
+    /**
+     * @param typeName Type name.
+     * @param ctx Context.
+     * @param typeId Type id.
+     */
+    public PortableBuilderImpl(PortableContext ctx, int typeId, String typeName) {
+        this.typeId = typeId;
+        this.typeName = typeName;
+        this.ctx = ctx;
+
+        start = -1;
+        reader = null;
+        hdrLen = DFLT_HDR_LEN;
+
+        readCache = Collections.emptyMap();
+    }
+
+    /**
+     * @param obj Object to wrap.
+     */
+    public PortableBuilderImpl(PortableObjectImpl obj) {
+        this(new PortableBuilderReader(obj), obj.start());
+
+        reader.registerObject(this);
+    }
+
+    /**
+     * @param reader ctx
+     * @param start Start.
+     */
+    PortableBuilderImpl(PortableBuilderReader reader, int start) {
+        this.reader = reader;
+        this.start = start;
+
+        int typeId = reader.readIntAbsolute(start + TYPE_ID_POS);
+        ctx = reader.portableContext();
+        hashCode = reader.readIntAbsolute(start + HASH_CODE_POS);
+
+        if (typeId == UNREGISTERED_TYPE_ID) {
+            int mark = reader.position();
+
+            reader.position(start + CLS_NAME_POS);
+
+            clsNameToWrite = reader.readString();
+
+            Class cls;
+
+            try {
+                // TODO: IGNITE-1272 - Is class loader needed here?
+                cls = U.forName(clsNameToWrite, null);
+            }
+            catch (ClassNotFoundException e) {
+                throw new PortableInvalidClassException("Failed to load the class: " + clsNameToWrite, e);
+            }
+
+            this.typeId = ctx.descriptorForClass(cls).typeId();
+
+            registeredType = false;
+
+            hdrLen = reader.position() - mark;
+
+            reader.position(mark);
+        }
+        else {
+            this.typeId = typeId;
+            hdrLen = DFLT_HDR_LEN;
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public PortableObject build() {
+        try (PortableWriterExImpl writer = new PortableWriterExImpl(ctx, 0, typeId, false)) {
+
+            PortableBuilderSerializer serializationCtx = new PortableBuilderSerializer();
+
+            serializationCtx.registerObjectWriting(this, 0);
+
+            serializeTo(writer, serializationCtx);
+
+            byte[] arr = writer.array();
+
+            return new PortableObjectImpl(ctx, arr, 0);
+        }
+    }
+
+    /**
+     * @param writer Writer.
+     * @param serializer Serializer.
+     */
+    void serializeTo(PortableWriterExImpl writer, PortableBuilderSerializer serializer) {
+        writer.doWriteByte(GridPortableMarshaller.OBJ);
+        writer.doWriteBoolean(true);
+        writer.doWriteInt(registeredType ? typeId : UNREGISTERED_TYPE_ID);
+        writer.doWriteInt(hashCode);
+
+        // Length and raw offset.
+        writer.reserve(8);
+
+        if (!registeredType)
+            writer.writeString(clsNameToWrite);
+
+        Set<Integer> remainsFlds = null;
+
+        if (reader != null) {
+            Map<Integer, Object> assignedFldsById;
+
+            if (assignedVals != null) {
+                assignedFldsById = U.newHashMap(assignedVals.size());
+
+                for (Map.Entry<String, Object> entry : assignedVals.entrySet()) {
+                    int fldId = ctx.fieldId(typeId, entry.getKey());
+
+                    assignedFldsById.put(fldId, entry.getValue());
+                }
+
+                remainsFlds = assignedFldsById.keySet();
+            }
+            else
+                assignedFldsById = Collections.emptyMap();
+
+            int rawOff = start + reader.readIntAbsolute(start + RAW_DATA_OFF_POS);
+
+            reader.position(start + hdrLen);
+
+            int cpStart = -1;
+
+            while (reader.position() < rawOff) {
+                int fldId = reader.readInt();
+
+                int len = reader.readInt();
+
+                if (assignedFldsById.containsKey(fldId)) {
+                    if (cpStart >= 0) {
+                        writer.write(reader.array(), cpStart, reader.position() - 4 - 4 - cpStart);
+
+                        cpStart = -1;
+                    }
+
+                    Object assignedVal = assignedFldsById.remove(fldId);
+
+                    reader.skip(len);
+
+                    if (assignedVal != REMOVED_FIELD_MARKER) {
+                        writer.writeInt(fldId);
+
+                        int lenPos = writer.reserveAndMark(4);
+
+                        serializer.writeValue(writer, assignedVal);
+
+                        writer.writeDelta(lenPos);
+                    }
+                }
+                else {
+                    int type = len != 0 ? reader.readByte(0) : 0;
+
+                    if (len != 0 && !PortableUtils.isPlainArrayType(type) && PortableUtils.isPlainType(type)) {
+                        if (cpStart < 0)
+                            cpStart = reader.position() - 4 - 4;
+
+                        reader.skip(len);
+                    }
+                    else {
+                        if (cpStart >= 0) {
+                            writer.write(reader.array(), cpStart, reader.position() - 4 - cpStart);
+
+                            cpStart = -1;
+                        }
+                        else
+                            writer.writeInt(fldId);
+
+                        Object val;
+
+                        if (len == 0)
+                            val = null;
+                        else if (readCache == null) {
+                            int savedPos = reader.position();
+
+                            val = reader.parseValue();
+
+                            assert reader.position() == savedPos + len;
+                        }
+                        else {
+                            val = readCache.get(fldId);
+
+                            reader.skip(len);
+                        }
+
+                        int lenPos = writer.reserveAndMark(4);
+
+                        serializer.writeValue(writer, val);
+
+                        writer.writeDelta(lenPos);
+                    }
+                }
+            }
+
+            if (cpStart >= 0)
+                writer.write(reader.array(), cpStart, reader.position() - cpStart);
+        }
+
+        if (assignedVals != null && (remainsFlds == null || !remainsFlds.isEmpty())) {
+            boolean metadataEnabled = ctx.isMetaDataEnabled(typeId);
+
+            PortableMetadata metadata = null;
+
+            if (metadataEnabled)
+                metadata = ctx.metaData(typeId);
+
+            Map<String, String> newFldsMetadata = null;
+
+            for (Map.Entry<String, Object> entry : assignedVals.entrySet()) {
+                Object val = entry.getValue();
+
+                if (val == REMOVED_FIELD_MARKER)
+                    continue;
+
+                String name = entry.getKey();
+
+                int fldId = ctx.fieldId(typeId, name);
+
+                if (remainsFlds != null && !remainsFlds.contains(fldId))
+                    continue;
+
+                writer.writeInt(fldId);
+
+                int lenPos = writer.reserveAndMark(4);
+
+                serializer.writeValue(writer, val);
+
+                writer.writeDelta(lenPos);
+
+                if (metadataEnabled) {
+                    String oldFldTypeName = metadata == null ? null : metadata.fieldTypeName(name);
+
+                    String newFldTypeName;
+
+                    if (val instanceof PortableValueWithType)
+                        newFldTypeName = ((PortableValueWithType)val).typeName();
+                    else {
+                        byte type = PortableUtils.typeByClass(val.getClass());
+
+                        newFldTypeName = CacheObjectPortableProcessorImpl.fieldTypeName(type);
+                    }
+
+                    if (oldFldTypeName == null) {
+                        // It's a new field, we have to add it to metadata.
+
+                        if (newFldsMetadata == null)
+                            newFldsMetadata = new HashMap<>();
+
+                        newFldsMetadata.put(name, newFldTypeName);
+                    }
+                    else {
+                        if (!"Object".equals(oldFldTypeName) && !oldFldTypeName.equals(newFldTypeName)) {
+                            throw new PortableException(
+                                "Wrong value has been set [" +
+                                    "typeName=" + (typeName == null ? metadata.typeName() : typeName) +
+                                    ", fieldName=" + name +
+                                    ", fieldType=" + oldFldTypeName +
+                                    ", assignedValueType=" + newFldTypeName +
+                                    ", assignedValue=" + (((PortableValueWithType)val).value()) + ']'
+                            );
+                        }
+                    }
+                }
+            }
+
+            if (newFldsMetadata != null) {
+                String typeName = this.typeName;
+
+                if (typeName == null)
+                    typeName = metadata.typeName();
+
+                ctx.updateMetaData(typeId, typeName, newFldsMetadata);
+            }
+        }
+
+        writer.writeRawOffsetIfNeeded();
+
+        if (reader != null) {
+            int rawOff = reader.readIntAbsolute(start + RAW_DATA_OFF_POS);
+            int len = reader.readIntAbsolute(start + TOTAL_LEN_POS);
+
+            if (rawOff < len)
+                writer.write(reader.array(), rawOff, len - rawOff);
+        }
+
+        writer.writeLength();
+    }
+
+    /** {@inheritDoc} */
+    @Override public PortableBuilderImpl hashCode(int hashCode) {
+        this.hashCode = hashCode;
+
+        return this;
+    }
+
+    /**
+     *
+     */
+    private void ensureReadCacheInit() {
+        if (readCache == null) {
+            Map<Integer, Object> readCache = new HashMap<>();
+
+            int pos = start + hdrLen;
+            int end = start + reader.readIntAbsolute(start + RAW_DATA_OFF_POS);
+
+            while (pos < end) {
+                int fieldId = reader.readIntAbsolute(pos);
+
+                pos += 4;
+
+                int len = reader.readIntAbsolute(pos);
+
+                pos += 4;
+
+                Object val = reader.getValueQuickly(pos, len);
+
+                readCache.put(fieldId, val);
+
+                pos += len;
+            }
+
+            this.readCache = readCache;
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public <F> F getField(String name) {
+        Object val;
+
+        if (assignedVals != null && assignedVals.containsKey(name)) {
+            val = assignedVals.get(name);
+
+            if (val == REMOVED_FIELD_MARKER)
+                return null;
+        }
+        else {
+            ensureReadCacheInit();
+
+            int fldId = ctx.fieldId(typeId, name);
+
+            val = readCache.get(fldId);
+        }
+
+        return (F)PortableUtils.unwrapLazy(val);
+    }
+
+    /** {@inheritDoc} */
+    @Override public PortableBuilder setField(String name, Object val) {
+        GridArgumentCheck.notNull(val, name);
+
+        if (assignedVals == null)
+            assignedVals = new LinkedHashMap<>();
+
+        Object oldVal = assignedVals.put(name, val);
+
+        if (oldVal instanceof PortableValueWithType) {
+            ((PortableValueWithType)oldVal).value(val);
+
+            assignedVals.put(name, oldVal);
+        }
+
+        return this;
+    }
+
+    /** {@inheritDoc} */
+    @Override public <T> PortableBuilder setField(String name, @Nullable T val, Class<? super T> type) {
+        if (assignedVals == null)
+            assignedVals = new LinkedHashMap<>();
+
+        //int fldId = ctx.fieldId(typeId, fldName);
+
+        assignedVals.put(name, new PortableValueWithType(PortableUtils.typeByClass(type), val));
+
+        return this;
+    }
+
+    /** {@inheritDoc} */
+    @Override public PortableBuilder setField(String name, @Nullable PortableBuilder builder) {
+        if (builder == null)
+            return setField(name, null, Object.class);
+        else
+            return setField(name, (Object)builder);
+    }
+
+    /**
+     * Removes field from portable object.
+     *
+     * @param name Field name.
+     * @return {@code this} instance for chaining.
+     */
+    @Override public PortableBuilderImpl removeField(String name) {
+        if (assignedVals == null)
+            assignedVals = new LinkedHashMap<>();
+
+        assignedVals.put(name, REMOVED_FIELD_MARKER);
+
+        return this;
+    }
+
+    /**
+     * Creates builder initialized by specified portable object.
+     *
+     * @param obj Portable object to initialize builder.
+     * @return New builder.
+     */
+    public static PortableBuilderImpl wrap(PortableObject obj) {
+        PortableObjectImpl heapObj;
+
+        if (obj instanceof PortableObjectOffheapImpl)
+            heapObj = (PortableObjectImpl)((PortableObjectOffheapImpl)obj).heapCopy();
+        else
+            heapObj = (PortableObjectImpl)obj;
+
+        return new PortableBuilderImpl(heapObj);
+    }
+
+    /**
+     * @return Object start position in source array.
+     */
+    int start() {
+        return start;
+    }
+
+    /**
+     * @return Object type id.
+     */
+    public int typeId() {
+        return typeId;
+    }
+}
\ No newline at end of file


[15/39] ignite git commit: IGNITE-1019 - Cache store factory injection

Posted by ag...@apache.org.
IGNITE-1019 - Cache store factory injection


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

Branch: refs/heads/ignite-264
Commit: b8471482613288d607202e8903185ed10cf15f8d
Parents: 2a4839c
Author: sevdokimov <se...@gridgain.com>
Authored: Wed Sep 2 19:59:57 2015 -0700
Committer: Valentin Kulichenko <va...@gmail.com>
Committed: Wed Sep 2 19:59:57 2015 -0700

----------------------------------------------------------------------
 .../GridCacheLoaderWriterStoreFactory.java      |  20 +++-
 .../processors/cache/GridCacheProcessor.java    |  10 +-
 .../store/StoreResourceInjectionSelfTest.java   | 104 +++++++++++++++++++
 .../ignite/testsuites/IgniteCacheTestSuite.java |   4 +-
 4 files changed, 133 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/b8471482/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheLoaderWriterStoreFactory.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheLoaderWriterStoreFactory.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheLoaderWriterStoreFactory.java
index 73573bd..9ba1190 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheLoaderWriterStoreFactory.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheLoaderWriterStoreFactory.java
@@ -5,9 +5,9 @@
  * 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.
@@ -56,4 +56,18 @@ class GridCacheLoaderWriterStoreFactory<K, V> implements Factory<CacheStore<K, V
 
         return new GridCacheLoaderWriterStore<>(ldr, writer);
     }
-}
\ No newline at end of file
+
+    /**
+     * @return Loader factory.
+     */
+    Factory<CacheLoader<K, V>> loaderFactory() {
+        return ldrFactory;
+    }
+
+    /**
+     * @return Writer factory.
+     */
+    Factory<CacheWriter<K, V>> writerFactory() {
+        return writerFactory;
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/b8471482/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 cf8c8d6..75d4c43 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
@@ -1252,7 +1252,15 @@ public class GridCacheProcessor extends GridProcessorAdapter {
     {
         assert cfg != null;
 
-        prepare(cfg, cfg.getCacheStoreFactory(), false);
+        if (cfg.getCacheStoreFactory() instanceof GridCacheLoaderWriterStoreFactory) {
+            GridCacheLoaderWriterStoreFactory factory = (GridCacheLoaderWriterStoreFactory)cfg.getCacheStoreFactory();
+
+            prepare(cfg, factory.loaderFactory(), false);
+            prepare(cfg, factory.writerFactory(), false);
+        }
+        else
+            prepare(cfg, cfg.getCacheStoreFactory(), false);
+
 
         CacheStore cfgStore = cfg.getCacheStoreFactory() != null ? cfg.getCacheStoreFactory().create() : null;
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/b8471482/modules/core/src/test/java/org/apache/ignite/cache/store/StoreResourceInjectionSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/cache/store/StoreResourceInjectionSelfTest.java b/modules/core/src/test/java/org/apache/ignite/cache/store/StoreResourceInjectionSelfTest.java
new file mode 100644
index 0000000..207cbfb
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/cache/store/StoreResourceInjectionSelfTest.java
@@ -0,0 +1,104 @@
+package org.apache.ignite.cache.store;/*
+ * 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.
+ */
+
+import org.apache.ignite.*;
+import org.apache.ignite.configuration.*;
+import org.apache.ignite.internal.processors.cache.*;
+import org.apache.ignite.resources.*;
+import org.apache.ignite.spi.discovery.tcp.*;
+import org.apache.ignite.spi.discovery.tcp.ipfinder.*;
+import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.*;
+import org.apache.ignite.testframework.junits.common.*;
+
+import javax.cache.configuration.*;
+
+/**
+ * Test resource injection.
+ */
+public class StoreResourceInjectionSelfTest extends GridCommonAbstractTest {
+    /** */
+    private static final TcpDiscoveryIpFinder IP_FINDER = new TcpDiscoveryVmIpFinder(true);
+
+    /** */
+    private CacheConfiguration<Integer, String> cacheCfg = new CacheConfiguration<>();
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(gridName);
+
+        cacheCfg.setReadThrough(true);
+        cacheCfg.setWriteThrough(true);
+
+        cfg.setCacheConfiguration(cacheCfg);
+
+        TcpDiscoverySpi disco = new TcpDiscoverySpi();
+
+        disco.setIpFinder(IP_FINDER);
+
+        cfg.setDiscoverySpi(disco);
+
+        return cfg;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTest() throws Exception {
+        stopAllGrids();
+    }
+
+    /**
+     *
+     */
+    public void testResourcesInStoreFactory() throws Exception {
+        cacheCfg.setCacheStoreFactory(new MyCacheStoreFactory());
+
+        startGrid(0);
+    }
+
+    /**
+     *
+     */
+    public void testResourcesInLoaderFactory() throws Exception {
+        cacheCfg.setCacheLoaderFactory(new MyCacheStoreFactory());
+
+        startGrid(0);
+    }
+
+    /**
+     *
+     */
+    public void testResourcesInWriterFactory() throws Exception {
+        cacheCfg.setCacheWriterFactory(new MyCacheStoreFactory());
+
+        startGrid(0);
+    }
+
+    /**
+     *
+     */
+    public static class MyCacheStoreFactory implements Factory<CacheStore<Integer, String>> {
+        /** */
+        @IgniteInstanceResource
+        private Ignite ignite;
+
+        /** {@inheritDoc} */
+        @Override public CacheStore<Integer, String> create() {
+            assert ignite != null;
+
+            return new GridCacheTestStore();
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/b8471482/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite.java
index dac38de..1deb3bc 100644
--- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite.java
+++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite.java
@@ -28,6 +28,7 @@ import org.apache.ignite.cache.affinity.fair.GridFairAffinityFunctionSelfTest;
 import org.apache.ignite.cache.affinity.fair.IgniteFairAffinityDynamicCacheSelfTest;
 import org.apache.ignite.cache.store.GridCacheBalancingStoreSelfTest;
 import org.apache.ignite.cache.store.GridCacheLoadOnlyStoreAdapterSelfTest;
+import org.apache.ignite.cache.store.StoreResourceInjectionSelfTest;
 import org.apache.ignite.cache.store.jdbc.CacheJdbcPojoStoreMultitreadedSelfTest;
 import org.apache.ignite.cache.store.jdbc.CacheJdbcPojoStoreTest;
 import org.apache.ignite.cache.store.jdbc.GridCacheJdbcBlobStoreMultithreadedSelfTest;
@@ -220,6 +221,7 @@ public class IgniteCacheTestSuite extends TestSuite {
         suite.addTestSuite(GridCacheObjectToStringSelfTest.class);
         suite.addTestSuite(GridCacheLoadOnlyStoreAdapterSelfTest.class);
         suite.addTestSuite(GridCacheGetStoreErrorSelfTest.class);
+        suite.addTestSuite(StoreResourceInjectionSelfTest.class);
         suite.addTestSuite(CacheFutureExceptionSelfTest.class);
         suite.addTestSuite(GridCacheAsyncOperationsLimitSelfTest.class);
         suite.addTestSuite(IgniteCacheManyAsyncOperationsTest.class);
@@ -266,4 +268,4 @@ public class IgniteCacheTestSuite extends TestSuite {
 
         return suite;
     }
-}
\ No newline at end of file
+}


[25/39] ignite git commit: IGNITE-1362: Fixed IgniteKernal instance leak to update notifier.

Posted by ag...@apache.org.
IGNITE-1362: Fixed IgniteKernal instance leak to update notifier.


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

Branch: refs/heads/ignite-264
Commit: 0b9d7cab98dddeabdc05e4981457a960c332b26d
Parents: 3a280a0
Author: vozerov-gridgain <vo...@gridgain.com>
Authored: Thu Sep 3 11:25:00 2015 +0300
Committer: vozerov-gridgain <vo...@gridgain.com>
Committed: Thu Sep 3 11:25:00 2015 +0300

----------------------------------------------------------------------
 .../apache/ignite/internal/IgniteKernal.java    | 97 ++++++++++++++------
 1 file changed, 69 insertions(+), 28 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/0b9d7cab/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java b/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java
index d9fef86..9a724df 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java
@@ -28,6 +28,7 @@ import java.io.ObjectStreamException;
 import java.io.Serializable;
 import java.lang.management.ManagementFactory;
 import java.lang.management.RuntimeMXBean;
+import java.lang.ref.WeakReference;
 import java.lang.reflect.Constructor;
 import java.text.DateFormat;
 import java.text.DecimalFormat;
@@ -820,34 +821,8 @@ public class IgniteKernal implements IgniteEx, IgniteMXBean, Externalizable {
                     updateNtfTimer = new Timer("ignite-update-notifier-timer", true);
 
                     // Setup periodic version check.
-                    updateNtfTimer.scheduleAtFixedRate(new GridTimerTask() {
-                        private boolean first = true;
-
-                        @Override public void safeRun() throws InterruptedException {
-                            if (!first)
-                                verChecker.topologySize(cluster().nodes().size());
-
-                            verChecker.checkForNewVersion(execSvc, log);
-
-                            // Just wait for 10 secs.
-                            Thread.sleep(PERIODIC_VER_CHECK_CONN_TIMEOUT);
-
-                            // Just wait another 60 secs in order to get
-                            // version info even on slow connection.
-                            for (int i = 0; i < 60 && verChecker.latestVersion() == null; i++)
-                                Thread.sleep(1000);
-
-                            // Report status if one is available.
-                            // No-op if status is NOT available.
-                            verChecker.reportStatus(log);
-
-                            if (first) {
-                                first = false;
-
-                                verChecker.reportOnlyNew(true);
-                            }
-                        }
-                    }, 0, PERIODIC_VER_CHECK_DELAY);
+                    updateNtfTimer.scheduleAtFixedRate(new UpdateNotifierTimerTask(this, verChecker),
+                        0, PERIODIC_VER_CHECK_DELAY);
                 }
                 catch (IgniteCheckedException e) {
                     if (log.isDebugEnabled())
@@ -3194,4 +3169,70 @@ public class IgniteKernal implements IgniteEx, IgniteMXBean, Externalizable {
     @Override public String toString() {
         return S.toString(IgniteKernal.class, this);
     }
+
+    /**
+     * Update notifier timer task.
+     */
+    private static class UpdateNotifierTimerTask extends GridTimerTask {
+        /** Reference to kernal. */
+        private final WeakReference<IgniteKernal> kernalRef;
+
+        /** Logger. */
+        private final IgniteLogger log;
+
+        /** Executor service. */
+        private final ExecutorService execSvc;
+
+        /** Version checker. */
+        private final GridUpdateNotifier verChecker;
+
+        /** Whether this is the first run. */
+        private boolean first = true;
+
+        /**
+         * Constructor.
+         *
+         * @param kernal Kernal.
+         * @param verChecker Version checker.
+         */
+        private UpdateNotifierTimerTask(IgniteKernal kernal, GridUpdateNotifier verChecker) {
+            kernalRef = new WeakReference<>(kernal);
+
+            log = kernal.log.getLogger(UpdateNotifierTimerTask.class);
+
+            execSvc = kernal.executorService();
+
+            this.verChecker = verChecker;
+        }
+
+        /** {@inheritDoc} */
+        @Override public void safeRun() throws InterruptedException {
+            if (!first) {
+                IgniteKernal kernal = kernalRef.get();
+
+                if (kernal != null)
+                    verChecker.topologySize(kernal.cluster().nodes().size());
+            }
+
+            verChecker.checkForNewVersion(execSvc, log);
+
+            // Just wait for 10 secs.
+            Thread.sleep(PERIODIC_VER_CHECK_CONN_TIMEOUT);
+
+            // Just wait another 60 secs in order to get
+            // version info even on slow connection.
+            for (int i = 0; i < 60 && verChecker.latestVersion() == null; i++)
+                Thread.sleep(1000);
+
+            // Report status if one is available.
+            // No-op if status is NOT available.
+            verChecker.reportStatus(log);
+
+            if (first) {
+                first = false;
+
+                verChecker.reportOnlyNew(true);
+            }
+        }
+    }
 }


[06/39] ignite git commit: Fixed test.

Posted by ag...@apache.org.
Fixed test.


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

Branch: refs/heads/ignite-264
Commit: 144bd4a3679a0c1720bf602c892d6934d3af94a4
Parents: 7016bd9
Author: sboikov <sb...@gridgain.com>
Authored: Wed Sep 2 13:23:27 2015 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Wed Sep 2 13:23:27 2015 +0300

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


http://git-wip-us.apache.org/repos/asf/ignite/blob/144bd4a3/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/GridCachePartitionNotLoadedEventSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/GridCachePartitionNotLoadedEventSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/GridCachePartitionNotLoadedEventSelfTest.java
index 9ef99f7..e712cc9 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/GridCachePartitionNotLoadedEventSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/GridCachePartitionNotLoadedEventSelfTest.java
@@ -89,7 +89,7 @@ public class GridCachePartitionNotLoadedEventSelfTest extends GridCommonAbstract
     }
 
     /**
-     *
+     * @throws Exception If failed.
      */
     public void testPrimaryAndBackupDead() throws Exception {
         backupCnt = 1;
@@ -131,12 +131,14 @@ public class GridCachePartitionNotLoadedEventSelfTest extends GridCommonAbstract
     }
 
     /**
-     *
+     * @throws Exception If failed.
      */
     public void testPrimaryDead() throws Exception {
         startGrid(0);
         startGrid(1);
 
+        awaitPartitionMapExchange();
+
         PartitionNotFullyLoadedListener lsnr = new PartitionNotFullyLoadedListener();
 
         ignite(1).events().localListen(lsnr, EventType.EVT_CACHE_REBALANCE_PART_DATA_LOST);


[24/39] ignite git commit: Merge branch 'ignite-1.4' of https://git-wip-us.apache.org/repos/asf/ignite into ignite-1.4

Posted by ag...@apache.org.
Merge branch 'ignite-1.4' of https://git-wip-us.apache.org/repos/asf/ignite into ignite-1.4


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

Branch: refs/heads/ignite-264
Commit: 3a280a0e552c6f474b3112ad74cfa7f9ec9e7c21
Parents: 9057a4c b847148
Author: Denis Magda <dm...@gridgain.com>
Authored: Thu Sep 3 09:08:12 2015 +0300
Committer: Denis Magda <dm...@gridgain.com>
Committed: Thu Sep 3 09:08:12 2015 +0300

----------------------------------------------------------------------
 modules/aop/pom.xml                             |   1 +
 modules/apache-license-gen/pom.xml              |   1 +
 modules/aws/pom.xml                             |   1 +
 modules/clients/pom.xml                         |   1 +
 modules/cloud/pom.xml                           |   1 +
 modules/codegen/pom.xml                         |   1 +
 modules/core/pom.xml                            |   1 +
 .../GridCacheLoaderWriterStoreFactory.java      |  20 +++-
 .../processors/cache/GridCacheProcessor.java    |  10 +-
 .../store/StoreResourceInjectionSelfTest.java   | 104 +++++++++++++++++++
 .../ignite/testsuites/IgniteCacheTestSuite.java |   4 +-
 modules/gce/pom.xml                             |   1 +
 modules/geospatial/pom.xml                      |   1 +
 modules/hadoop/pom.xml                          |   1 +
 modules/hibernate/pom.xml                       |   1 +
 modules/indexing/pom.xml                        |   1 +
 modules/jcl/pom.xml                             |   1 +
 modules/jms11/pom.xml                           |   1 +
 modules/jta/pom.xml                             |   1 +
 modules/kafka/pom.xml                           |   1 +
 modules/log4j/pom.xml                           |   1 +
 modules/log4j2/pom.xml                          |   1 +
 modules/mesos/pom.xml                           |   1 +
 modules/platform/pom.xml                        |   1 +
 .../platform/cache/PlatformCache.java           |   2 +-
 modules/rest-http/pom.xml                       |   1 +
 modules/scalar-2.10/pom.xml                     |   1 +
 modules/scalar/pom.xml                          |   1 +
 modules/schedule/pom.xml                        |   1 +
 modules/schema-import/pom.xml                   |   1 +
 modules/slf4j/pom.xml                           |   1 +
 modules/spark-2.10/pom.xml                      |   1 +
 modules/spark/pom.xml                           |   1 +
 modules/spring/pom.xml                          |   1 +
 modules/ssh/pom.xml                             |   1 +
 modules/tools/pom.xml                           |   1 +
 modules/urideploy/pom.xml                       |   1 +
 modules/visor-console-2.10/pom.xml              |   1 +
 modules/visor-console/pom.xml                   |   1 +
 modules/visor-plugins/pom.xml                   |   1 +
 modules/web/pom.xml                             |   1 +
 modules/yardstick/pom.xml                       |   1 +
 modules/yarn/pom.xml                            |   1 +
 modules/zookeeper/pom.xml                       |   1 +
 parent/pom.xml                                  |   1 +
 45 files changed, 174 insertions(+), 6 deletions(-)
----------------------------------------------------------------------



[27/39] ignite git commit: Merge remote-tracking branch 'origin/ignite-1.4' into ignite-1.4

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


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

Branch: refs/heads/ignite-264
Commit: 662bc331ce6e26b72809901bdd6edd64364814f9
Parents: 3de5f98 0b9d7ca
Author: sboikov <sb...@gridgain.com>
Authored: Thu Sep 3 11:40:07 2015 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Thu Sep 3 11:40:07 2015 +0300

----------------------------------------------------------------------
 .../apache/ignite/internal/IgniteKernal.java    | 97 ++++++++++++++------
 1 file changed, 69 insertions(+), 28 deletions(-)
----------------------------------------------------------------------



[35/39] ignite git commit: Topology validator test and javadoc improvement

Posted by ag...@apache.org.
Topology validator test and javadoc improvement


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

Branch: refs/heads/ignite-264
Commit: 2fbf328b3f5fd08f7d24d2a0dab6e482192281ad
Parents: c5d303b
Author: Anton Vinogradov <av...@apache.org>
Authored: Thu Sep 3 18:34:38 2015 +0300
Committer: Anton Vinogradov <av...@apache.org>
Committed: Thu Sep 3 18:34:38 2015 +0300

----------------------------------------------------------------------
 .../configuration/CacheConfiguration.java       | 45 +++++++++++++++++++
 .../ignite/configuration/TopologyValidator.java |  4 +-
 ...gniteTopologyValidatorAbstractCacheTest.java | 46 ++++++++++++++++----
 3 files changed, 85 insertions(+), 10 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/2fbf328b/modules/core/src/main/java/org/apache/ignite/configuration/CacheConfiguration.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/configuration/CacheConfiguration.java b/modules/core/src/main/java/org/apache/ignite/configuration/CacheConfiguration.java
index 792bb28..1bbc110 100644
--- a/modules/core/src/main/java/org/apache/ignite/configuration/CacheConfiguration.java
+++ b/modules/core/src/main/java/org/apache/ignite/configuration/CacheConfiguration.java
@@ -20,12 +20,14 @@ package org.apache.ignite.configuration;
 import java.io.Serializable;
 import java.util.Collection;
 import javax.cache.Cache;
+import javax.cache.CacheException;
 import javax.cache.configuration.CompleteConfiguration;
 import javax.cache.configuration.Factory;
 import javax.cache.configuration.MutableConfiguration;
 import javax.cache.expiry.ExpiryPolicy;
 import org.apache.ignite.Ignite;
 import org.apache.ignite.IgniteCache;
+import org.apache.ignite.IgniteException;
 import org.apache.ignite.cache.CacheAtomicWriteOrderMode;
 import org.apache.ignite.cache.CacheAtomicityMode;
 import org.apache.ignite.cache.CacheEntryProcessor;
@@ -1788,6 +1790,27 @@ public class CacheConfiguration<K, V> extends MutableConfiguration<K, V> {
 
     /**
      * Gets topology validator.
+     * <p>
+     * Topology validator checks whether the new topology is valid for specific cache at each topology change.
+     * Topology is always valid in case no topology validator used.
+     * <p>
+     * In case topology is valid for specific cache all operations on this cache are allowed.
+     * <p>
+     * In case topology is not valid for specific cache all update operations on this cache are restricted:
+     * <p>{@link CacheException} will be thrown at update operations (put, remove, etc) attempt.
+     * <p>{@link IgniteException} will be thrown at transaction commit attempt.
+     *
+     * <p>
+     * Usage example
+     * <p>
+     * Following validator allows to put data only in case topology contains exactly 2 nodes:
+     * <pre>{@code
+     * new TopologyValidator() {
+     *    @Override public boolean validate(Collection<ClusterNode> nodes) {
+     *       return nodes.size() == 2;
+     *    }
+     * }
+     * }</pre>
      * @return validator.
      */
     public TopologyValidator getTopologyValidator() {
@@ -1796,6 +1819,28 @@ public class CacheConfiguration<K, V> extends MutableConfiguration<K, V> {
 
     /**
      * Sets topology validator.
+     * <p>
+     * Topology validator checks whether the new topology is valid for specific cache at each topology change.
+     * Topology is always valid in case no topology validator used.
+     * <p>
+     * In case topology is valid for specific cache all operations on this cache are allowed.
+     * <p>
+     * In case topology is not valid for specific cache all update operations on this cache are restricted:
+     * <p>{@link CacheException} will be thrown at update operations (put, remove, etc) attempt.
+     * <p>{@link IgniteException} will be thrown at transaction commit attempt.
+     *
+     * <p>
+     * Usage example
+     * <p>
+     * Following validator allows to put data only in case topology contains exactly 2 nodes:
+     * <pre>{@code
+     * new TopologyValidator() {
+     *    @Override public boolean validate(Collection<ClusterNode> nodes) {
+     *       return nodes.size() == 2;
+     *    }
+     * }
+     * }</pre>
+     *
      * @param topValidator validator.
      * @return {@code this} for chaining.
      */

http://git-wip-us.apache.org/repos/asf/ignite/blob/2fbf328b/modules/core/src/main/java/org/apache/ignite/configuration/TopologyValidator.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/configuration/TopologyValidator.java b/modules/core/src/main/java/org/apache/ignite/configuration/TopologyValidator.java
index ef9284d..49c06a0 100644
--- a/modules/core/src/main/java/org/apache/ignite/configuration/TopologyValidator.java
+++ b/modules/core/src/main/java/org/apache/ignite/configuration/TopologyValidator.java
@@ -27,8 +27,8 @@ import org.apache.ignite.cluster.ClusterNode;
 public interface TopologyValidator extends Serializable {
     /**
      * Validates topology.
-     * @param nodes nodes collection to be validated.
-     * @return is topology valid or not.
+     * @param nodes Collection of nodes.
+     * @return {@code true} in case topology is valid for specific cache, otherwise {@code false}
      */
     boolean validate(Collection<ClusterNode> nodes);
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/2fbf328b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteTopologyValidatorAbstractCacheTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteTopologyValidatorAbstractCacheTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteTopologyValidatorAbstractCacheTest.java
index deb1fee..65f4694 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteTopologyValidatorAbstractCacheTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteTopologyValidatorAbstractCacheTest.java
@@ -86,13 +86,11 @@ public abstract class IgniteTopologyValidatorAbstractCacheTest extends IgniteCac
      */
     protected void putInvalid(String cacheName) {
         try {
-            assert grid(0).cache(cacheName).get(KEY_VALUE) == null;
-
             grid(0).cache(cacheName).put(KEY_VALUE, KEY_VALUE);
 
             assert false : "topology validation broken";
         }
-        catch (IgniteException | CacheException ex) {
+        catch (CacheException ex) {
             assert ex.getCause() instanceof IgniteCheckedException &&
                 ex.getCause().getMessage().contains("cache topology is not valid");
         }
@@ -105,18 +103,47 @@ public abstract class IgniteTopologyValidatorAbstractCacheTest extends IgniteCac
      */
     protected void putValid(String cacheName) {
         try {
-            assert grid(0).cache(cacheName).get(KEY_VALUE) == null;
-
             grid(0).cache(cacheName).put(KEY_VALUE, KEY_VALUE);
 
             assert grid(0).cache(cacheName).get(KEY_VALUE).equals(KEY_VALUE);
         }
-        catch (IgniteException | CacheException ex) {
+        catch (CacheException ex) {
+            assert false : "topology validation broken";
+        }
+    }
+
+    /**
+     * Gets when topology is invalid.
+     *
+     * @param cacheName cache name.
+     */
+    protected void getInvalid(String cacheName) {
+        try {
+            assert grid(0).cache(cacheName).get(KEY_VALUE).equals(KEY_VALUE);
+        }
+        catch (CacheException ex) {
             assert false : "topology validation broken";
         }
     }
 
     /**
+     * Remove when topology is invalid.
+     *
+     * @param cacheName cache name.
+     */
+    protected void removeInvalid(String cacheName) {
+        try {
+            grid(0).cache(cacheName).remove(KEY_VALUE);
+
+            assert false : "topology validation broken";
+        }
+        catch (CacheException ex) {
+            assert ex.getCause() instanceof IgniteCheckedException &&
+                ex.getCause().getMessage().contains("cache topology is not valid");
+        }
+    }
+
+    /**
      * Commits with error.
      *
      * @param tx transaction.
@@ -125,7 +152,7 @@ public abstract class IgniteTopologyValidatorAbstractCacheTest extends IgniteCac
         try {
             tx.commit();
         }
-        catch (IgniteException | CacheException ex) {
+        catch (IgniteException ex) {
             assert ex.getCause() instanceof IgniteCheckedException &&
                 ex.getCause().getMessage().contains("cache topology is not valid");
         }
@@ -158,8 +185,10 @@ public abstract class IgniteTopologyValidatorAbstractCacheTest extends IgniteCac
         remove(null);
 
         putInvalid(CACHE_NAME_1);
+        removeInvalid(CACHE_NAME_1);
 
         putInvalid(CACHE_NAME_2);
+        removeInvalid(CACHE_NAME_2);
 
         startGrid(1);
 
@@ -167,7 +196,6 @@ public abstract class IgniteTopologyValidatorAbstractCacheTest extends IgniteCac
         remove(null);
 
         putValid(CACHE_NAME_1);
-        remove(CACHE_NAME_1);
 
         putValid(CACHE_NAME_2);
         remove(CACHE_NAME_2);
@@ -177,7 +205,9 @@ public abstract class IgniteTopologyValidatorAbstractCacheTest extends IgniteCac
         putValid(null);
         remove(null);
 
+        getInvalid(CACHE_NAME_1);
         putInvalid(CACHE_NAME_1);
+        removeInvalid(CACHE_NAME_1);
 
         putValid(CACHE_NAME_2);
         remove(CACHE_NAME_2);


[09/39] ignite git commit: Merge remote-tracking branch 'remotes/origin/ignite-1.3.3-p7' into ignite-1.4

Posted by ag...@apache.org.
Merge remote-tracking branch 'remotes/origin/ignite-1.3.3-p7' into ignite-1.4


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

Branch: refs/heads/ignite-264
Commit: 7809d262931bf6231b6a96ec2bf247a05cb4cecf
Parents: ba756cd 81e0c40
Author: sboikov <sb...@gridgain.com>
Authored: Wed Sep 2 13:59:44 2015 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Wed Sep 2 13:59:44 2015 +0300

----------------------------------------------------------------------
 .../discovery/GridDiscoveryManager.java         |  52 ++++--
 .../cache/DynamicCacheChangeRequest.java        |  19 +-
 .../cache/DynamicCacheDescriptor.java           |  16 +-
 .../GridCachePartitionExchangeManager.java      |   4 +-
 .../processors/cache/GridCacheProcessor.java    | 187 +++++++++----------
 .../cache/IgniteDynamicCacheStartSelfTest.java  |   4 +-
 6 files changed, 142 insertions(+), 140 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/7809d262/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManager.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/ignite/blob/7809d262/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/DynamicCacheChangeRequest.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/ignite/blob/7809d262/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/DynamicCacheDescriptor.java
----------------------------------------------------------------------

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

http://git-wip-us.apache.org/repos/asf/ignite/blob/7809d262/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java
----------------------------------------------------------------------
diff --cc modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java
index c5f8168,687c987..57c1977
--- 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
@@@ -1837,22 -1673,16 +1802,20 @@@ public class GridCacheProcessor extend
          Collection<DynamicCacheChangeRequest> reqs =
              new ArrayList<>(registeredCaches.size() + registeredTemplates.size());
  
 -        for (DynamicCacheDescriptor desc : registeredCaches.values()) {
 +        boolean reconnect = ctx.localNodeId().equals(nodeId) && cachesOnDisconnect != null;
 +
 +        Map<String, DynamicCacheDescriptor> descs = reconnect ? cachesOnDisconnect : registeredCaches;
 +
 +        for (DynamicCacheDescriptor desc : descs.values()) {
-             if (!desc.cancelled()) {
-                 DynamicCacheChangeRequest req = new DynamicCacheChangeRequest(desc.cacheConfiguration().getName(), null);
+             DynamicCacheChangeRequest req = new DynamicCacheChangeRequest(desc.cacheConfiguration().getName(), null);
  
-                 req.startCacheConfiguration(desc.cacheConfiguration());
+             req.startCacheConfiguration(desc.cacheConfiguration());
  
-                 req.cacheType(desc.cacheType());
+             req.cacheType(desc.cacheType());
  
-                 req.deploymentId(desc.deploymentId());
+             req.deploymentId(desc.deploymentId());
  
-                 reqs.add(req);
-             }
+             reqs.add(req);
          }
  
          for (DynamicCacheDescriptor desc : registeredTemplates.values()) {

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


[04/39] ignite git commit: Fixed cache stop procedure

Posted by ag...@apache.org.
Fixed cache stop procedure


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

Branch: refs/heads/ignite-264
Commit: dadf4bd3beb5fde31b89b0c2a0fbfb3aa5b46e3a
Parents: c1b537f
Author: Valentin Kulichenko <va...@gmail.com>
Authored: Tue Sep 1 14:44:37 2015 -0700
Committer: Valentin Kulichenko <va...@gmail.com>
Committed: Tue Sep 1 14:44:37 2015 -0700

----------------------------------------------------------------------
 .../ignite/internal/processors/cache/GridCacheProcessor.java     | 4 ++--
 1 file changed, 2 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/dadf4bd3/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 f13af23..c08f1d4 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
@@ -1683,7 +1683,7 @@ public class GridCacheProcessor extends GridProcessorAdapter {
         GridCacheIoManager ioMgr = new GridCacheIoManager();
 
         CacheJtaManagerAdapter jta = JTA.createOptional();
-        
+
         return new GridCacheSharedContext(
             kernalCtx,
             tm,
@@ -2875,7 +2875,7 @@ public class GridCacheProcessor extends GridProcessorAdapter {
      * Cancel all user operations.
      */
     public void cancelUserOperations() {
-        for (GridCacheAdapter<?, ?> cache : caches.values())
+        for (GridCacheAdapter<?, ?> cache : stoppedCaches.values())
             cache.ctx.mvcc().cancelClientFutures();
     }