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/17 04:04:07 UTC

[01/55] [abbrv] ignite git commit: ignite-1462: hid portable API in 1.4 release

Repository: ignite
Updated Branches:
  refs/heads/ignite-1171 91115c81d -> 6aa0ee16c


http://git-wip-us.apache.org/repos/asf/ignite/blob/71379a80/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgnitePortableCacheQueryTestSuite.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgnitePortableCacheQueryTestSuite.java b/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgnitePortableCacheQueryTestSuite.java
deleted file mode 100644
index 27ac436..0000000
--- a/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgnitePortableCacheQueryTestSuite.java
+++ /dev/null
@@ -1,117 +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.testsuites;
-
-import junit.framework.TestSuite;
-import org.apache.ignite.internal.processors.cache.CacheLocalQueryMetricsSelfTest;
-import org.apache.ignite.internal.processors.cache.CachePartitionedQueryMetricsDistributedSelfTest;
-import org.apache.ignite.internal.processors.cache.CachePartitionedQueryMetricsLocalSelfTest;
-import org.apache.ignite.internal.processors.cache.CacheReplicatedQueryMetricsDistributedSelfTest;
-import org.apache.ignite.internal.processors.cache.CacheReplicatedQueryMetricsLocalSelfTest;
-import org.apache.ignite.internal.processors.cache.GridCacheQueryIndexDisabledSelfTest;
-import org.apache.ignite.internal.processors.cache.GridCacheQueryIndexingDisabledSelfTest;
-import org.apache.ignite.internal.processors.cache.GridCacheReduceQueryMultithreadedSelfTest;
-import org.apache.ignite.internal.processors.cache.IgniteCacheFieldsQueryNoDataSelfTest;
-import org.apache.ignite.internal.processors.cache.IgniteCacheLargeResultSelfTest;
-import org.apache.ignite.internal.processors.cache.IgniteCacheOffheapTieredMultithreadedSelfTest;
-import org.apache.ignite.internal.processors.cache.IgniteCacheP2pUnmarshallingQueryErrorTest;
-import org.apache.ignite.internal.processors.cache.IgniteCachePartitionedQueryMultiThreadedSelfTest;
-import org.apache.ignite.internal.processors.cache.IgniteCacheQueryEvictsMultiThreadedSelfTest;
-import org.apache.ignite.internal.processors.cache.IgniteCacheQueryMultiThreadedSelfTest;
-import org.apache.ignite.internal.processors.cache.IgniteCacheQueryOffheapMultiThreadedSelfTest;
-import org.apache.ignite.internal.processors.cache.portable.distributed.dht.GridCachePortableDuplicateIndexObjectPartitionedAtomicSelfTest;
-import org.apache.ignite.internal.processors.cache.portable.distributed.dht.GridCachePortableDuplicateIndexObjectPartitionedTransactionalSelfTest;
-import org.apache.ignite.internal.processors.cache.query.continuous.GridCacheContinuousQueryAtomicNearEnabledSelfTest;
-import org.apache.ignite.internal.processors.cache.query.continuous.GridCacheContinuousQueryAtomicP2PDisabledSelfTest;
-import org.apache.ignite.internal.processors.cache.query.continuous.GridCacheContinuousQueryAtomicSelfTest;
-import org.apache.ignite.internal.processors.cache.query.continuous.GridCacheContinuousQueryLocalAtomicSelfTest;
-import org.apache.ignite.internal.processors.cache.query.continuous.GridCacheContinuousQueryPartitionedOnlySelfTest;
-import org.apache.ignite.internal.processors.cache.query.continuous.GridCacheContinuousQueryReplicatedAtomicSelfTest;
-import org.apache.ignite.internal.processors.query.h2.sql.BaseH2CompareQueryTest;
-import org.apache.ignite.internal.processors.query.h2.sql.GridQueryParsingTest;
-import org.apache.ignite.internal.processors.query.h2.sql.H2CompareBigQueryTest;
-import org.apache.ignite.marshaller.portable.PortableMarshaller;
-import org.apache.ignite.spi.communication.tcp.GridOrderedMessageCancelSelfTest;
-import org.apache.ignite.testframework.config.GridTestProperties;
-
-/**
- * Cache query suite with portable marshaller.
- */
-public class IgnitePortableCacheQueryTestSuite extends TestSuite {
-    /**
-     * @return Suite.
-     * @throws Exception In case of error.
-     */
-    public static TestSuite suite() throws Exception {
-        GridTestProperties.setProperty(GridTestProperties.MARSH_CLASS_NAME, PortableMarshaller.class.getName());
-
-        TestSuite suite = new TestSuite("Grid Cache Query Test Suite using PortableMarshaller");
-
-        // Parsing
-        suite.addTestSuite(GridQueryParsingTest.class);
-
-        // Queries tests.
-        suite.addTestSuite(GridCacheQueryIndexDisabledSelfTest.class);
-        suite.addTestSuite(IgniteCachePartitionedQueryMultiThreadedSelfTest.class);
-        suite.addTestSuite(IgniteCacheLargeResultSelfTest.class);
-        suite.addTestSuite(IgniteCacheQueryMultiThreadedSelfTest.class);
-        suite.addTestSuite(IgniteCacheQueryEvictsMultiThreadedSelfTest.class);
-        suite.addTestSuite(IgniteCacheQueryOffheapMultiThreadedSelfTest.class);
-
-        suite.addTestSuite(IgniteCacheOffheapTieredMultithreadedSelfTest.class);
-        suite.addTestSuite(GridCacheReduceQueryMultithreadedSelfTest.class);
-
-
-        // Fields queries.
-        suite.addTestSuite(IgniteCacheFieldsQueryNoDataSelfTest.class);
-
-        // Continuous queries.
-        suite.addTestSuite(GridCacheContinuousQueryLocalAtomicSelfTest.class);
-        suite.addTestSuite(GridCacheContinuousQueryReplicatedAtomicSelfTest.class);
-        suite.addTestSuite(GridCacheContinuousQueryPartitionedOnlySelfTest.class);
-        suite.addTestSuite(GridCacheContinuousQueryAtomicSelfTest.class);
-        suite.addTestSuite(GridCacheContinuousQueryAtomicNearEnabledSelfTest.class);
-        suite.addTestSuite(GridCacheContinuousQueryAtomicP2PDisabledSelfTest.class);
-
-        suite.addTestSuite(GridCacheQueryIndexingDisabledSelfTest.class);
-
-        //Should be adjusted. Not ready to be used with PortableMarshaller.
-        //suite.addTestSuite(GridCachePortableSwapScanQuerySelfTest.class);
-
-        suite.addTestSuite(GridOrderedMessageCancelSelfTest.class);
-
-        // Ignite cache and H2 comparison.
-        suite.addTestSuite(BaseH2CompareQueryTest.class);
-        suite.addTestSuite(H2CompareBigQueryTest.class);
-
-        // Metrics tests
-        suite.addTestSuite(CacheLocalQueryMetricsSelfTest.class);
-        suite.addTestSuite(CachePartitionedQueryMetricsDistributedSelfTest.class);
-        suite.addTestSuite(CachePartitionedQueryMetricsLocalSelfTest.class);
-        suite.addTestSuite(CacheReplicatedQueryMetricsDistributedSelfTest.class);
-        suite.addTestSuite(CacheReplicatedQueryMetricsLocalSelfTest.class);
-
-        //Unmarshallig query test.
-        suite.addTestSuite(IgniteCacheP2pUnmarshallingQueryErrorTest.class);
-
-        suite.addTestSuite(GridCachePortableDuplicateIndexObjectPartitionedAtomicSelfTest.class);
-        suite.addTestSuite(GridCachePortableDuplicateIndexObjectPartitionedTransactionalSelfTest.class);
-
-        return suite;
-    }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/71379a80/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/PlatformContextImpl.java
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/PlatformContextImpl.java b/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/PlatformContextImpl.java
index 3895506..738f910 100644
--- a/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/PlatformContextImpl.java
+++ b/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/PlatformContextImpl.java
@@ -37,6 +37,7 @@ import org.apache.ignite.internal.portable.GridPortableMarshaller;
 import org.apache.ignite.internal.portable.PortableMetaDataImpl;
 import org.apache.ignite.internal.portable.PortableRawReaderEx;
 import org.apache.ignite.internal.portable.PortableRawWriterEx;
+import org.apache.ignite.internal.portable.api.PortableMetadata;
 import org.apache.ignite.internal.processors.cache.portable.CacheObjectPortableProcessorImpl;
 import org.apache.ignite.internal.processors.platform.cache.PlatformCacheEntryFilter;
 import org.apache.ignite.internal.processors.platform.cache.PlatformCacheEntryFilterImpl;
@@ -70,7 +71,6 @@ import org.apache.ignite.internal.util.typedef.F;
 import org.apache.ignite.internal.util.typedef.T4;
 import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.lang.IgniteBiTuple;
-import org.apache.ignite.portable.PortableMetadata;
 import org.jetbrains.annotations.Nullable;
 
 import java.util.Collection;
@@ -373,7 +373,7 @@ public class PlatformContextImpl implements PlatformContext {
 
         writer.writeInt(metas.size());
 
-        for (org.apache.ignite.portable.PortableMetadata m : metas)
+        for (PortableMetadata m : metas)
             writeMetadata0(writer, cacheObjProc.typeId(m.typeName()), m);
     }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/71379a80/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/compute/PlatformCompute.java
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/compute/PlatformCompute.java b/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/compute/PlatformCompute.java
index 638b4b1..9e092c5 100644
--- a/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/compute/PlatformCompute.java
+++ b/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/compute/PlatformCompute.java
@@ -33,7 +33,7 @@ import org.apache.ignite.internal.processors.platform.PlatformContext;
 import org.apache.ignite.internal.util.typedef.C1;
 import org.apache.ignite.lang.IgniteFuture;
 import org.apache.ignite.lang.IgniteInClosure;
-import org.apache.ignite.portable.PortableObject;
+import org.apache.ignite.internal.portable.api.PortableObject;
 
 import static org.apache.ignite.internal.processors.task.GridTaskThreadContextKey.TC_SUBGRID;
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/71379a80/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/cpp/PlatformCppConfigurationClosure.java
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/cpp/PlatformCppConfigurationClosure.java b/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/cpp/PlatformCppConfigurationClosure.java
index d95a82b..ee8f9a3 100644
--- a/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/cpp/PlatformCppConfigurationClosure.java
+++ b/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/cpp/PlatformCppConfigurationClosure.java
@@ -25,7 +25,7 @@ import org.apache.ignite.internal.processors.platform.PlatformAbstractConfigurat
 import org.apache.ignite.internal.processors.platform.memory.PlatformMemoryManagerImpl;
 import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.marshaller.Marshaller;
-import org.apache.ignite.marshaller.portable.PortableMarshaller;
+import org.apache.ignite.internal.portable.api.PortableMarshaller;
 import org.apache.ignite.platform.cpp.PlatformCppConfiguration;
 
 import java.util.Collections;

http://git-wip-us.apache.org/repos/asf/ignite/blob/71379a80/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/dotnet/PlatformDotNetConfigurationClosure.java
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/dotnet/PlatformDotNetConfigurationClosure.java b/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/dotnet/PlatformDotNetConfigurationClosure.java
index 6e03dfe..21cd01a 100644
--- a/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/dotnet/PlatformDotNetConfigurationClosure.java
+++ b/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/dotnet/PlatformDotNetConfigurationClosure.java
@@ -36,10 +36,10 @@ import org.apache.ignite.internal.processors.platform.utils.PlatformUtils;
 import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.lifecycle.LifecycleBean;
 import org.apache.ignite.marshaller.Marshaller;
-import org.apache.ignite.marshaller.portable.PortableMarshaller;
+import org.apache.ignite.internal.portable.api.PortableMarshaller;
 import org.apache.ignite.platform.dotnet.PlatformDotNetLifecycleBean;
-import org.apache.ignite.portable.PortableException;
-import org.apache.ignite.portable.PortableMetadata;
+import org.apache.ignite.internal.portable.api.PortableException;
+import org.apache.ignite.internal.portable.api.PortableMetadata;
 
 import java.util.ArrayList;
 import java.util.Collections;

http://git-wip-us.apache.org/repos/asf/ignite/blob/71379a80/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Config/Compute/compute-grid1.xml
----------------------------------------------------------------------
diff --git a/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Config/Compute/compute-grid1.xml b/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Config/Compute/compute-grid1.xml
index 183676b..6164ef3 100644
--- a/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Config/Compute/compute-grid1.xml
+++ b/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Config/Compute/compute-grid1.xml
@@ -69,16 +69,16 @@
 
         <!-- Portable marshaller configuration -->
         <property name="marshaller">
-            <bean class="org.apache.ignite.marshaller.portable.PortableMarshaller">
+            <bean class="org.apache.ignite.internal.portable.api.PortableMarshaller">
                 <property name="typeConfigurations">
                     <list>
-                        <bean class="org.apache.ignite.portable.PortableTypeConfiguration">
+                        <bean class="org.apache.ignite.internal.portable.api.PortableTypeConfiguration">
                             <property name="className" value="org.apache.ignite.platform.PlatformComputePortable"/>
                         </bean>
-                        <bean class="org.apache.ignite.portable.PortableTypeConfiguration">
+                        <bean class="org.apache.ignite.internal.portable.api.PortableTypeConfiguration">
                             <property name="className" value="org.apache.ignite.platform.PlatformComputeJavaPortable"/>
                         </bean>
-                        <bean class="org.apache.ignite.portable.PortableTypeConfiguration">
+                        <bean class="org.apache.ignite.internal.portable.api.PortableTypeConfiguration">
                             <property name="className" value="org.apache.ignite.platform.PlatformComputeEnum"/>
                         </bean>
                     </list>

http://git-wip-us.apache.org/repos/asf/ignite/blob/71379a80/modules/platform/src/test/java/org/apache/ignite/platform/PlatformComputePortableArgTask.java
----------------------------------------------------------------------
diff --git a/modules/platform/src/test/java/org/apache/ignite/platform/PlatformComputePortableArgTask.java b/modules/platform/src/test/java/org/apache/ignite/platform/PlatformComputePortableArgTask.java
index 0e8b825..9b90209 100644
--- a/modules/platform/src/test/java/org/apache/ignite/platform/PlatformComputePortableArgTask.java
+++ b/modules/platform/src/test/java/org/apache/ignite/platform/PlatformComputePortableArgTask.java
@@ -24,9 +24,10 @@ import org.apache.ignite.compute.ComputeJob;
 import org.apache.ignite.compute.ComputeJobAdapter;
 import org.apache.ignite.compute.ComputeJobResult;
 import org.apache.ignite.compute.ComputeTaskAdapter;
+import org.apache.ignite.internal.IgniteEx;
 import org.apache.ignite.internal.util.typedef.F;
-import org.apache.ignite.portable.PortableMetadata;
-import org.apache.ignite.portable.PortableObject;
+import org.apache.ignite.internal.portable.api.PortableMetadata;
+import org.apache.ignite.internal.portable.api.PortableObject;
 import org.apache.ignite.resources.IgniteInstanceResource;
 import org.jetbrains.annotations.Nullable;
 
@@ -89,7 +90,8 @@ public class PlatformComputePortableArgTask extends ComputeTaskAdapter<Object, I
         @Nullable @Override public Object execute() {
             PortableObject arg0 = ((PortableObject)arg);
 
-            PortableMetadata meta = ignite.portables().metadata(arg0.typeId());
+            PortableMetadata meta = ignite instanceof IgniteEx ?
+                ((IgniteEx)ignite).portables().metadata(arg0.typeId()) : null;
 
             if (meta == null)
                 throw new IgniteException("Metadata doesn't exist.");

http://git-wip-us.apache.org/repos/asf/ignite/blob/71379a80/modules/spring/src/main/java/org/apache/ignite/IgniteSpringBean.java
----------------------------------------------------------------------
diff --git a/modules/spring/src/main/java/org/apache/ignite/IgniteSpringBean.java b/modules/spring/src/main/java/org/apache/ignite/IgniteSpringBean.java
index 4320df5..42514e3 100644
--- a/modules/spring/src/main/java/org/apache/ignite/IgniteSpringBean.java
+++ b/modules/spring/src/main/java/org/apache/ignite/IgniteSpringBean.java
@@ -347,13 +347,6 @@ public class IgniteSpringBean implements Ignite, DisposableBean, InitializingBea
     }
 
     /** {@inheritDoc} */
-    @Override public IgnitePortables portables() {
-        assert g != null;
-
-        return g.portables();
-    }
-
-    /** {@inheritDoc} */
     @Override public void close() throws IgniteException {
         g.close();
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/71379a80/parent/pom.xml
----------------------------------------------------------------------
diff --git a/parent/pom.xml b/parent/pom.xml
index f5a29cb..ba44c85 100644
--- a/parent/pom.xml
+++ b/parent/pom.xml
@@ -319,10 +319,6 @@
                                 <packages>org.apache.ignite.marshaller*</packages>
                             </group>
                             <group>
-                                <title>Portable Objects API</title>
-                                <packages>org.apache.ignite.portable*</packages>
-                            </group>
-                            <group>
                                 <title>Visor Plugins</title>
                                 <packages>org.apache.ignite.visor.plugin</packages>
                             </group>
@@ -712,10 +708,6 @@
                                         <exclude>dev-tools/.gradle/**/*</exclude>
                                         <exclude>dev-tools/gradle/wrapper/**/*</exclude>
                                         <exclude>dev-tools/gradlew</exclude>
-                                        <exclude>src/test/portables/repo/org/apache/ignite/portable/test2/1.1/test2-1.1.pom</exclude>
-                                        <exclude>src/test/portables/repo/org/apache/ignite/portable/test2/maven-metadata-local.xml</exclude>
-                                        <exclude>src/test/portables/repo/org/apache/ignite/portable/test1/1.1/test1-1.1.pom</exclude>
-                                        <exclude>src/test/portables/repo/org/apache/ignite/portable/test1/maven-metadata-local.xml</exclude>
                                         <!--shmem-->
                                         <exclude>ipc/shmem/**/Makefile.in</exclude><!--auto generated files-->
                                         <exclude>ipc/shmem/**/Makefile</exclude><!--auto generated files-->
@@ -745,8 +737,6 @@
                                         <exclude>src/main/java/META-INF/services/org.apache.ignite.internal.processors.platform.PlatformBootstrapFactory</exclude>
                                         <exclude>src/main/resources/META-INF/services/org.apache.ignite.internal.processors.platform.PlatformBootstrapFactory</exclude>
                                         <exclude>src/test/dotnet/Apache.Ignite.Core.Tests/Apache.Ignite.Core.Tests.csproj</exclude>
-                                        <exclude>src/test/portables/repo/org/apache/ignite/portable/test1/1.1/test1-1.1.jar</exclude>
-                                        <exclude>src/test/portables/repo/org/apache/ignite/portable/test2/1.1/test2-1.1.jar</exclude>
                                         <exclude>**/Makefile.am</exclude>
                                         <exclude>**/configure.ac</exclude>
                                         <exclude>**/*.pc.in</exclude>


[44/55] [abbrv] 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/6bd0f8f4
Tree: http://git-wip-us.apache.org/repos/asf/ignite/tree/6bd0f8f4
Diff: http://git-wip-us.apache.org/repos/asf/ignite/diff/6bd0f8f4

Branch: refs/heads/ignite-1171
Commit: 6bd0f8f405268214c62e2289a0081a238053fc11
Parents: 72e270c dcd8b42
Author: Yakov Zhdanov <yz...@gridgain.com>
Authored: Tue Sep 15 16:55:19 2015 +0300
Committer: Yakov Zhdanov <yz...@gridgain.com>
Committed: Tue Sep 15 16:55:19 2015 +0300

----------------------------------------------------------------------
 .../processors/cache/GridCacheProcessor.java    |  10 +-
 .../dht/GridClientPartitionTopology.java        | 104 +++++++++++--------
 .../dht/GridDhtPartitionTopology.java           |   4 +-
 .../dht/GridDhtPartitionTopologyImpl.java       |   7 +-
 .../cache/transactions/IgniteTxHandler.java     |   2 +-
 .../ignite/internal/util/IgniteUtils.java       |  16 +++
 .../processors/igfs/IgfsAbstractSelfTest.java   |   3 +-
 .../testsuites/IgniteHadoopTestSuite.java       |   6 +-
 ...CacheScanPartitionQueryFallbackSelfTest.java |  54 ++++------
 9 files changed, 114 insertions(+), 92 deletions(-)
----------------------------------------------------------------------



[21/55] [abbrv] ignite git commit: IGNITE-1465: Fixed.

Posted by ag...@apache.org.
IGNITE-1465: Fixed.


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

Branch: refs/heads/ignite-1171
Commit: f8b798d75c53e051ed2171441a2325e8d108300d
Parents: 961a467
Author: vozerov-gridgain <vo...@gridgain.com>
Authored: Tue Sep 15 09:34:27 2015 +0300
Committer: vozerov-gridgain <vo...@gridgain.com>
Committed: Tue Sep 15 09:34:27 2015 +0300

----------------------------------------------------------------------
 .../internal/processors/hadoop/SecondaryFileSystemProvider.java  | 4 ++--
 1 file changed, 2 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/f8b798d7/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/SecondaryFileSystemProvider.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/SecondaryFileSystemProvider.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/SecondaryFileSystemProvider.java
index fdb61e8..d5be074 100644
--- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/SecondaryFileSystemProvider.java
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/SecondaryFileSystemProvider.java
@@ -60,8 +60,8 @@ public class SecondaryFileSystemProvider {
 
             if (url == null) {
                 // If secConfPath is given, it should be resolvable:
-                throw new IllegalArgumentException("Failed to resolve secondary file system " +
-                    "configuration path: " + secConfPath);
+                throw new IllegalArgumentException("Failed to resolve secondary file system configuration path " +
+                    "(ensure that it exists locally and you have read access to it): " + secConfPath);
             }
 
             cfg.addResource(url);


[38/55] [abbrv] ignite git commit: Fixed RendezvousAffinityFunction.hashIdResolver compatibility.

Posted by ag...@apache.org.
Fixed RendezvousAffinityFunction.hashIdResolver compatibility.


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

Branch: refs/heads/ignite-1171
Commit: 6e19979b329d266d3be2f4abe2bdbd8dd443d64d
Parents: c257e07
Author: sboikov <sb...@gridgain.com>
Authored: Tue Sep 15 15:57:03 2015 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Tue Sep 15 15:57:03 2015 +0300

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


http://git-wip-us.apache.org/repos/asf/ignite/blob/6e19979b/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 4317f70..4ae0baa 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
@@ -232,16 +232,14 @@ public class GridCacheProcessor extends GridProcessorAdapter {
             if (cfg.getCacheMode() == PARTITIONED) {
                 RendezvousAffinityFunction aff = new RendezvousAffinityFunction();
 
-                if (internalCache)
-                    aff.setHashIdResolver(new AffinityNodeAddressHashResolver());
+                aff.setHashIdResolver(new AffinityNodeAddressHashResolver());
 
                 cfg.setAffinity(aff);
             }
             else if (cfg.getCacheMode() == REPLICATED) {
                 RendezvousAffinityFunction aff = new RendezvousAffinityFunction(false, 512);
 
-                if (internalCache)
-                    aff.setHashIdResolver(new AffinityNodeAddressHashResolver());
+                aff.setHashIdResolver(new AffinityNodeAddressHashResolver());
 
                 cfg.setAffinity(aff);
 
@@ -251,11 +249,11 @@ public class GridCacheProcessor extends GridProcessorAdapter {
                 cfg.setAffinity(new LocalAffinityFunction());
         }
         else {
-            if (cfg.getCacheMode() == PARTITIONED) {
+            if (cfg.getCacheMode() != LOCAL) {
                 if (cfg.getAffinity() instanceof RendezvousAffinityFunction) {
                     RendezvousAffinityFunction aff = (RendezvousAffinityFunction)cfg.getAffinity();
 
-                    if (internalCache && aff.getHashIdResolver() == null)
+                    if (aff.getHashIdResolver() == null)
                         aff.setHashIdResolver(new AffinityNodeAddressHashResolver());
                 }
             }


[22/55] [abbrv] ignite git commit: IGNITE-1378 - Fixed exception handling in GridContinuousProcessor.startRoutine()

Posted by ag...@apache.org.
IGNITE-1378 - Fixed exception handling in GridContinuousProcessor.startRoutine()


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

Branch: refs/heads/ignite-1171
Commit: 1914c0216608dc8eecf97fb3ee4bdfb6fdec740c
Parents: f8b798d
Author: Valentin Kulichenko <va...@gmail.com>
Authored: Mon Sep 14 23:37:26 2015 -0700
Committer: Valentin Kulichenko <va...@gmail.com>
Committed: Mon Sep 14 23:37:26 2015 -0700

----------------------------------------------------------------------
 .../continuous/GridContinuousProcessor.java     | 22 +++++++-------------
 1 file changed, 7 insertions(+), 15 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/1914c021/modules/core/src/main/java/org/apache/ignite/internal/processors/continuous/GridContinuousProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/continuous/GridContinuousProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/continuous/GridContinuousProcessor.java
index 3dcfff8..18c1f36 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/continuous/GridContinuousProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/continuous/GridContinuousProcessor.java
@@ -566,30 +566,22 @@ public class GridContinuousProcessor extends GridProcessorAdapter {
         startFuts.put(routineId, fut);
 
         try {
+            if (locIncluded && registerHandler(ctx.localNodeId(), routineId, hnd, bufSize, interval, autoUnsubscribe, true))
+                hnd.onListenerRegistered(routineId, ctx);
+
             ctx.discovery().sendCustomEvent(new StartRoutineDiscoveryMessage(routineId, reqData));
         }
-        catch (IgniteCheckedException e) { // Marshaller exception may occurs if user pass unmarshallable filter.
+        catch (IgniteCheckedException e) {
             startFuts.remove(routineId);
-
             locInfos.remove(routineId);
 
+            unregisterHandler(routineId, hnd, true);
+
             fut.onDone(e);
 
             return fut;
         }
 
-        // Register local handler if needed.
-        if (locIncluded) {
-            try {
-                if (registerHandler(ctx.localNodeId(), routineId, hnd, bufSize, interval, autoUnsubscribe, true))
-                    hnd.onListenerRegistered(routineId, ctx);
-            }
-            catch (IgniteCheckedException e) {
-                return new GridFinishedFuture<>(
-                    new IgniteCheckedException("Failed to register handler locally: " + hnd, e));
-            }
-        }
-
         // Handler is registered locally.
         fut.onLocalRegistered();
 
@@ -1624,4 +1616,4 @@ public class GridContinuousProcessor extends GridProcessorAdapter {
             return S.toString(SyncMessageAckFuture.class, this);
         }
     }
-}
\ No newline at end of file
+}


[24/55] [abbrv] 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/c0655121
Tree: http://git-wip-us.apache.org/repos/asf/ignite/tree/c0655121
Diff: http://git-wip-us.apache.org/repos/asf/ignite/diff/c0655121

Branch: refs/heads/ignite-1171
Commit: c0655121696dd1668b6b8eb0bbf8369240e41c05
Parents: 2db59e6 1914c02
Author: vozerov-gridgain <vo...@gridgain.com>
Authored: Tue Sep 15 09:40:58 2015 +0300
Committer: vozerov-gridgain <vo...@gridgain.com>
Committed: Tue Sep 15 09:40:58 2015 +0300

----------------------------------------------------------------------
 .../continuous/GridContinuousProcessor.java     | 22 +++++++-------------
 1 file changed, 7 insertions(+), 15 deletions(-)
----------------------------------------------------------------------



[03/55] [abbrv] ignite git commit: ignite-1462: hid portable API in 1.4 release

Posted by ag...@apache.org.
http://git-wip-us.apache.org/repos/asf/ignite/blob/71379a80/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/GridCachePortableObjectsAbstractDataStreamerSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/GridCachePortableObjectsAbstractDataStreamerSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/GridCachePortableObjectsAbstractDataStreamerSelfTest.java
deleted file mode 100644
index 9ba38d9..0000000
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/GridCachePortableObjectsAbstractDataStreamerSelfTest.java
+++ /dev/null
@@ -1,190 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.ignite.internal.processors.cache.portable;
-
-import java.io.Serializable;
-import java.util.Arrays;
-import java.util.concurrent.Callable;
-import java.util.concurrent.ThreadLocalRandom;
-import java.util.concurrent.atomic.AtomicBoolean;
-import org.apache.ignite.IgniteDataStreamer;
-import org.apache.ignite.cache.CacheAtomicityMode;
-import org.apache.ignite.cache.CacheMode;
-import org.apache.ignite.cache.CacheWriteSynchronizationMode;
-import org.apache.ignite.configuration.CacheConfiguration;
-import org.apache.ignite.configuration.IgniteConfiguration;
-import org.apache.ignite.configuration.NearCacheConfiguration;
-import org.apache.ignite.internal.IgniteInternalFuture;
-import org.apache.ignite.marshaller.portable.PortableMarshaller;
-import org.apache.ignite.portable.PortableException;
-import org.apache.ignite.portable.PortableMarshalAware;
-import org.apache.ignite.portable.PortableReader;
-import org.apache.ignite.portable.PortableTypeConfiguration;
-import org.apache.ignite.portable.PortableWriter;
-import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
-import org.jsr166.LongAdder8;
-
-import static org.apache.ignite.cache.CacheWriteSynchronizationMode.PRIMARY_SYNC;
-
-/**
- * Test for portable objects stored in cache.
- */
-public abstract class GridCachePortableObjectsAbstractDataStreamerSelfTest extends GridCommonAbstractTest {
-    /** */
-    private static final int THREAD_CNT = 64;
-
-    /** {@inheritDoc} */
-    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
-        IgniteConfiguration cfg = super.getConfiguration(gridName);
-
-        CacheConfiguration cacheCfg = new CacheConfiguration();
-
-        cacheCfg.setCacheMode(cacheMode());
-        cacheCfg.setAtomicityMode(atomicityMode());
-        cacheCfg.setNearConfiguration(nearConfiguration());
-        cacheCfg.setWriteSynchronizationMode(writeSynchronizationMode());
-
-        cfg.setCacheConfiguration(cacheCfg);
-
-        PortableMarshaller marsh = new PortableMarshaller();
-
-        marsh.setTypeConfigurations(Arrays.asList(
-            new PortableTypeConfiguration(TestObject.class.getName())));
-
-        cfg.setMarshaller(marsh);
-
-        return cfg;
-    }
-
-    /**
-     * @return Sync mode.
-     */
-    protected CacheWriteSynchronizationMode writeSynchronizationMode() {
-        return PRIMARY_SYNC;
-    }
-
-    /** {@inheritDoc} */
-    @Override protected void beforeTestsStarted() throws Exception {
-        startGridsMultiThreaded(gridCount());
-    }
-
-    /** {@inheritDoc} */
-    @Override protected void afterTestsStopped() throws Exception {
-        stopAllGrids();
-    }
-
-    /**
-     * @return Cache mode.
-     */
-    protected abstract CacheMode cacheMode();
-
-    /**
-     * @return Atomicity mode.
-     */
-    protected abstract CacheAtomicityMode atomicityMode();
-
-    /**
-     * @return Near configuration.
-     */
-    protected abstract NearCacheConfiguration nearConfiguration();
-
-    /**
-     * @return Grid count.
-     */
-    protected int gridCount() {
-        return 1;
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    @SuppressWarnings("BusyWait")
-    public void testGetPut() throws Exception {
-        final AtomicBoolean flag = new AtomicBoolean();
-
-        final LongAdder8 cnt = new LongAdder8();
-
-        try (IgniteDataStreamer<Object, Object> ldr = grid(0).dataStreamer(null)) {
-            IgniteInternalFuture<?> f = multithreadedAsync(
-                new Callable<Object>() {
-                    @Override public Object call() throws Exception {
-                        ThreadLocalRandom rnd = ThreadLocalRandom.current();
-
-                        while (!flag.get()) {
-                            ldr.addData(rnd.nextInt(10000), new TestObject(rnd.nextInt(10000)));
-
-                            cnt.add(1);
-                        }
-
-                        return null;
-                    }
-                },
-                THREAD_CNT
-            );
-
-            for (int i = 0; i < 30 && !f.isDone(); i++)
-                Thread.sleep(1000);
-
-            flag.set(true);
-
-            f.get();
-        }
-
-        info("Operations in 30 sec: " + cnt.sum());
-    }
-
-    /**
-     */
-    private static class TestObject implements PortableMarshalAware, Serializable {
-        /** */
-        private int val;
-
-        /**
-         */
-        private TestObject() {
-            // No-op.
-        }
-
-        /**
-         * @param val Value.
-         */
-        private TestObject(int val) {
-            this.val = val;
-        }
-
-        /** {@inheritDoc} */
-        @Override public int hashCode() {
-            return val;
-        }
-
-        /** {@inheritDoc} */
-        @Override public boolean equals(Object obj) {
-            return obj instanceof TestObject && ((TestObject)obj).val == val;
-        }
-
-        /** {@inheritDoc} */
-        @Override public void writePortable(PortableWriter writer) throws PortableException {
-            writer.writeInt("val", val);
-        }
-
-        /** {@inheritDoc} */
-        @Override public void readPortable(PortableReader reader) throws PortableException {
-            val = reader.readInt("val");
-        }
-    }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/71379a80/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/GridCachePortableObjectsAbstractMultiThreadedSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/GridCachePortableObjectsAbstractMultiThreadedSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/GridCachePortableObjectsAbstractMultiThreadedSelfTest.java
deleted file mode 100644
index 67f0e52..0000000
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/GridCachePortableObjectsAbstractMultiThreadedSelfTest.java
+++ /dev/null
@@ -1,231 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.ignite.internal.processors.cache.portable;
-
-import java.io.Serializable;
-import java.util.Arrays;
-import java.util.concurrent.Callable;
-import java.util.concurrent.ThreadLocalRandom;
-import java.util.concurrent.atomic.AtomicBoolean;
-import java.util.concurrent.atomic.AtomicInteger;
-import org.apache.ignite.IgniteCache;
-import org.apache.ignite.cache.CacheAtomicityMode;
-import org.apache.ignite.cache.CacheMode;
-import org.apache.ignite.cache.CacheWriteSynchronizationMode;
-import org.apache.ignite.configuration.CacheConfiguration;
-import org.apache.ignite.configuration.IgniteConfiguration;
-import org.apache.ignite.configuration.NearCacheConfiguration;
-import org.apache.ignite.internal.IgniteInternalFuture;
-import org.apache.ignite.internal.processors.cache.IgniteCacheProxy;
-import org.apache.ignite.marshaller.portable.PortableMarshaller;
-import org.apache.ignite.portable.PortableException;
-import org.apache.ignite.portable.PortableMarshalAware;
-import org.apache.ignite.portable.PortableObject;
-import org.apache.ignite.portable.PortableReader;
-import org.apache.ignite.portable.PortableTypeConfiguration;
-import org.apache.ignite.portable.PortableWriter;
-import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
-import org.jsr166.LongAdder8;
-
-import static org.apache.ignite.cache.CacheWriteSynchronizationMode.PRIMARY_SYNC;
-
-/**
- * Test for portable objects stored in cache.
- */
-public abstract class GridCachePortableObjectsAbstractMultiThreadedSelfTest extends GridCommonAbstractTest {
-    /** */
-    private static final int THREAD_CNT = 64;
-
-    /** */
-    private static final AtomicInteger idxGen = new AtomicInteger();
-
-    /** {@inheritDoc} */
-    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
-        IgniteConfiguration cfg = super.getConfiguration(gridName);
-
-        CacheConfiguration cacheCfg = new CacheConfiguration();
-
-        cacheCfg.setCacheMode(cacheMode());
-        cacheCfg.setAtomicityMode(atomicityMode());
-        cacheCfg.setNearConfiguration(nearConfiguration());
-        cacheCfg.setWriteSynchronizationMode(writeSynchronizationMode());
-
-        cfg.setCacheConfiguration(cacheCfg);
-
-        PortableMarshaller marsh = new PortableMarshaller();
-
-        marsh.setTypeConfigurations(Arrays.asList(
-            new PortableTypeConfiguration(TestObject.class.getName())));
-
-        cfg.setMarshaller(marsh);
-
-        return cfg;
-    }
-
-    /**
-     * @return Sync mode.
-     */
-    protected CacheWriteSynchronizationMode writeSynchronizationMode() {
-        return PRIMARY_SYNC;
-    }
-
-    /** {@inheritDoc} */
-    @Override protected void beforeTestsStarted() throws Exception {
-        startGridsMultiThreaded(gridCount());
-    }
-
-    /** {@inheritDoc} */
-    @Override protected void afterTestsStopped() throws Exception {
-        stopAllGrids();
-    }
-
-    /**
-     * @return Cache mode.
-     */
-    protected abstract CacheMode cacheMode();
-
-    /**
-     * @return Atomicity mode.
-     */
-    protected abstract CacheAtomicityMode atomicityMode();
-
-    /**
-     * @return Distribution mode.
-     */
-    protected abstract NearCacheConfiguration nearConfiguration();
-
-    /**
-     * @return Grid count.
-     */
-    protected int gridCount() {
-        return 1;
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    @SuppressWarnings("BusyWait") public void testGetPut() throws Exception {
-        final AtomicBoolean flag = new AtomicBoolean();
-
-        final LongAdder8 cnt = new LongAdder8();
-
-        IgniteInternalFuture<?> f = multithreadedAsync(
-            new Callable<Object>() {
-                @Override public Object call() throws Exception {
-                    int threadId = idxGen.getAndIncrement() % 2;
-
-                    ThreadLocalRandom rnd = ThreadLocalRandom.current();
-
-                    while (!flag.get()) {
-                        IgniteCache<Object, Object> c = jcache(rnd.nextInt(gridCount()));
-
-                        switch (threadId) {
-                            case 0:
-                                // Put/get/remove portable -> portable.
-
-                                c.put(new TestObject(rnd.nextInt(10000)), new TestObject(rnd.nextInt(10000)));
-
-                                IgniteCache<Object, Object> p2 = ((IgniteCacheProxy<Object, Object>)c).keepPortable();
-
-                                PortableObject v = (PortableObject)p2.get(new TestObject(rnd.nextInt(10000)));
-
-                                if (v != null)
-                                    v.deserialize();
-
-                                c.remove(new TestObject(rnd.nextInt(10000)));
-
-                                break;
-
-                            case 1:
-                                // Put/get int -> portable.
-                                c.put(rnd.nextInt(10000), new TestObject(rnd.nextInt(10000)));
-
-                                IgniteCache<Integer, PortableObject> p4 = ((IgniteCacheProxy<Object, Object>)c).keepPortable();
-
-                                PortableObject v1 = p4.get(rnd.nextInt(10000));
-
-                                if (v1 != null)
-                                    v1.deserialize();
-
-                                p4.remove(rnd.nextInt(10000));
-
-                                break;
-
-                            default:
-                                assert false;
-                        }
-
-                        cnt.add(3);
-                    }
-
-                    return null;
-                }
-            },
-            THREAD_CNT
-        );
-
-        for (int i = 0; i < 30 && !f.isDone(); i++)
-            Thread.sleep(1000);
-
-        flag.set(true);
-
-        f.get();
-
-        info("Operations in 30 sec: " + cnt.sum());
-    }
-
-    /**
-     */
-    private static class TestObject implements PortableMarshalAware, Serializable {
-        /** */
-        private int val;
-
-        /**
-         */
-        private TestObject() {
-            // No-op.
-        }
-
-        /**
-         * @param val Value.
-         */
-        private TestObject(int val) {
-            this.val = val;
-        }
-
-        /** {@inheritDoc} */
-        @Override public int hashCode() {
-            return val;
-        }
-
-        /** {@inheritDoc} */
-        @Override public boolean equals(Object obj) {
-            return obj instanceof TestObject && ((TestObject)obj).val == val;
-        }
-
-        /** {@inheritDoc} */
-        @Override public void writePortable(PortableWriter writer) throws PortableException {
-            writer.writeInt("val", val);
-        }
-
-        /** {@inheritDoc} */
-        @Override public void readPortable(PortableReader reader) throws PortableException {
-            val = reader.readInt("val");
-        }
-    }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/71379a80/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/GridCachePortableObjectsAbstractSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/GridCachePortableObjectsAbstractSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/GridCachePortableObjectsAbstractSelfTest.java
deleted file mode 100644
index 7ac8712..0000000
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/GridCachePortableObjectsAbstractSelfTest.java
+++ /dev/null
@@ -1,978 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.ignite.internal.processors.cache.portable;
-
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.Iterator;
-import java.util.Map;
-import java.util.Set;
-import javax.cache.Cache;
-import javax.cache.processor.EntryProcessor;
-import javax.cache.processor.MutableEntry;
-import org.apache.ignite.Ignite;
-import org.apache.ignite.IgniteCache;
-import org.apache.ignite.IgnitePortables;
-import org.apache.ignite.cache.CacheAtomicityMode;
-import org.apache.ignite.cache.CacheMode;
-import org.apache.ignite.cache.CachePeekMode;
-import org.apache.ignite.cache.store.CacheStoreAdapter;
-import org.apache.ignite.configuration.CacheConfiguration;
-import org.apache.ignite.configuration.IgniteConfiguration;
-import org.apache.ignite.configuration.NearCacheConfiguration;
-import org.apache.ignite.internal.IgniteKernal;
-import org.apache.ignite.internal.portable.PortableObjectImpl;
-import org.apache.ignite.internal.processors.cache.GridCacheAdapter;
-import org.apache.ignite.internal.processors.cache.GridCacheEntryEx;
-import org.apache.ignite.internal.processors.cache.IgniteCacheProxy;
-import org.apache.ignite.internal.util.typedef.P2;
-import org.apache.ignite.internal.util.typedef.internal.CU;
-import org.apache.ignite.internal.util.typedef.internal.U;
-import org.apache.ignite.lang.IgniteBiInClosure;
-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.PortableObject;
-import org.apache.ignite.portable.PortableReader;
-import org.apache.ignite.portable.PortableWriter;
-import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi;
-import org.apache.ignite.spi.discovery.tcp.ipfinder.TcpDiscoveryIpFinder;
-import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder;
-import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
-import org.apache.ignite.transactions.Transaction;
-import org.jetbrains.annotations.Nullable;
-
-import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL;
-import static org.apache.ignite.cache.CacheMemoryMode.OFFHEAP_TIERED;
-import static org.apache.ignite.cache.CacheWriteSynchronizationMode.FULL_SYNC;
-import static org.apache.ignite.transactions.TransactionConcurrency.PESSIMISTIC;
-import static org.apache.ignite.transactions.TransactionIsolation.READ_COMMITTED;
-import static org.apache.ignite.transactions.TransactionIsolation.REPEATABLE_READ;
-
-/**
- * Test for portable objects stored in cache.
- */
-public abstract class GridCachePortableObjectsAbstractSelfTest extends GridCommonAbstractTest {
-    /** */
-    public static final TcpDiscoveryIpFinder IP_FINDER = new TcpDiscoveryVmIpFinder(true);
-
-    /** */
-    private static final int ENTRY_CNT = 100;
-
-    /** {@inheritDoc} */
-    @SuppressWarnings("unchecked")
-    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
-        IgniteConfiguration cfg = super.getConfiguration(gridName);
-
-        TcpDiscoverySpi disco = new TcpDiscoverySpi();
-
-        disco.setIpFinder(IP_FINDER);
-
-        cfg.setDiscoverySpi(disco);
-
-        CacheConfiguration cacheCfg = new CacheConfiguration();
-
-        cacheCfg.setCacheMode(cacheMode());
-        cacheCfg.setAtomicityMode(atomicityMode());
-        cacheCfg.setNearConfiguration(nearConfiguration());
-        cacheCfg.setWriteSynchronizationMode(FULL_SYNC);
-        cacheCfg.setCacheStoreFactory(singletonFactory(new TestStore()));
-        cacheCfg.setReadThrough(true);
-        cacheCfg.setWriteThrough(true);
-        cacheCfg.setLoadPreviousValue(true);
-        cacheCfg.setBackups(1);
-
-        if (offheapTiered()) {
-            cacheCfg.setMemoryMode(OFFHEAP_TIERED);
-            cacheCfg.setOffHeapMaxMemory(0);
-        }
-
-        cfg.setCacheConfiguration(cacheCfg);
-
-        cfg.setMarshaller(new PortableMarshaller());
-
-        return cfg;
-    }
-
-    /**
-     * @return {@code True} if should use OFFHEAP_TIERED mode.
-     */
-    protected boolean offheapTiered() {
-        return false;
-    }
-
-    /** {@inheritDoc} */
-    @Override protected void beforeTestsStarted() throws Exception {
-        startGridsMultiThreaded(gridCount());
-    }
-
-    /** {@inheritDoc} */
-    @Override protected void afterTestsStopped() throws Exception {
-        stopAllGrids();
-    }
-
-    /** {@inheritDoc} */
-    @Override protected void afterTest() throws Exception {
-        for (int i = 0; i < gridCount(); i++) {
-            GridCacheAdapter<Object, Object> c = ((IgniteKernal)grid(i)).internalCache();
-
-            for (GridCacheEntryEx e : c.map().entries0()) {
-                Object key = e.key().value(c.context().cacheObjectContext(), false);
-                Object val = CU.value(e.rawGet(), c.context(), false);
-
-                if (key instanceof PortableObject)
-                    assert ((PortableObjectImpl)key).detached() : val;
-
-                if (val instanceof PortableObject)
-                    assert ((PortableObjectImpl)val).detached() : val;
-            }
-        }
-
-        IgniteCache<Object, Object> c = jcache(0);
-
-        for (int i = 0; i < ENTRY_CNT; i++)
-            c.remove(i);
-
-        if (offheapTiered()) {
-            for (int k = 0; k < 100; k++)
-                c.remove(k);
-        }
-
-        assertEquals(0, c.size());
-    }
-
-    /**
-     * @return Cache mode.
-     */
-    protected abstract CacheMode cacheMode();
-
-    /**
-     * @return Atomicity mode.
-     */
-    protected abstract CacheAtomicityMode atomicityMode();
-
-    /**
-     * @return Distribution mode.
-     */
-    protected abstract NearCacheConfiguration nearConfiguration();
-
-    /**
-     * @return Grid count.
-     */
-    protected abstract int gridCount();
-
-    /**
-     * @throws Exception If failed.
-     */
-    @SuppressWarnings("unchecked")
-    public void testCircularReference() throws Exception {
-        IgniteCache c = keepPortableCache();
-
-        TestReferenceObject obj1 = new TestReferenceObject();
-
-        obj1.obj = new TestReferenceObject(obj1);
-
-        c.put(1, obj1);
-
-        PortableObject po = (PortableObject)c.get(1);
-
-        String str = po.toString();
-
-        log.info("toString: " + str);
-
-        assertNotNull(str);
-
-        assertTrue("Unexpected toString: " + str,
-            str.startsWith("TestReferenceObject") && str.contains("obj=TestReferenceObject ["));
-
-        TestReferenceObject obj1_r = po.deserialize();
-
-        assertNotNull(obj1_r);
-
-        TestReferenceObject obj2_r = obj1_r.obj;
-
-        assertNotNull(obj2_r);
-
-        assertSame(obj1_r, obj2_r.obj);
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testGet() throws Exception {
-        IgniteCache<Integer, TestObject> c = jcache(0);
-
-        for (int i = 0; i < ENTRY_CNT; i++)
-            c.put(i, new TestObject(i));
-
-        for (int i = 0; i < ENTRY_CNT; i++) {
-            TestObject obj = c.get(i);
-
-            assertEquals(i, obj.val);
-        }
-
-        IgniteCache<Integer, PortableObject> kpc = keepPortableCache();
-
-        for (int i = 0; i < ENTRY_CNT; i++) {
-            PortableObject po = kpc.get(i);
-
-            assertEquals(i, (int)po.field("val"));
-        }
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testIterator() throws Exception {
-        IgniteCache<Integer, TestObject> c = jcache(0);
-
-        Map<Integer, TestObject> entries = new HashMap<>();
-
-        for (int i = 0; i < ENTRY_CNT; i++) {
-            TestObject val = new TestObject(i);
-
-            c.put(i, val);
-
-            entries.put(i, val);
-        }
-
-        IgniteCache<Integer, PortableObject> prj = ((IgniteCacheProxy)c).keepPortable();
-
-        Iterator<Cache.Entry<Integer, PortableObject>> it = prj.iterator();
-
-        assertTrue(it.hasNext());
-
-        while (it.hasNext()) {
-            Cache.Entry<Integer, PortableObject> entry = it.next();
-
-            assertTrue(entries.containsKey(entry.getKey()));
-
-            TestObject o = entries.get(entry.getKey());
-
-            PortableObject po = entry.getValue();
-
-            assertEquals(o.val, (int)po.field("val"));
-
-            entries.remove(entry.getKey());
-        }
-
-        assertEquals(0, entries.size());
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testCollection() throws Exception {
-        IgniteCache<Integer, Collection<TestObject>> c = jcache(0);
-
-        for (int i = 0; i < ENTRY_CNT; i++) {
-            Collection<TestObject> col = new ArrayList<>(3);
-
-            for (int j = 0; j < 3; j++)
-                col.add(new TestObject(i * 10 + j));
-
-            c.put(i, col);
-        }
-
-        for (int i = 0; i < ENTRY_CNT; i++) {
-            Collection<TestObject> col = c.get(i);
-
-            assertEquals(3, col.size());
-
-            Iterator<TestObject> it = col.iterator();
-
-            for (int j = 0; j < 3; j++) {
-                assertTrue(it.hasNext());
-
-                assertEquals(i * 10 + j, it.next().val);
-            }
-        }
-
-        IgniteCache<Integer, Collection<PortableObject>> kpc = keepPortableCache();
-
-        for (int i = 0; i < ENTRY_CNT; i++) {
-            Collection<PortableObject> col = kpc.get(i);
-
-            assertEquals(3, col.size());
-
-            Iterator<PortableObject> it = col.iterator();
-
-            for (int j = 0; j < 3; j++) {
-                assertTrue(it.hasNext());
-
-                assertEquals(i * 10 + j, (int)it.next().field("val"));
-            }
-        }
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testMap() throws Exception {
-        IgniteCache<Integer, Map<Integer, TestObject>> c = jcache(0);
-
-        for (int i = 0; i < ENTRY_CNT; i++) {
-            Map<Integer, TestObject> map = U.newHashMap(3);
-
-            for (int j = 0; j < 3; j++) {
-                int idx = i * 10 + j;
-
-                map.put(idx, new TestObject(idx));
-            }
-
-            c.put(i, map);
-        }
-
-        for (int i = 0; i < ENTRY_CNT; i++) {
-            Map<Integer, TestObject> map = c.get(i);
-
-            assertEquals(3, map.size());
-
-            for (int j = 0; j < 3; j++) {
-                int idx = i * 10 + j;
-
-                assertEquals(idx, map.get(idx).val);
-            }
-        }
-
-        IgniteCache<Integer, Map<Integer, PortableObject>> kpc = keepPortableCache();
-
-        for (int i = 0; i < ENTRY_CNT; i++) {
-            Map<Integer, PortableObject> map = kpc.get(i);
-
-            assertEquals(3, map.size());
-
-            for (int j = 0; j < 3; j++) {
-                int idx = i * 10 + j;
-
-                assertEquals(idx, (int)map.get(idx).field("val"));
-            }
-        }
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testGetAsync() throws Exception {
-        IgniteCache<Integer, TestObject> c = jcache(0);
-
-        IgniteCache<Integer, TestObject> cacheAsync = c.withAsync();
-
-        for (int i = 0; i < ENTRY_CNT; i++)
-            c.put(i, new TestObject(i));
-
-        for (int i = 0; i < ENTRY_CNT; i++) {
-            cacheAsync.get(i);
-            TestObject obj = cacheAsync.<TestObject>future().get();
-
-            assertEquals(i, obj.val);
-        }
-
-        IgniteCache<Integer, PortableObject> kpc = keepPortableCache();
-
-        IgniteCache<Integer, PortableObject> cachePortableAsync = kpc.withAsync();
-
-        for (int i = 0; i < ENTRY_CNT; i++) {
-            cachePortableAsync.get(i);
-
-            PortableObject po = cachePortableAsync.<PortableObject>future().get();
-
-            assertEquals(i, (int)po.field("val"));
-        }
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testGetTx() throws Exception {
-        if (atomicityMode() != TRANSACTIONAL)
-            return;
-
-        IgniteCache<Integer, TestObject> c = jcache(0);
-
-        for (int i = 0; i < ENTRY_CNT; i++)
-            c.put(i, new TestObject(i));
-
-        for (int i = 0; i < ENTRY_CNT; i++) {
-            try (Transaction tx = grid(0).transactions().txStart(PESSIMISTIC, REPEATABLE_READ)) {
-                TestObject obj = c.get(i);
-
-                assertEquals(i, obj.val);
-
-                tx.commit();
-            }
-        }
-
-        for (int i = 0; i < ENTRY_CNT; i++) {
-            try (Transaction tx = grid(0).transactions().txStart(PESSIMISTIC, READ_COMMITTED)) {
-                TestObject obj = c.get(i);
-
-                assertEquals(i, obj.val);
-
-                tx.commit();
-            }
-        }
-
-        IgniteCache<Integer, PortableObject> kpc = keepPortableCache();
-
-        for (int i = 0; i < ENTRY_CNT; i++) {
-            try (Transaction tx = grid(0).transactions().txStart(PESSIMISTIC, REPEATABLE_READ)) {
-                PortableObject po = kpc.get(i);
-
-                assertEquals(i, (int)po.field("val"));
-
-                tx.commit();
-            }
-        }
-
-        for (int i = 0; i < ENTRY_CNT; i++) {
-            try (Transaction tx = grid(0).transactions().txStart(PESSIMISTIC, READ_COMMITTED)) {
-                PortableObject po = kpc.get(i);
-
-                assertEquals(i, (int)po.field("val"));
-
-                tx.commit();
-            }
-        }
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testGetAsyncTx() throws Exception {
-        if (atomicityMode() != TRANSACTIONAL)
-            return;
-
-        IgniteCache<Integer, TestObject> c = jcache(0);
-
-        IgniteCache<Integer, TestObject> cacheAsync = c.withAsync();
-
-        for (int i = 0; i < ENTRY_CNT; i++)
-            c.put(i, new TestObject(i));
-
-        for (int i = 0; i < ENTRY_CNT; i++) {
-            try (Transaction tx = grid(0).transactions().txStart(PESSIMISTIC, REPEATABLE_READ)) {
-                cacheAsync.get(i);
-
-                TestObject obj = cacheAsync.<TestObject>future().get();
-
-                assertEquals(i, obj.val);
-
-                tx.commit();
-            }
-        }
-
-        IgniteCache<Integer, PortableObject> kpc = keepPortableCache();
-        IgniteCache<Integer, PortableObject> cachePortableAsync = kpc.withAsync();
-
-        for (int i = 0; i < ENTRY_CNT; i++) {
-            try (Transaction tx = grid(0).transactions().txStart(PESSIMISTIC, REPEATABLE_READ)) {
-                cachePortableAsync.get(i);
-
-                PortableObject po = cachePortableAsync.<PortableObject>future().get();
-
-                assertEquals(i, (int)po.field("val"));
-
-                tx.commit();
-            }
-        }
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testGetAll() throws Exception {
-        IgniteCache<Integer, TestObject> c = jcache(0);
-
-        for (int i = 0; i < ENTRY_CNT; i++)
-            c.put(i, new TestObject(i));
-
-        for (int i = 0; i < ENTRY_CNT; ) {
-            Set<Integer> keys = new HashSet<>();
-
-            for (int j = 0; j < 10; j++)
-                keys.add(i++);
-
-            Map<Integer, TestObject> objs = c.getAll(keys);
-
-            assertEquals(10, objs.size());
-
-            for (Map.Entry<Integer, TestObject> e : objs.entrySet())
-                assertEquals(e.getKey().intValue(), e.getValue().val);
-        }
-
-        IgniteCache<Integer, PortableObject> kpc = keepPortableCache();
-
-        for (int i = 0; i < ENTRY_CNT; ) {
-            Set<Integer> keys = new HashSet<>();
-
-            for (int j = 0; j < 10; j++)
-                keys.add(i++);
-
-            Map<Integer, PortableObject> objs = kpc.getAll(keys);
-
-            assertEquals(10, objs.size());
-
-            for (Map.Entry<Integer, PortableObject> e : objs.entrySet())
-                assertEquals(new Integer(e.getKey().intValue()), e.getValue().field("val"));
-        }
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testGetAllAsync() throws Exception {
-        IgniteCache<Integer, TestObject> c = jcache(0);
-
-        IgniteCache<Integer, TestObject> cacheAsync = c.withAsync();
-
-        for (int i = 0; i < ENTRY_CNT; i++)
-            c.put(i, new TestObject(i));
-
-        for (int i = 0; i < ENTRY_CNT; ) {
-            Set<Integer> keys = new HashSet<>();
-
-            for (int j = 0; j < 10; j++)
-                keys.add(i++);
-
-            cacheAsync.getAll(keys);
-
-            Map<Integer, TestObject> objs = cacheAsync.<Map<Integer, TestObject>>future().get();
-
-            assertEquals(10, objs.size());
-
-            for (Map.Entry<Integer, TestObject> e : objs.entrySet())
-                assertEquals(e.getKey().intValue(), e.getValue().val);
-        }
-
-        IgniteCache<Integer, PortableObject> kpc = keepPortableCache();
-        IgniteCache<Integer, PortableObject> cachePortableAsync = kpc.withAsync();
-
-        for (int i = 0; i < ENTRY_CNT; ) {
-            Set<Integer> keys = new HashSet<>();
-
-            for (int j = 0; j < 10; j++)
-                keys.add(i++);
-
-
-            cachePortableAsync.getAll(keys);
-
-            Map<Integer, PortableObject> objs = cachePortableAsync.<Map<Integer, PortableObject>>future().get();
-
-            assertEquals(10, objs.size());
-
-            for (Map.Entry<Integer, PortableObject> e : objs.entrySet())
-                assertEquals(new Integer(e.getKey().intValue()), e.getValue().field("val"));
-        }
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testGetAllTx() throws Exception {
-        if (atomicityMode() != TRANSACTIONAL)
-            return;
-
-        IgniteCache<Integer, TestObject> c = jcache(0);
-
-        for (int i = 0; i < ENTRY_CNT; i++)
-            c.put(i, new TestObject(i));
-
-        for (int i = 0; i < ENTRY_CNT; ) {
-            Set<Integer> keys = new HashSet<>();
-
-            for (int j = 0; j < 10; j++)
-                keys.add(i++);
-
-            try (Transaction tx = grid(0).transactions().txStart(PESSIMISTIC, REPEATABLE_READ)) {
-                Map<Integer, TestObject> objs = c.getAll(keys);
-
-                assertEquals(10, objs.size());
-
-                for (Map.Entry<Integer, TestObject> e : objs.entrySet())
-                    assertEquals(e.getKey().intValue(), e.getValue().val);
-
-                tx.commit();
-            }
-
-            try (Transaction tx = grid(0).transactions().txStart(PESSIMISTIC, READ_COMMITTED)) {
-                Map<Integer, TestObject> objs = c.getAll(keys);
-
-                assertEquals(10, objs.size());
-
-                for (Map.Entry<Integer, TestObject> e : objs.entrySet())
-                    assertEquals(e.getKey().intValue(), e.getValue().val);
-
-                tx.commit();
-            }
-        }
-
-        IgniteCache<Integer, PortableObject> kpc = keepPortableCache();
-
-        for (int i = 0; i < ENTRY_CNT; ) {
-            Set<Integer> keys = new HashSet<>();
-
-            for (int j = 0; j < 10; j++)
-                keys.add(i++);
-
-            try (Transaction tx = grid(0).transactions().txStart(PESSIMISTIC, REPEATABLE_READ)) {
-                Map<Integer, PortableObject> objs = kpc.getAll(keys);
-
-                assertEquals(10, objs.size());
-
-                for (Map.Entry<Integer, PortableObject> e : objs.entrySet())
-                    assertEquals(new Integer(e.getKey().intValue()), e.getValue().field("val"));
-
-                tx.commit();
-            }
-
-            try (Transaction tx = grid(0).transactions().txStart(PESSIMISTIC, READ_COMMITTED)) {
-                Map<Integer, PortableObject> objs = kpc.getAll(keys);
-
-                assertEquals(10, objs.size());
-
-                for (Map.Entry<Integer, PortableObject> e : objs.entrySet())
-                    assertEquals(new Integer(e.getKey().intValue()), e.getValue().field("val"));
-
-                tx.commit();
-            }
-        }
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testGetAllAsyncTx() throws Exception {
-        if (atomicityMode() != TRANSACTIONAL)
-            return;
-
-        IgniteCache<Integer, TestObject> c = jcache(0);
-        IgniteCache<Integer, TestObject> cacheAsync = c.withAsync();
-
-        for (int i = 0; i < ENTRY_CNT; i++)
-            c.put(i, new TestObject(i));
-
-        for (int i = 0; i < ENTRY_CNT; ) {
-            Set<Integer> keys = new HashSet<>();
-
-            for (int j = 0; j < 10; j++)
-                keys.add(i++);
-
-            try (Transaction tx = grid(0).transactions().txStart(PESSIMISTIC, REPEATABLE_READ)) {
-                cacheAsync.getAll(keys);
-
-                Map<Integer, TestObject> objs = cacheAsync.<Map<Integer, TestObject>>future().get();
-
-                assertEquals(10, objs.size());
-
-                for (Map.Entry<Integer, TestObject> e : objs.entrySet())
-                    assertEquals(e.getKey().intValue(), e.getValue().val);
-
-                tx.commit();
-            }
-        }
-
-        IgniteCache<Integer, PortableObject> cache = keepPortableCache();
-
-        for (int i = 0; i < ENTRY_CNT; ) {
-            Set<Integer> keys = new HashSet<>();
-
-            for (int j = 0; j < 10; j++)
-                keys.add(i++);
-
-            IgniteCache<Integer, PortableObject> asyncCache = cache.withAsync();
-
-            try (Transaction tx = grid(0).transactions().txStart(PESSIMISTIC, REPEATABLE_READ)) {
-                asyncCache.getAll(keys);
-
-                Map<Integer, PortableObject> objs = asyncCache.<Map<Integer, PortableObject>>future().get();
-
-                assertEquals(10, objs.size());
-
-                for (Map.Entry<Integer, PortableObject> e : objs.entrySet())
-                    assertEquals(new Integer(e.getKey().intValue()), e.getValue().field("val"));
-
-                tx.commit();
-            }
-        }
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testLoadCache() throws Exception {
-        for (int i = 0; i < gridCount(); i++)
-            jcache(i).localLoadCache(null);
-
-        IgniteCache<Integer, TestObject> cache = jcache(0);
-
-        assertEquals(3, cache.size(CachePeekMode.PRIMARY));
-
-        assertEquals(1, cache.get(1).val);
-        assertEquals(2, cache.get(2).val);
-        assertEquals(3, cache.get(3).val);
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testLoadCacheAsync() throws Exception {
-        for (int i = 0; i < gridCount(); i++) {
-            IgniteCache<Object, Object> jcache = jcache(i).withAsync();
-
-            jcache.loadCache(null);
-
-            jcache.future().get();
-        }
-
-        IgniteCache<Integer, TestObject> cache = jcache(0);
-
-        assertEquals(3, cache.size(CachePeekMode.PRIMARY));
-
-        assertEquals(1, cache.get(1).val);
-        assertEquals(2, cache.get(2).val);
-        assertEquals(3, cache.get(3).val);
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testLoadCacheFilteredAsync() throws Exception {
-        for (int i = 0; i < gridCount(); i++) {
-            IgniteCache<Integer, TestObject> c = this.<Integer, TestObject>jcache(i).withAsync();
-
-            c.loadCache(new P2<Integer, TestObject>() {
-                @Override public boolean apply(Integer key, TestObject val) {
-                    return val.val < 3;
-                }
-            });
-
-            c.future().get();
-        }
-
-        IgniteCache<Integer, TestObject> cache = jcache(0);
-
-        assertEquals(2, cache.size(CachePeekMode.PRIMARY));
-
-        assertEquals(1, cache.get(1).val);
-        assertEquals(2, cache.get(2).val);
-
-        assertNull(cache.get(3));
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testTransform() throws Exception {
-        IgniteCache<Integer, PortableObject> c = keepPortableCache();
-
-        checkTransform(primaryKey(c));
-
-        if (cacheMode() != CacheMode.LOCAL) {
-            checkTransform(backupKey(c));
-
-            if (nearConfiguration() != null)
-                checkTransform(nearKey(c));
-        }
-    }
-
-    /**
-     * @return Cache with keep portable flag.
-     */
-    private <K, V> IgniteCache<K, V> keepPortableCache() {
-        return ignite(0).cache(null).withKeepPortable();
-    }
-
-    /**
-     * @param key Key.
-     * @throws Exception If failed.
-     */
-    private void checkTransform(Integer key) throws Exception {
-        log.info("Transform: " + key);
-
-        IgniteCache<Integer, PortableObject> c = keepPortableCache();
-
-        try {
-            c.invoke(key, new EntryProcessor<Integer, PortableObject, Void>() {
-                @Override public Void process(MutableEntry<Integer, PortableObject> e, Object... args) {
-                    PortableObject val = e.getValue();
-
-                    assertNull("Unexpected value: " + val, val);
-
-                    return null;
-                }
-            });
-
-            jcache(0).put(key, new TestObject(1));
-
-            c.invoke(key, new EntryProcessor<Integer, PortableObject, Void>() {
-                @Override public Void process(MutableEntry<Integer, PortableObject> e, Object... args) {
-                    PortableObject val = e.getValue();
-
-                    assertNotNull("Unexpected value: " + val, val);
-
-                    assertEquals(new Integer(1), val.field("val"));
-
-                    Ignite ignite = e.unwrap(Ignite.class);
-
-                    IgnitePortables portables = ignite.portables();
-
-                    PortableBuilder builder = portables.builder(val);
-
-                    builder.setField("val", 2);
-
-                    e.setValue(builder.build());
-
-                    return null;
-                }
-            });
-
-            PortableObject obj = c.get(key);
-
-            assertEquals(new Integer(2), obj.field("val"));
-
-            c.invoke(key, new EntryProcessor<Integer, PortableObject, Void>() {
-                @Override public Void process(MutableEntry<Integer, PortableObject> e, Object... args) {
-                    PortableObject val = e.getValue();
-
-                    assertNotNull("Unexpected value: " + val, val);
-
-                    assertEquals(new Integer(2), val.field("val"));
-
-                    e.setValue(val);
-
-                    return null;
-                }
-            });
-
-            obj = c.get(key);
-
-            assertEquals(new Integer(2), obj.field("val"));
-
-            c.invoke(key, new EntryProcessor<Integer, PortableObject, Void>() {
-                @Override public Void process(MutableEntry<Integer, PortableObject> e, Object... args) {
-                    PortableObject val = e.getValue();
-
-                    assertNotNull("Unexpected value: " + val, val);
-
-                    assertEquals(new Integer(2), val.field("val"));
-
-                    e.remove();
-
-                    return null;
-                }
-            });
-
-            assertNull(c.get(key));
-        }
-        finally {
-            c.remove(key);
-        }
-    }
-
-    /**
-     *
-     */
-    private static class TestObject implements PortableMarshalAware {
-        /** */
-        private int val;
-
-        /**
-         */
-        private TestObject() {
-            // No-op.
-        }
-
-        /**
-         * @param val Value.
-         */
-        private TestObject(int val) {
-            this.val = val;
-        }
-
-        /** {@inheritDoc} */
-        @Override public void writePortable(PortableWriter writer) throws PortableException {
-            writer.writeInt("val", val);
-        }
-
-        /** {@inheritDoc} */
-        @Override public void readPortable(PortableReader reader) throws PortableException {
-            val = reader.readInt("val");
-        }
-    }
-
-    /**
-     *
-     */
-    private static class TestReferenceObject implements PortableMarshalAware {
-        /** */
-        private TestReferenceObject obj;
-
-        /**
-         */
-        private TestReferenceObject() {
-            // No-op.
-        }
-
-        /**
-         * @param obj Object.
-         */
-        private TestReferenceObject(TestReferenceObject obj) {
-            this.obj = obj;
-        }
-
-        /** {@inheritDoc} */
-        @Override public void writePortable(PortableWriter writer) throws PortableException {
-            writer.writeObject("obj", obj);
-        }
-
-        /** {@inheritDoc} */
-        @Override public void readPortable(PortableReader reader) throws PortableException {
-            obj = reader.readObject("obj");
-        }
-    }
-
-    /**
-     *
-     */
-    private static class TestStore extends CacheStoreAdapter<Integer, Object> {
-        /** {@inheritDoc} */
-        @Override public void loadCache(IgniteBiInClosure<Integer, Object> clo, Object... args) {
-            for (int i = 1; i <= 3; i++)
-                clo.apply(i, new TestObject(i));
-        }
-
-        /** {@inheritDoc} */
-        @Nullable @Override public Object load(Integer key) {
-            return null;
-        }
-
-        /** {@inheritDoc} */
-        @Override public void write(Cache.Entry<? extends Integer, ?> e) {
-            // No-op.
-        }
-
-        /** {@inheritDoc} */
-        @Override public void delete(Object key) {
-            // No-op.
-        }
-    }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/71379a80/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/GridCachePortableStoreAbstractSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/GridCachePortableStoreAbstractSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/GridCachePortableStoreAbstractSelfTest.java
deleted file mode 100644
index 7c605b5..0000000
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/GridCachePortableStoreAbstractSelfTest.java
+++ /dev/null
@@ -1,297 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.ignite.internal.processors.cache.portable;
-
-import com.google.common.collect.ImmutableSet;
-import java.util.Arrays;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.Map;
-import java.util.Set;
-import javax.cache.Cache;
-import org.apache.ignite.cache.store.CacheStoreAdapter;
-import org.apache.ignite.configuration.CacheConfiguration;
-import org.apache.ignite.configuration.IgniteConfiguration;
-import org.apache.ignite.marshaller.portable.PortableMarshaller;
-import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi;
-import org.apache.ignite.spi.discovery.tcp.ipfinder.TcpDiscoveryIpFinder;
-import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder;
-import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
-import org.jetbrains.annotations.Nullable;
-import org.jsr166.ConcurrentHashMap8;
-
-/**
- * Tests for cache store with portables.
- */
-public abstract class GridCachePortableStoreAbstractSelfTest extends GridCommonAbstractTest {
-    /** */
-    private static final TcpDiscoveryIpFinder IP_FINDER = new TcpDiscoveryVmIpFinder(true);
-
-    /** */
-    private static final TestStore STORE = new TestStore();
-
-    /** {@inheritDoc} */
-    @SuppressWarnings("unchecked")
-    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
-        IgniteConfiguration cfg = super.getConfiguration(gridName);
-
-        PortableMarshaller marsh = new PortableMarshaller();
-
-        marsh.setClassNames(Arrays.asList(Key.class.getName(), Value.class.getName()));
-
-        cfg.setMarshaller(marsh);
-
-        CacheConfiguration cacheCfg = new CacheConfiguration();
-
-        cacheCfg.setCacheStoreFactory(singletonFactory(STORE));
-        cacheCfg.setKeepPortableInStore(keepPortableInStore());
-        cacheCfg.setReadThrough(true);
-        cacheCfg.setWriteThrough(true);
-        cacheCfg.setLoadPreviousValue(true);
-
-        cfg.setCacheConfiguration(cacheCfg);
-
-        TcpDiscoverySpi disco = new TcpDiscoverySpi();
-
-        disco.setIpFinder(IP_FINDER);
-
-        cfg.setDiscoverySpi(disco);
-
-        return cfg;
-    }
-
-    /**
-     * @return Keep portables in store flag.
-     */
-    protected abstract boolean keepPortableInStore();
-
-    /** {@inheritDoc} */
-    @Override protected void beforeTestsStarted() throws Exception {
-        startGrid();
-    }
-
-    /** {@inheritDoc} */
-    @Override protected void afterTestsStopped() throws Exception {
-        stopGrid();
-    }
-
-    /** {@inheritDoc} */
-    @Override protected void beforeTest() throws Exception {
-        STORE.map().clear();
-
-        jcache().clear();
-
-        assert jcache().size() == 0;
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testPut() throws Exception {
-        jcache().put(new Key(1), new Value(1));
-
-        checkMap(STORE.map(), 1);
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testPutAll() throws Exception {
-        Map<Object, Object> map = new HashMap<>();
-
-        for (int i = 1; i <= 3; i++)
-            map.put(new Key(i), new Value(i));
-
-        jcache().putAll(map);
-
-        checkMap(STORE.map(), 1, 2, 3);
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testLoad() throws Exception {
-        populateMap(STORE.map(), 1);
-
-        Object val = jcache().get(new Key(1));
-
-        assertTrue(String.valueOf(val), val instanceof Value);
-
-        assertEquals(1, ((Value)val).index());
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testLoadAll() throws Exception {
-        populateMap(STORE.map(), 1, 2, 3);
-
-        Set<Object> keys = new HashSet<>();
-
-        for (int i = 1; i <= 3; i++)
-            keys.add(new Key(i));
-
-        Map<Object, Object> res = jcache().getAll(keys);
-
-        assertEquals(3, res.size());
-
-        for (int i = 1; i <= 3; i++) {
-            Object val = res.get(new Key(i));
-
-            assertTrue(String.valueOf(val), val instanceof Value);
-
-            assertEquals(i, ((Value)val).index());
-        }
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testRemove() throws Exception {
-        for (int i = 1; i <= 3; i++)
-            jcache().put(new Key(i), new Value(i));
-
-        jcache().remove(new Key(1));
-
-        checkMap(STORE.map(), 2, 3);
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testRemoveAll() throws Exception {
-        for (int i = 1; i <= 3; i++)
-            jcache().put(new Key(i), new Value(i));
-
-        jcache().removeAll(ImmutableSet.of(new Key(1), new Key(2)));
-
-        checkMap(STORE.map(), 3);
-    }
-
-    /**
-     * @param map Map.
-     * @param idxs Indexes.
-     */
-    protected abstract void populateMap(Map<Object, Object> map, int... idxs);
-
-    /**
-     * @param map Map.
-     * @param idxs Indexes.
-     */
-    protected abstract void checkMap(Map<Object, Object> map, int... idxs);
-
-    /**
-     */
-    protected static class Key {
-        /** */
-        private int idx;
-
-        /**
-         * @param idx Index.
-         */
-        public Key(int idx) {
-            this.idx = idx;
-        }
-
-        /**
-         * @return Index.
-         */
-        int index() {
-            return idx;
-        }
-
-        /** {@inheritDoc} */
-        @Override public boolean equals(Object o) {
-            if (this == o)
-                return true;
-
-            if (o == null || getClass() != o.getClass())
-                return false;
-
-            Key key = (Key)o;
-
-            return idx == key.idx;
-        }
-
-        /** {@inheritDoc} */
-        @Override public int hashCode() {
-            return idx;
-        }
-
-        /** {@inheritDoc} */
-        @Override public String toString() {
-            return "Key [idx=" + idx + ']';
-        }
-    }
-
-    /**
-     */
-    protected static class Value {
-        /** */
-        private int idx;
-
-        /**
-         * @param idx Index.
-         */
-        public Value(int idx) {
-            this.idx = idx;
-        }
-
-        /**
-         * @return Index.
-         */
-        int index() {
-            return idx;
-        }
-
-        /** {@inheritDoc} */
-        @Override public String toString() {
-            return "Value [idx=" + idx + ']';
-        }
-    }
-
-    /**
-     *
-     */
-    private static class TestStore extends CacheStoreAdapter<Object, Object> {
-        /** */
-        private final Map<Object, Object> map = new ConcurrentHashMap8<>();
-
-        /** {@inheritDoc} */
-        @Nullable @Override public Object load(Object key) {
-            return map.get(key);
-        }
-
-        /** {@inheritDoc} */
-        @Override public void write(Cache.Entry<?, ?> e) {
-            map.put(e.getKey(), e.getValue());
-        }
-
-        /** {@inheritDoc} */
-        @Override public void delete(Object key) {
-            map.remove(key);
-        }
-
-        /**
-         * @return Map.
-         */
-        Map<Object, Object> map() {
-            return map;
-        }
-    }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/71379a80/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/GridCachePortableStoreObjectsSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/GridCachePortableStoreObjectsSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/GridCachePortableStoreObjectsSelfTest.java
deleted file mode 100644
index 1c1c99e..0000000
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/GridCachePortableStoreObjectsSelfTest.java
+++ /dev/null
@@ -1,55 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.ignite.internal.processors.cache.portable;
-
-import java.util.Map;
-
-/**
- * Tests for cache store with portables.
- */
-public class GridCachePortableStoreObjectsSelfTest extends GridCachePortableStoreAbstractSelfTest {
-    /** {@inheritDoc} */
-    @Override protected boolean keepPortableInStore() {
-        return false;
-    }
-
-    /** {@inheritDoc} */
-    @Override protected void populateMap(Map<Object, Object> map, int... idxs) {
-        assert map != null;
-        assert idxs != null;
-
-        for (int idx : idxs)
-            map.put(new Key(idx), new Value(idx));
-    }
-
-    /** {@inheritDoc} */
-    @Override protected void checkMap(Map<Object, Object> map, int... idxs) {
-        assert map != null;
-        assert idxs != null;
-
-        assertEquals(idxs.length, map.size());
-
-        for (int idx : idxs) {
-            Object val = map.get(new Key(idx));
-
-            assertTrue(String.valueOf(val), val instanceof Value);
-
-            assertEquals(idx, ((Value)val).index());
-        }
-    }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/71379a80/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/GridCachePortableStorePortablesSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/GridCachePortableStorePortablesSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/GridCachePortableStorePortablesSelfTest.java
deleted file mode 100644
index 5c0fc8e..0000000
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/GridCachePortableStorePortablesSelfTest.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.processors.cache.portable;
-
-import java.util.Map;
-import org.apache.ignite.portable.PortableObject;
-
-/**
- * Tests for cache store with portables.
- */
-public class GridCachePortableStorePortablesSelfTest extends GridCachePortableStoreAbstractSelfTest {
-    /** {@inheritDoc} */
-    @Override protected boolean keepPortableInStore() {
-        return true;
-    }
-
-    /** {@inheritDoc} */
-    @Override protected void populateMap(Map<Object, Object> map, int... idxs) {
-        assert map != null;
-        assert idxs != null;
-
-        for (int idx : idxs)
-            map.put(portable(new Key(idx)), portable(new Value(idx)));
-    }
-
-    /** {@inheritDoc} */
-    @Override protected void checkMap(Map<Object, Object> map, int... idxs) {
-        assert map != null;
-        assert idxs != null;
-
-        assertEquals(idxs.length, map.size());
-
-        for (int idx : idxs) {
-            Object val = map.get(portable(new Key(idx)));
-
-            assertTrue(String.valueOf(val), val instanceof PortableObject);
-
-            PortableObject po = (PortableObject)val;
-
-            assertEquals("Value", po.metaData().typeName());
-            assertEquals(new Integer(idx), po.field("idx"));
-        }
-    }
-
-    /**
-     * @param obj Object.
-     * @return Portable object.
-     */
-    private Object portable(Object obj) {
-        return grid().portables().toPortable(obj);
-    }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/71379a80/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/GridPortableCacheEntryMemorySizeSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/GridPortableCacheEntryMemorySizeSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/GridPortableCacheEntryMemorySizeSelfTest.java
deleted file mode 100644
index 0db650e..0000000
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/GridPortableCacheEntryMemorySizeSelfTest.java
+++ /dev/null
@@ -1,55 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.ignite.internal.processors.cache.portable;
-
-import org.apache.ignite.IgniteCheckedException;
-import org.apache.ignite.internal.portable.PortableContext;
-import org.apache.ignite.internal.portable.PortableMetaDataHandler;
-import org.apache.ignite.internal.processors.cache.GridCacheEntryMemorySizeSelfTest;
-import org.apache.ignite.internal.util.IgniteUtils;
-import org.apache.ignite.marshaller.Marshaller;
-import org.apache.ignite.marshaller.MarshallerContextTestImpl;
-import org.apache.ignite.marshaller.portable.PortableMarshaller;
-import org.apache.ignite.portable.PortableException;
-import org.apache.ignite.portable.PortableMetadata;
-
-/**
- *
- */
-public class GridPortableCacheEntryMemorySizeSelfTest extends GridCacheEntryMemorySizeSelfTest {
-    /** {@inheritDoc} */
-    @Override protected Marshaller createMarshaller() throws IgniteCheckedException {
-        PortableMarshaller marsh = new PortableMarshaller();
-
-        marsh.setContext(new MarshallerContextTestImpl(null));
-
-        PortableContext pCtx = new PortableContext(new PortableMetaDataHandler() {
-            @Override public void addMeta(int typeId, PortableMetadata meta) throws PortableException {
-                // No-op
-            }
-
-            @Override public PortableMetadata metadata(int typeId) throws PortableException {
-                return null;
-            }
-        }, null);
-
-        IgniteUtils.invoke(PortableMarshaller.class, marsh, "setPortableContext", pCtx);
-
-        return marsh;
-    }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/71379a80/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/GridPortableDuplicateIndexObjectsAbstractSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/GridPortableDuplicateIndexObjectsAbstractSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/GridPortableDuplicateIndexObjectsAbstractSelfTest.java
deleted file mode 100644
index a1a623b..0000000
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/GridPortableDuplicateIndexObjectsAbstractSelfTest.java
+++ /dev/null
@@ -1,158 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.ignite.internal.processors.cache.portable;
-
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import org.apache.ignite.IgniteCache;
-import org.apache.ignite.cache.CacheAtomicityMode;
-import org.apache.ignite.cache.CacheMode;
-import org.apache.ignite.cache.CacheTypeMetadata;
-import org.apache.ignite.cache.query.SqlFieldsQuery;
-import org.apache.ignite.configuration.CacheConfiguration;
-import org.apache.ignite.configuration.IgniteConfiguration;
-import org.apache.ignite.internal.processors.cache.GridCacheAbstractSelfTest;
-import org.apache.ignite.internal.util.typedef.F;
-import org.apache.ignite.marshaller.portable.PortableMarshaller;
-import org.apache.ignite.portable.PortableObject;
-
-/**
- * Tests that portable object is the same in cache entry and in index.
- */
-public abstract class GridPortableDuplicateIndexObjectsAbstractSelfTest extends GridCacheAbstractSelfTest {
-    /** {@inheritDoc} */
-    @Override protected int gridCount() {
-        return 1;
-    }
-
-    /** {@inheritDoc} */
-    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
-        IgniteConfiguration cfg = super.getConfiguration(gridName);
-
-        PortableMarshaller marsh = new PortableMarshaller();
-
-        marsh.setClassNames(Collections.singletonList(TestPortable.class.getName()));
-
-        cfg.setMarshaller(marsh);
-
-        return cfg;
-    }
-
-    /** {@inheritDoc} */
-    @Override protected CacheConfiguration cacheConfiguration(String gridName) throws Exception {
-        CacheConfiguration ccfg = super.cacheConfiguration(gridName);
-
-        ccfg.setCopyOnRead(false);
-
-        CacheTypeMetadata meta = new CacheTypeMetadata();
-
-        meta.setKeyType(Integer.class);
-        meta.setValueType(TestPortable.class.getName());
-
-        Map<String, Class<?>> idx = new HashMap<>();
-
-        idx.put("fieldOne", String.class);
-        idx.put("fieldTwo", Integer.class);
-
-        meta.setAscendingFields(idx);
-
-        ccfg.setTypeMetadata(Collections.singletonList(meta));
-
-        return ccfg;
-    }
-
-    /** {@inheritDoc} */
-    @Override public abstract CacheAtomicityMode atomicityMode();
-
-    /** {@inheritDoc} */
-    @Override public abstract CacheMode cacheMode();
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testIndexReferences() throws Exception {
-        IgniteCache<Integer, TestPortable> cache = grid(0).cache(null);
-
-        String fieldOneVal = "123";
-        int fieldTwoVal = 123;
-        int key = 0;
-
-        cache.put(key, new TestPortable(fieldOneVal, fieldTwoVal));
-
-        IgniteCache<Integer, PortableObject> prj = grid(0).cache(null).withKeepPortable();
-
-        PortableObject cacheVal = prj.get(key);
-
-        assertEquals(fieldOneVal, cacheVal.field("fieldOne"));
-        assertEquals(new Integer(fieldTwoVal), cacheVal.field("fieldTwo"));
-
-        List<?> row = F.first(prj.query(new SqlFieldsQuery("select _val from " +
-            "TestPortable where _key = ?").setArgs(key)).getAll());
-
-        assertEquals(1, row.size());
-
-        PortableObject qryVal = (PortableObject)row.get(0);
-
-        assertEquals(fieldOneVal, qryVal.field("fieldOne"));
-        assertEquals(new Integer(fieldTwoVal), qryVal.field("fieldTwo"));
-        assertSame(cacheVal, qryVal);
-    }
-
-    /**
-     * Test portable object.
-     */
-    private static class TestPortable {
-        /** */
-        private String fieldOne;
-
-        /** */
-        private int fieldTwo;
-
-        /**
-         *
-         */
-        private TestPortable() {
-            // No-op.
-        }
-
-        /**
-         * @param fieldOne Field one.
-         * @param fieldTwo Field two.
-         */
-        private TestPortable(String fieldOne, int fieldTwo) {
-            this.fieldOne = fieldOne;
-            this.fieldTwo = fieldTwo;
-        }
-
-        /**
-         * @return Field one.
-         */
-        public String fieldOne() {
-            return fieldOne;
-        }
-
-        /**
-         * @return Field two.
-         */
-        public int fieldTwo() {
-            return fieldTwo;
-        }
-    }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/71379a80/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/datastreaming/DataStreamProcessorPortableSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/datastreaming/DataStreamProcessorPortableSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/datastreaming/DataStreamProcessorPortableSelfTest.java
deleted file mode 100644
index 836440a..0000000
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/datastreaming/DataStreamProcessorPortableSelfTest.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.processors.cache.portable.datastreaming;
-
-import java.util.Collection;
-import java.util.Map;
-import org.apache.ignite.IgniteCache;
-import org.apache.ignite.configuration.IgniteConfiguration;
-import org.apache.ignite.internal.processors.datastreamer.DataStreamProcessorSelfTest;
-import org.apache.ignite.marshaller.portable.PortableMarshaller;
-import org.apache.ignite.portable.PortableObject;
-import org.apache.ignite.stream.StreamReceiver;
-
-/**
- *
- */
-public class DataStreamProcessorPortableSelfTest extends DataStreamProcessorSelfTest {
-    /** {@inheritDoc} */
-    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
-        IgniteConfiguration cfg = super.getConfiguration(gridName);
-
-        PortableMarshaller marsh = new PortableMarshaller();
-
-        cfg.setMarshaller(marsh);
-
-        return cfg;
-    }
-
-    /** {@inheritDoc} */
-    @Override protected StreamReceiver<String, TestObject> getStreamReceiver() {
-        return new TestDataReceiver();
-    }
-
-    /**
-     *
-     */
-    private static class TestDataReceiver implements StreamReceiver<String, TestObject> {
-        /** {@inheritDoc} */
-        @Override public void receive(IgniteCache<String, TestObject> cache,
-            Collection<Map.Entry<String, TestObject>> entries) {
-            for (Map.Entry<String, TestObject> e : entries) {
-                assertTrue(e.getKey() instanceof String);
-                assertTrue(e.getValue() instanceof PortableObject);
-
-                TestObject obj = ((PortableObject)e.getValue()).deserialize();
-
-                cache.put(e.getKey(), new TestObject(obj.val + 1));
-            }
-        }
-    }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/71379a80/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/datastreaming/GridDataStreamerImplSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/datastreaming/GridDataStreamerImplSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/datastreaming/GridDataStreamerImplSelfTest.java
deleted file mode 100644
index 2f7bdb0..0000000
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/datastreaming/GridDataStreamerImplSelfTest.java
+++ /dev/null
@@ -1,345 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.ignite.internal.processors.cache.portable.datastreaming;
-
-import java.io.Serializable;
-import java.util.Map;
-import java.util.Random;
-import org.apache.ignite.Ignite;
-import org.apache.ignite.IgniteCache;
-import org.apache.ignite.IgniteDataStreamer;
-import org.apache.ignite.cache.CachePeekMode;
-import org.apache.ignite.cluster.ClusterNode;
-import org.apache.ignite.configuration.CacheConfiguration;
-import org.apache.ignite.configuration.IgniteConfiguration;
-import org.apache.ignite.internal.processors.cache.IgniteCacheProxy;
-import org.apache.ignite.internal.util.typedef.G;
-import org.apache.ignite.internal.util.typedef.internal.U;
-import org.apache.ignite.marshaller.portable.PortableMarshaller;
-import org.apache.ignite.portable.PortableException;
-import org.apache.ignite.portable.PortableMarshalAware;
-import org.apache.ignite.portable.PortableObject;
-import org.apache.ignite.portable.PortableReader;
-import org.apache.ignite.portable.PortableWriter;
-import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi;
-import org.apache.ignite.spi.discovery.tcp.ipfinder.TcpDiscoveryIpFinder;
-import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder;
-import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
-
-import static org.apache.ignite.cache.CacheMode.PARTITIONED;
-import static org.apache.ignite.cache.CacheWriteSynchronizationMode.FULL_SYNC;
-
-/**
- * Tests for {@code IgniteDataStreamerImpl}.
- */
-public class GridDataStreamerImplSelfTest extends GridCommonAbstractTest {
-    /** IP finder. */
-    private static final TcpDiscoveryIpFinder IP_FINDER = new TcpDiscoveryVmIpFinder(true);
-
-    /** Number of keys to load via data streamer. */
-    private static final int KEYS_COUNT = 1000;
-
-    /** Flag indicating should be cache configured with portables or not.  */
-    private static boolean portables;
-
-    /** {@inheritDoc} */
-    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
-        IgniteConfiguration cfg = super.getConfiguration(gridName);
-
-        TcpDiscoverySpi discoSpi = new TcpDiscoverySpi();
-        discoSpi.setIpFinder(IP_FINDER);
-
-        cfg.setDiscoverySpi(discoSpi);
-
-        if (portables) {
-            PortableMarshaller marsh = new PortableMarshaller();
-
-            cfg.setMarshaller(marsh);
-        }
-
-        cfg.setCacheConfiguration(cacheConfiguration());
-
-        return cfg;
-    }
-
-    /**
-     * Gets cache configuration.
-     *
-     * @return Cache configuration.
-     */
-    private CacheConfiguration cacheConfiguration() {
-        CacheConfiguration cacheCfg = defaultCacheConfiguration();
-
-        cacheCfg.setCacheMode(PARTITIONED);
-        cacheCfg.setNearConfiguration(null);
-        cacheCfg.setBackups(0);
-        cacheCfg.setWriteSynchronizationMode(FULL_SYNC);
-
-        return cacheCfg;
-    }
-
-    /**
-     * Data streamer should correctly load entries from HashMap in case of grids with more than one node
-     *  and with GridOptimizedMarshaller that requires serializable.
-     *
-     * @throws Exception If failed.
-     */
-    public void testAddDataFromMap() throws Exception {
-        try {
-            portables = false;
-
-            startGrids(2);
-
-            awaitPartitionMapExchange();
-
-            Ignite g0 = grid(0);
-
-            IgniteDataStreamer<Integer, String> dataLdr = g0.dataStreamer(null);
-
-            Map<Integer, String> map = U.newHashMap(KEYS_COUNT);
-
-            for (int i = 0; i < KEYS_COUNT; i ++)
-                map.put(i, String.valueOf(i));
-
-            dataLdr.addData(map);
-
-            dataLdr.close();
-
-            checkDistribution(grid(0));
-
-            checkDistribution(grid(1));
-
-            // Check several random keys in cache.
-            Random rnd = new Random();
-
-            IgniteCache<Integer, String> c0 = g0.cache(null);
-
-            for (int i = 0; i < 100; i ++) {
-                Integer k = rnd.nextInt(KEYS_COUNT);
-
-                String v = c0.get(k);
-
-                assertEquals(k.toString(), v);
-            }
-        }
-        finally {
-            G.stopAll(true);
-        }
-    }
-
-    /**
-     * Data streamer should add portable object that weren't registered explicitly.
-     *
-     * @throws Exception If failed.
-     */
-    public void testAddMissingPortable() throws Exception {
-        try {
-            portables = true;
-
-            startGrids(2);
-
-            awaitPartitionMapExchange();
-
-            Ignite g0 = grid(0);
-
-            IgniteDataStreamer<Integer, TestObject2> dataLdr = g0.dataStreamer(null);
-
-            dataLdr.perNodeBufferSize(1);
-            dataLdr.autoFlushFrequency(1L);
-
-            Map<Integer, TestObject2> map = U.newHashMap(KEYS_COUNT);
-
-            for (int i = 0; i < KEYS_COUNT; i ++)
-                map.put(i, new TestObject2(i));
-
-            dataLdr.addData(map).get();
-
-            dataLdr.close();
-        }
-        finally {
-            G.stopAll(true);
-        }
-    }
-
-    /**
-     * Data streamer should correctly load portable entries from HashMap in case of grids with more than one node
-     *  and with GridOptimizedMarshaller that requires serializable.
-     *
-     * @throws Exception If failed.
-     */
-    public void testAddPortableDataFromMap() throws Exception {
-        try {
-            portables = true;
-
-            startGrids(2);
-
-            awaitPartitionMapExchange();
-
-            Ignite g0 = grid(0);
-
-            IgniteDataStreamer<Integer, TestObject> dataLdr = g0.dataStreamer(null);
-
-            Map<Integer, TestObject> map = U.newHashMap(KEYS_COUNT);
-
-            for (int i = 0; i < KEYS_COUNT; i ++)
-                map.put(i, new TestObject(i));
-
-            dataLdr.addData(map);
-
-            dataLdr.close(false);
-
-            checkDistribution(grid(0));
-
-            checkDistribution(grid(1));
-
-            // Read random keys. Take values as TestObject.
-            Random rnd = new Random();
-
-            IgniteCache<Integer, TestObject> c = g0.cache(null);
-
-            for (int i = 0; i < 100; i ++) {
-                Integer k = rnd.nextInt(KEYS_COUNT);
-
-                TestObject v = c.get(k);
-
-                assertEquals(k, v.val());
-            }
-
-            // Read random keys. Take values as PortableObject.
-            IgniteCache<Integer, PortableObject> c2 = ((IgniteCacheProxy)c).keepPortable();
-
-            for (int i = 0; i < 100; i ++) {
-                Integer k = rnd.nextInt(KEYS_COUNT);
-
-                PortableObject v = c2.get(k);
-
-                assertEquals(k, v.field("val"));
-            }
-        }
-        finally {
-            G.stopAll(true);
-        }
-    }
-
-    /**
-     * Check that keys correctly destributed by nodes after data streamer.
-     *
-     * @param g Grid to check.
-     */
-    private void checkDistribution(Ignite g) {
-        ClusterNode n = g.cluster().localNode();
-        IgniteCache c = g.cache(null);
-
-        // Check that data streamer correctly split data by nodes.
-        for (int i = 0; i < KEYS_COUNT; i ++) {
-            if (g.affinity(null).isPrimary(n, i))
-                assertNotNull(c.localPeek(i, CachePeekMode.ONHEAP));
-            else
-                assertNull(c.localPeek(i, CachePeekMode.ONHEAP));
-        }
-    }
-
-    /**
-     */
-    private static class TestObject implements PortableMarshalAware, Serializable {
-        /** */
-        private int val;
-
-        /**
-         *
-         */
-        private TestObject() {
-            // No-op.
-        }
-
-        /**
-         * @param val Value.
-         */
-        private TestObject(int val) {
-            this.val = val;
-        }
-
-        public Integer val() {
-            return val;
-        }
-
-        /** {@inheritDoc} */
-        @Override public int hashCode() {
-            return val;
-        }
-
-        /** {@inheritDoc} */
-        @Override public boolean equals(Object obj) {
-            return obj instanceof TestObject && ((TestObject)obj).val == val;
-        }
-
-        /** {@inheritDoc} */
-        @Override public void writePortable(PortableWriter writer) throws PortableException {
-            writer.writeInt("val", val);
-        }
-
-        /** {@inheritDoc} */
-        @Override public void readPortable(PortableReader reader) throws PortableException {
-            val = reader.readInt("val");
-        }
-    }
-
-    /**
-     */
-    private static class TestObject2 implements PortableMarshalAware, Serializable {
-        /** */
-        private int val;
-
-        /**
-         */
-        private TestObject2() {
-            // No-op.
-        }
-
-        /**
-         * @param val Value.
-         */
-        private TestObject2(int val) {
-            this.val = val;
-        }
-
-        public Integer val() {
-            return val;
-        }
-
-        /** {@inheritDoc} */
-        @Override public int hashCode() {
-            return val;
-        }
-
-        /** {@inheritDoc} */
-        @Override public boolean equals(Object obj) {
-            return obj instanceof TestObject2 && ((TestObject2)obj).val == val;
-        }
-
-        /** {@inheritDoc} */
-        @Override public void writePortable(PortableWriter writer) throws PortableException {
-            writer.writeInt("val", val);
-        }
-
-        /** {@inheritDoc} */
-        @Override public void readPortable(PortableReader reader) throws PortableException {
-            val = reader.readInt("val");
-        }
-    }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/71379a80/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/distributed/dht/GridCacheAffinityRoutingPortableSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/distributed/dht/GridCacheAffinityRoutingPortableSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/distributed/dht/GridCacheAffinityRoutingPortableSelfTest.java
deleted file mode 100644
index 155ba48..0000000
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/distributed/dht/GridCacheAffinityRoutingPortableSelfTest.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.processors.cache.portable.distributed.dht;
-
-import java.util.Collections;
-import org.apache.ignite.configuration.IgniteConfiguration;
-import org.apache.ignite.internal.processors.cache.GridCacheAffinityRoutingSelfTest;
-import org.apache.ignite.marshaller.portable.PortableMarshaller;
-import org.apache.ignite.portable.PortableTypeConfiguration;
-
-/**
- *
- */
-public class GridCacheAffinityRoutingPortableSelfTest extends GridCacheAffinityRoutingSelfTest {
-    /** {@inheritDoc} */
-    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
-        IgniteConfiguration cfg = super.getConfiguration(gridName);
-
-        PortableTypeConfiguration typeCfg = new PortableTypeConfiguration();
-
-        typeCfg.setClassName(AffinityTestKey.class.getName());
-        typeCfg.setAffinityKeyFieldName("affKey");
-
-        PortableMarshaller marsh = new PortableMarshaller();
-
-        marsh.setTypeConfigurations(Collections.singleton(typeCfg));
-
-        cfg.setMarshaller(marsh);
-
-        return cfg;
-    }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/71379a80/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/distributed/dht/GridCacheAtomicPartitionedOnlyPortableDataStreamerMultiNodeSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/distributed/dht/GridCacheAtomicPartitionedOnlyPortableDataStreamerMultiNodeSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/distributed/dht/GridCacheAtomicPartitionedOnlyPortableDataStreamerMultiNodeSelfTest.java
deleted file mode 100644
index b3b988e..0000000
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/distributed/dht/GridCacheAtomicPartitionedOnlyPortableDataStreamerMultiNodeSelfTest.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.processors.cache.portable.distributed.dht;
-
-/**
- *
- */
-public class GridCacheAtomicPartitionedOnlyPortableDataStreamerMultiNodeSelfTest extends
-    GridCacheAtomicPartitionedOnlyPortableDataStreamerMultithreadedSelfTest {
-    /** {@inheritDoc} */
-    @Override protected int gridCount() {
-        return 4;
-    }
-}
\ No newline at end of file


[17/55] [abbrv] ignite git commit: IGNITE-1402 - Fixed logging categories

Posted by ag...@apache.org.
IGNITE-1402 - Fixed logging categories


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

Branch: refs/heads/ignite-1171
Commit: cc0d1f57141d24b42c0f0ee0b7572e43aa47b339
Parents: cb9b766
Author: Valentin Kulichenko <va...@gmail.com>
Authored: Mon Sep 14 17:09:58 2015 -0700
Committer: Valentin Kulichenko <va...@gmail.com>
Committed: Mon Sep 14 17:09:58 2015 -0700

----------------------------------------------------------------------
 .../org/apache/ignite/internal/GridKernalContext.java     |  7 ++++---
 .../org/apache/ignite/internal/GridKernalContextImpl.java | 10 +++++-----
 .../java/org/apache/ignite/internal/GridLoggerProxy.java  |  6 ++++--
 .../java/org/apache/ignite/internal/IgniteKernal.java     |  6 +++---
 .../ignite/internal/executor/GridExecutorService.java     |  4 ++--
 .../managers/deployment/GridDeploymentStoreAdapter.java   |  4 ++--
 .../internal/processors/cache/GridCacheAdapter.java       |  4 ++--
 .../processors/cache/GridCacheClearAllRunnable.java       |  4 ++--
 .../ignite/internal/processors/cache/GridCacheLogger.java |  4 ++--
 .../internal/processors/cache/GridCacheSharedContext.java |  4 ++--
 .../datastructures/GridCacheAtomicLongImpl.java           |  4 ++--
 .../datastructures/GridCacheAtomicReferenceImpl.java      |  4 ++--
 .../datastructures/GridCacheAtomicSequenceImpl.java       |  4 ++--
 .../datastructures/GridCacheAtomicStampedImpl.java        |  4 ++--
 .../datastructures/GridCacheCountDownLatchImpl.java       |  4 ++--
 .../internal/processors/igfs/IgfsFragmentizerManager.java |  8 +++++---
 .../internal/processors/igfs/IgfsServerManager.java       |  5 +++--
 .../ignite/internal/processors/job/GridJobWorker.java     |  4 ++--
 .../ignite/internal/processors/task/GridTaskWorker.java   |  4 ++--
 19 files changed, 50 insertions(+), 44 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/cc0d1f57/modules/core/src/main/java/org/apache/ignite/internal/GridKernalContext.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/GridKernalContext.java b/modules/core/src/main/java/org/apache/ignite/internal/GridKernalContext.java
index 3f73c84..c0b50a2 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/GridKernalContext.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/GridKernalContext.java
@@ -94,11 +94,12 @@ public interface GridKernalContext extends Iterable<GridComponent> {
     public String gridName();
 
     /**
-     * Gets logger.
+     * Gets logger for given category.
      *
+     * @param ctgr Category.
      * @return Logger.
      */
-    public IgniteLogger log();
+    public IgniteLogger log(String ctgr);
 
     /**
      * Gets logger for given class.
@@ -572,4 +573,4 @@ public interface GridKernalContext extends Iterable<GridComponent> {
      * @return Platform processor.
      */
     public PlatformProcessor platform();
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/cc0d1f57/modules/core/src/main/java/org/apache/ignite/internal/GridKernalContextImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/GridKernalContextImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/GridKernalContextImpl.java
index 6101836..ebf83bd 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/GridKernalContextImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/GridKernalContextImpl.java
@@ -754,13 +754,13 @@ public class GridKernalContextImpl implements GridKernalContext, Externalizable
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteLogger log() {
-        return config().getGridLogger();
+    @Override public IgniteLogger log(String ctgr) {
+        return config().getGridLogger().getLogger(ctgr);
     }
 
     /** {@inheritDoc} */
     @Override public IgniteLogger log(Class<?> cls) {
-        return config().getGridLogger().getLogger(cls);
+        return log(cls.getName());
     }
 
     /** {@inheritDoc} */
@@ -808,7 +808,7 @@ public class GridKernalContextImpl implements GridKernalContext, Externalizable
 
     /** {@inheritDoc} */
     @Override public String userVersion(ClassLoader ldr) {
-        return spring != null ? spring.userVersion(ldr, log()) : U.DFLT_USER_VERSION;
+        return spring != null ? spring.userVersion(ldr, log(spring.getClass())) : U.DFLT_USER_VERSION;
     }
 
     /** {@inheritDoc} */
@@ -967,4 +967,4 @@ public class GridKernalContextImpl implements GridKernalContext, Externalizable
     @Override public String toString() {
         return S.toString(GridKernalContextImpl.class, this);
     }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/cc0d1f57/modules/core/src/main/java/org/apache/ignite/internal/GridLoggerProxy.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/GridLoggerProxy.java b/modules/core/src/main/java/org/apache/ignite/internal/GridLoggerProxy.java
index 742552b..f6bddca 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/GridLoggerProxy.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/GridLoggerProxy.java
@@ -206,7 +206,9 @@ public class GridLoggerProxy implements IgniteLogger, LifecycleAware, Externaliz
             String gridNameR = t.get1();
             Object ctgrR = t.get2();
 
-            return IgnitionEx.gridx(gridNameR).log().getLogger(ctgrR);
+            IgniteLogger log = IgnitionEx.gridx(gridNameR).log();
+
+            return ctgrR != null ? log.getLogger(ctgrR) : log;
         }
         catch (IllegalStateException e) {
             throw U.withCause(new InvalidObjectException(e.getMessage()), e);
@@ -220,4 +222,4 @@ public class GridLoggerProxy implements IgniteLogger, LifecycleAware, Externaliz
     @Override public String toString() {
         return S.toString(GridLoggerProxy.class, this);
     }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/cc0d1f57/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 abab1f3..daf7d23 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
@@ -715,8 +715,8 @@ public class IgniteKernal implements IgniteEx, IgniteMXBean, Externalizable {
 
         this.cfg = cfg;
 
-        log = (GridLoggerProxy)cfg.getGridLogger().getLogger(getClass().getName() +
-            (gridName != null ? '%' + gridName : ""));
+        log = (GridLoggerProxy)cfg.getGridLogger().getLogger(
+            getClass().getName() + (gridName != null ? '%' + gridName : ""));
 
         RuntimeMXBean rtBean = ManagementFactory.getRuntimeMXBean();
 
@@ -2247,7 +2247,7 @@ public class IgniteKernal implements IgniteEx, IgniteMXBean, Externalizable {
 
     /** {@inheritDoc} */
     @Override public IgniteLogger log() {
-        return log;
+        return cfg.getGridLogger();
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/cc0d1f57/modules/core/src/main/java/org/apache/ignite/internal/executor/GridExecutorService.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/executor/GridExecutorService.java b/modules/core/src/main/java/org/apache/ignite/internal/executor/GridExecutorService.java
index 860504f..e8db977 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/executor/GridExecutorService.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/executor/GridExecutorService.java
@@ -135,7 +135,7 @@ public class GridExecutorService implements ExecutorService, Externalizable {
 
         this.prj = prj;
         this.ctx = ctx;
-        this.log = ctx.log().getLogger(GridExecutorService.class);
+        this.log = ctx.log(GridExecutorService.class);
     }
 
     /** {@inheritDoc} */
@@ -733,4 +733,4 @@ public class GridExecutorService implements ExecutorService, Externalizable {
             }
         }
     }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/cc0d1f57/modules/core/src/main/java/org/apache/ignite/internal/managers/deployment/GridDeploymentStoreAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/deployment/GridDeploymentStoreAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/deployment/GridDeploymentStoreAdapter.java
index c2690b1..fcbc801 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/managers/deployment/GridDeploymentStoreAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/deployment/GridDeploymentStoreAdapter.java
@@ -57,7 +57,7 @@ abstract class GridDeploymentStoreAdapter implements GridDeploymentStore {
         this.ctx = ctx;
         this.comm = comm;
 
-        log = ctx.config().getGridLogger().getLogger(getClass());
+        log = ctx.log(getClass());
     }
 
     /**
@@ -155,4 +155,4 @@ abstract class GridDeploymentStoreAdapter implements GridDeploymentStore {
     @Override public String toString() {
         return S.toString(GridDeploymentStoreAdapter.class, this);
     }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/cc0d1f57/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java
index 4460a2a..9329e94 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java
@@ -318,7 +318,7 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
 
         this.map = map;
 
-        log = ctx.gridConfig().getGridLogger().getLogger(getClass());
+        log = ctx.logger(getClass());
 
         metrics = new CacheMetricsImpl(ctx);
 
@@ -6028,4 +6028,4 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
             return null;
         }
     }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/cc0d1f57/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheClearAllRunnable.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheClearAllRunnable.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheClearAllRunnable.java
index e517e3a..feafc58 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheClearAllRunnable.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheClearAllRunnable.java
@@ -70,7 +70,7 @@ public class GridCacheClearAllRunnable<K, V> implements Runnable {
         this.totalCnt = totalCnt;
 
         ctx = cache.context();
-        log = ctx.gridConfig().getGridLogger().getLogger(getClass());
+        log = ctx.logger(getClass());
     }
 
     /** {@inheritDoc} */
@@ -176,4 +176,4 @@ public class GridCacheClearAllRunnable<K, V> implements Runnable {
     @Override public String toString() {
         return S.toString(GridCacheClearAllRunnable.class, this);
     }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/cc0d1f57/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheLogger.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheLogger.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheLogger.java
index f86e445..75547fb 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheLogger.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheLogger.java
@@ -72,7 +72,7 @@ class GridCacheLogger implements IgniteLogger, Externalizable {
 
         cacheName = '<' + cctx.namexx() + "> ";
 
-        log = cctx.kernalContext().log().getLogger(ctgr);
+        log = cctx.kernalContext().log(ctgr);
     }
 
     /**
@@ -190,4 +190,4 @@ class GridCacheLogger implements IgniteLogger, Externalizable {
     @Override public String toString() {
         return S.toString(GridCacheLogger.class, this);
     }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/cc0d1f57/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheSharedContext.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheSharedContext.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheSharedContext.java
index 90e0921..13e390a 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheSharedContext.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheSharedContext.java
@@ -520,7 +520,7 @@ public class GridCacheSharedContext<K, V> {
      * @return Logger.
      */
     public IgniteLogger logger(String category) {
-        return kernalCtx.log().getLogger(category);
+        return kernalCtx.log(category);
     }
 
     /**
@@ -685,4 +685,4 @@ public class GridCacheSharedContext<K, V> {
     public void txContextReset() {
         mvccMgr.contextReset();
     }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/cc0d1f57/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheAtomicLongImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheAtomicLongImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheAtomicLongImpl.java
index 3572409..944fc5f 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheAtomicLongImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheAtomicLongImpl.java
@@ -223,7 +223,7 @@ public final class GridCacheAtomicLongImpl implements GridCacheAtomicLongEx, Ext
         this.atomicView = atomicView;
         this.name = name;
 
-        log = ctx.gridConfig().getGridLogger().getLogger(getClass());
+        log = ctx.logger(getClass());
     }
 
     /** {@inheritDoc} */
@@ -584,4 +584,4 @@ public final class GridCacheAtomicLongImpl implements GridCacheAtomicLongEx, Ext
     @Override public String toString() {
         return S.toString(GridCacheAtomicLongImpl.class, this);
     }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/cc0d1f57/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheAtomicReferenceImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheAtomicReferenceImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheAtomicReferenceImpl.java
index b8794ea..b25e111 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheAtomicReferenceImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheAtomicReferenceImpl.java
@@ -118,7 +118,7 @@ public final class GridCacheAtomicReferenceImpl<T> implements GridCacheAtomicRef
         this.atomicView = atomicView;
         this.name = name;
 
-        log = ctx.gridConfig().getGridLogger().getLogger(getClass());
+        log = ctx.logger(getClass());
     }
 
     /** {@inheritDoc} */
@@ -371,4 +371,4 @@ public final class GridCacheAtomicReferenceImpl<T> implements GridCacheAtomicRef
     @Override public String toString() {
         return S.toString(GridCacheAtomicReferenceImpl.class, this);
     }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/cc0d1f57/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheAtomicSequenceImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheAtomicSequenceImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheAtomicSequenceImpl.java
index 3e1afc0..956265b 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheAtomicSequenceImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheAtomicSequenceImpl.java
@@ -148,7 +148,7 @@ public final class GridCacheAtomicSequenceImpl implements GridCacheAtomicSequenc
         this.locVal = locVal;
         this.name = name;
 
-        log = ctx.gridConfig().getGridLogger().getLogger(getClass());
+        log = ctx.logger(getClass());
     }
 
     /** {@inheritDoc} */
@@ -587,4 +587,4 @@ public final class GridCacheAtomicSequenceImpl implements GridCacheAtomicSequenc
     @Override public String toString() {
         return S.toString(GridCacheAtomicSequenceImpl.class, this);
     }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/cc0d1f57/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheAtomicStampedImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheAtomicStampedImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheAtomicStampedImpl.java
index dff32eb..f7a82a9 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheAtomicStampedImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheAtomicStampedImpl.java
@@ -142,7 +142,7 @@ public final class GridCacheAtomicStampedImpl<T, S> implements GridCacheAtomicSt
         this.atomicView = atomicView;
         this.name = name;
 
-        log = ctx.gridConfig().getGridLogger().getLogger(getClass());
+        log = ctx.logger(getClass());
     }
 
     /** {@inheritDoc} */
@@ -411,4 +411,4 @@ public final class GridCacheAtomicStampedImpl<T, S> implements GridCacheAtomicSt
     @Override public String toString() {
         return GridToStringBuilder.toString(GridCacheAtomicStampedImpl.class, this);
     }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/cc0d1f57/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheCountDownLatchImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheCountDownLatchImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheCountDownLatchImpl.java
index cdd5f90..2667938 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheCountDownLatchImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheCountDownLatchImpl.java
@@ -129,7 +129,7 @@ public final class GridCacheCountDownLatchImpl implements GridCacheCountDownLatc
         this.latchView = latchView;
         this.ctx = ctx;
 
-        log = ctx.gridConfig().getGridLogger().getLogger(getClass());
+        log = ctx.logger(getClass());
     }
 
     /** {@inheritDoc} */
@@ -408,4 +408,4 @@ public final class GridCacheCountDownLatchImpl implements GridCacheCountDownLatc
             }
         }
     }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/cc0d1f57/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsFragmentizerManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsFragmentizerManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsFragmentizerManager.java
index 773d758..899730d 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsFragmentizerManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsFragmentizerManager.java
@@ -407,7 +407,8 @@ public class IgfsFragmentizerManager extends IgfsManager {
          * Constructor.
          */
         protected FragmentizerCoordinator() {
-            super(igfsCtx.kernalContext().gridName(), "fragmentizer-coordinator", igfsCtx.kernalContext().log());
+            super(igfsCtx.kernalContext().gridName(), "fragmentizer-coordinator",
+                igfsCtx.kernalContext().log(IgfsFragmentizerManager.class));
 
             igfsCtx.kernalContext().event().addLocalEventListener(this, EVT_NODE_LEFT, EVT_NODE_FAILED);
             igfsCtx.kernalContext().io().addMessageListener(topic, this);
@@ -719,7 +720,8 @@ public class IgfsFragmentizerManager extends IgfsManager {
          * Constructor.
          */
         protected FragmentizerWorker() {
-            super(igfsCtx.kernalContext().gridName(), "fragmentizer-worker", igfsCtx.kernalContext().log());
+            super(igfsCtx.kernalContext().gridName(), "fragmentizer-worker",
+                igfsCtx.kernalContext().log(IgfsFragmentizerManager.class));
         }
 
         /** {@inheritDoc} */
@@ -849,4 +851,4 @@ public class IgfsFragmentizerManager extends IgfsManager {
             return this == o;
         }
     }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/cc0d1f57/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsServerManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsServerManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsServerManager.java
index d692013..c12b367 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsServerManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsServerManager.java
@@ -163,7 +163,8 @@ public class IgfsServerManager extends IgfsManager {
          * Constructor.
          */
         private BindWorker() {
-            super(igfsCtx.kernalContext().gridName(), "bind-worker", igfsCtx.kernalContext().log());
+            super(igfsCtx.kernalContext().gridName(), "bind-worker",
+                igfsCtx.kernalContext().log(IgfsServerManager.class));
         }
 
         /**
@@ -210,4 +211,4 @@ public class IgfsServerManager extends IgfsManager {
             }
         }
     }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/cc0d1f57/modules/core/src/main/java/org/apache/ignite/internal/processors/job/GridJobWorker.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/job/GridJobWorker.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/job/GridJobWorker.java
index c243435..ae6d212 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/job/GridJobWorker.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/job/GridJobWorker.java
@@ -179,7 +179,7 @@ public class GridJobWorker extends GridWorker implements GridTimeoutObject {
         boolean internal,
         GridJobEventListener evtLsnr,
         GridJobHoldListener holdLsnr) {
-        super(ctx.gridName(), "grid-job-worker", ctx.log());
+        super(ctx.gridName(), "grid-job-worker", ctx.log(GridJobWorker.class));
 
         assert ctx != null;
         assert ses != null;
@@ -925,4 +925,4 @@ public class GridJobWorker extends GridWorker implements GridTimeoutObject {
     @Override public String toString() {
         return S.toString(GridJobWorker.class, this);
     }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/cc0d1f57/modules/core/src/main/java/org/apache/ignite/internal/processors/task/GridTaskWorker.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/task/GridTaskWorker.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/task/GridTaskWorker.java
index 8fb7e5f..26a41de 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/task/GridTaskWorker.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/task/GridTaskWorker.java
@@ -274,7 +274,7 @@ class GridTaskWorker<T, R> extends GridWorker implements GridTimeoutObject {
         GridTaskEventListener evtLsnr,
         @Nullable Map<GridTaskThreadContextKey, Object> thCtx,
         UUID subjId) {
-        super(ctx.config().getGridName(), "grid-task-worker", ctx.config().getGridLogger());
+        super(ctx.config().getGridName(), "grid-task-worker", ctx.log(GridTaskWorker.class));
 
         assert ses != null;
         assert fut != null;
@@ -1467,4 +1467,4 @@ class GridTaskWorker<T, R> extends GridWorker implements GridTimeoutObject {
             return S.toString(GridTaskWorker.class, this);
         }
     }
-}
\ No newline at end of file
+}


[41/55] [abbrv] ignite git commit: Fixed javadoc.

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


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

Branch: refs/heads/ignite-1171
Commit: ae1e1dc947a10335f79b087950a9c0f964092510
Parents: 3e762e2
Author: sboikov <sb...@gridgain.com>
Authored: Tue Sep 15 16:34:22 2015 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Tue Sep 15 16:34:22 2015 +0300

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


http://git-wip-us.apache.org/repos/asf/ignite/blob/ae1e1dc9/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxHandler.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxHandler.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxHandler.java
index 756672a..c2cc629 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxHandler.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxHandler.java
@@ -1049,7 +1049,7 @@ public class IgniteTxHandler {
      *
      * @param nodeId Node id that originated finish request.
      * @param req Request.
-     * @param {@code True} if transaction committed on this node.
+     * @param committed {@code True} if transaction committed on this node.
      */
     protected void sendReply(UUID nodeId, GridDhtTxFinishRequest req, boolean committed) {
         if (req.replyRequired()) {


[47/55] [abbrv] ignite git commit: IGFS: Reverted back some test changes.

Posted by ag...@apache.org.
IGFS: Reverted back some test changes.


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

Branch: refs/heads/ignite-1171
Commit: 7a71b206227452706bbee4e9b64b2294eb39750d
Parents: cb7d972
Author: vozerov-gridgain <vo...@gridgain.com>
Authored: Tue Sep 15 17:08:16 2015 +0300
Committer: vozerov-gridgain <vo...@gridgain.com>
Committed: Tue Sep 15 17:08:16 2015 +0300

----------------------------------------------------------------------
 .../internal/processors/igfs/IgfsAbstractSelfTest.java       | 8 +++++---
 .../internal/processors/igfs/IgfsDualAbstractSelfTest.java   | 6 +++---
 2 files changed, 8 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/7a71b206/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsAbstractSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsAbstractSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsAbstractSelfTest.java
index 3e63cdc..0a1e626 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsAbstractSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsAbstractSelfTest.java
@@ -2296,12 +2296,11 @@ public abstract class IgfsAbstractSelfTest extends IgfsCommonAbstractTest {
      * Create the file in the given IGFS and write provided data chunks to it.
      *
      * @param file File.
-     * @param overwrite Overwrite flag.
      * @param chunks Data chunks.
      * @throws IOException In case of IO exception.
      */
-    protected static void createFile(UniversalFileSystemAdapter uni, IgfsPath file, boolean overwrite,
-        @Nullable byte[]... chunks) throws IOException {
+    protected static void createFile(UniversalFileSystemAdapter uni, IgfsPath file, @Nullable byte[]... chunks)
+        throws IOException {
         OutputStream os = null;
 
         try {
@@ -2689,5 +2688,8 @@ public abstract class IgfsAbstractSelfTest extends IgfsCommonAbstractTest {
 
         if (igfsEx != null)
             clear(igfsEx);
+
+        // Clear the filesystem.
+        uni.format();
     }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/7a71b206/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsDualAbstractSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsDualAbstractSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsDualAbstractSelfTest.java
index 9527459..683054b 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsDualAbstractSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsDualAbstractSelfTest.java
@@ -1129,7 +1129,7 @@ public abstract class IgfsDualAbstractSelfTest extends IgfsAbstractSelfTest {
         create(igfsSecondary, paths(DIR, SUBDIR), null);
         create(igfs, null, null);
 
-        createFile(igfsSecondary, FILE, true, chunk);
+        createFile(igfsSecondary, FILE, chunk);
 
         checkFileContent(igfs, FILE, chunk);
     }
@@ -1348,7 +1348,7 @@ public abstract class IgfsDualAbstractSelfTest extends IgfsAbstractSelfTest {
 
         igfsSecondaryFileSystem.update(SUBDIR, props);
 
-        createFile(igfsSecondary, FILE, true, /*BLOCK_SIZE,*/ chunk);
+        createFile(igfsSecondary, FILE, /*BLOCK_SIZE,*/ chunk);
 
         appendFile(igfs, FILE, chunk);
 
@@ -1375,7 +1375,7 @@ public abstract class IgfsDualAbstractSelfTest extends IgfsAbstractSelfTest {
         igfsSecondaryFileSystem.update(DIR, propsDir);
         igfsSecondaryFileSystem.update(SUBDIR, propsSubDir);
 
-        createFile(igfsSecondary, FILE, true, /*BLOCK_SIZE,*/ chunk);
+        createFile(igfsSecondary, FILE, /*BLOCK_SIZE,*/ chunk);
 
         appendFile(igfs, FILE, chunk);
 


[35/55] [abbrv] ignite git commit: Hadoop: commented out external execution tests.

Posted by ag...@apache.org.
Hadoop: commented out external execution tests.


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

Branch: refs/heads/ignite-1171
Commit: 2aa292c47b87c0099d6d602dc282c231cd408f2b
Parents: 3f75c1c
Author: vozerov-gridgain <vo...@gridgain.com>
Authored: Tue Sep 15 15:42:42 2015 +0300
Committer: vozerov-gridgain <vo...@gridgain.com>
Committed: Tue Sep 15 15:42:42 2015 +0300

----------------------------------------------------------------------
 .../org/apache/ignite/testsuites/IgniteHadoopTestSuite.java    | 6 +++---
 1 file changed, 3 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/2aa292c4/modules/hadoop/src/test/java/org/apache/ignite/testsuites/IgniteHadoopTestSuite.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/testsuites/IgniteHadoopTestSuite.java b/modules/hadoop/src/test/java/org/apache/ignite/testsuites/IgniteHadoopTestSuite.java
index 0b3ff22..23f85d2 100644
--- a/modules/hadoop/src/test/java/org/apache/ignite/testsuites/IgniteHadoopTestSuite.java
+++ b/modules/hadoop/src/test/java/org/apache/ignite/testsuites/IgniteHadoopTestSuite.java
@@ -151,9 +151,9 @@ public class IgniteHadoopTestSuite extends TestSuite {
 
         suite.addTest(new TestSuite(ldr.loadClass(HadoopSortingTest.class.getName())));
 
-        suite.addTest(new TestSuite(ldr.loadClass(HadoopExternalTaskExecutionSelfTest.class.getName())));
-        suite.addTest(new TestSuite(ldr.loadClass(HadoopExternalCommunicationSelfTest.class.getName())));
-        suite.addTest(new TestSuite(ldr.loadClass(HadoopSortingExternalTest.class.getName())));
+//        suite.addTest(new TestSuite(ldr.loadClass(HadoopExternalTaskExecutionSelfTest.class.getName())));
+//        suite.addTest(new TestSuite(ldr.loadClass(HadoopExternalCommunicationSelfTest.class.getName())));
+//        suite.addTest(new TestSuite(ldr.loadClass(HadoopSortingExternalTest.class.getName())));
 
         suite.addTest(new TestSuite(ldr.loadClass(HadoopGroupingTest.class.getName())));
 


[53/55] [abbrv] ignite git commit: Basic suite tests uncommented

Posted by ag...@apache.org.
Basic suite tests uncommented


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

Branch: refs/heads/ignite-1171
Commit: 6187241ecf68a9541172c98e85e86cecdc0fe7e9
Parents: e75917d
Author: agura <ag...@gridgain.com>
Authored: Wed Sep 16 18:36:34 2015 +0300
Committer: agura <ag...@gridgain.com>
Committed: Wed Sep 16 18:36:34 2015 +0300

----------------------------------------------------------------------
 .../java/org/apache/ignite/testsuites/IgniteBasicTestSuite.java | 5 -----
 1 file changed, 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/6187241e/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteBasicTestSuite.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteBasicTestSuite.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteBasicTestSuite.java
index 390807c..1a9913f 100644
--- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteBasicTestSuite.java
+++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteBasicTestSuite.java
@@ -36,7 +36,6 @@ import org.apache.ignite.internal.processors.affinity.GridAffinityProcessorRende
 import org.apache.ignite.internal.processors.cache.GridProjectionForCachesOnDaemonNodeSelfTest;
 import org.apache.ignite.internal.processors.cache.IgniteDaemonNodeMarshallerCacheTest;
 import org.apache.ignite.internal.processors.cache.OffHeapTieredTransactionSelfTest;
-import org.apache.ignite.internal.processors.cache.distributed.CacheAffEarlySelfTest;
 import org.apache.ignite.internal.processors.closure.GridClosureProcessorSelfTest;
 import org.apache.ignite.internal.processors.continuous.GridEventConsumeSelfTest;
 import org.apache.ignite.internal.processors.continuous.GridMessageListenSelfTest;
@@ -69,9 +68,6 @@ public class IgniteBasicTestSuite extends TestSuite {
     public static TestSuite suite(Set<Class> ignoredTests) throws Exception {
         TestSuite suite = new TestSuite("Ignite Basic Test Suite");
 
-        suite.addTest(new TestSuite(CacheAffEarlySelfTest.class));
-
-/*
         suite.addTest(IgniteLangSelfTestSuite.suite());
         suite.addTest(IgniteUtilSelfTestSuite.suite(ignoredTests));
         suite.addTest(IgniteMarshallerSelfTestSuite.suite(ignoredTests));
@@ -114,7 +110,6 @@ public class IgniteBasicTestSuite extends TestSuite {
         suite.addTestSuite(OffHeapTieredTransactionSelfTest.class);
         suite.addTestSuite(IgniteSlowClientDetectionSelfTest.class);
         suite.addTestSuite(IgniteDaemonNodeMarshallerCacheTest.class);
-*/
 
         return suite;
     }


[29/55] [abbrv] ignite git commit: ignite-971 Fixed offheap to swap eviction, added failover tests with swap/offheap, added retries for tx 'check backup' rollback.

Posted by ag...@apache.org.
http://git-wip-us.apache.org/repos/asf/ignite/blob/a7490a6e/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCachePutAllFailoverSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCachePutAllFailoverSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCachePutAllFailoverSelfTest.java
index b401907..0c0d9c1 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCachePutAllFailoverSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCachePutAllFailoverSelfTest.java
@@ -60,6 +60,7 @@ import org.apache.ignite.spi.discovery.tcp.ipfinder.TcpDiscoveryIpFinder;
 import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder;
 import org.apache.ignite.spi.failover.FailoverContext;
 import org.apache.ignite.spi.failover.always.AlwaysFailoverSpi;
+import org.apache.ignite.testframework.GridTestUtils;
 import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
 
 import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL;
@@ -101,6 +102,9 @@ public class GridCachePutAllFailoverSelfTest extends GridCommonAbstractTest {
     /** Backups count. */
     private int backups;
 
+    /** */
+    private GridTestUtils.TestMemoryMode memMode = GridTestUtils.TestMemoryMode.HEAP;
+
     /** Filter to include only worker nodes. */
     private static final IgnitePredicate<ClusterNode> workerNodesFilter = new PN() {
         @SuppressWarnings("unchecked")
@@ -202,6 +206,60 @@ public class GridCachePutAllFailoverSelfTest extends GridCommonAbstractTest {
         checkPutAllFailoverColocated(false, 5, 2);
     }
 
+    /**
+     * @throws Exception If failed.
+     */
+    public void testPutAllFailoverColocatedNearEnabledTwoBackupsSwap() throws Exception {
+        memMode = GridTestUtils.TestMemoryMode.SWAP;
+
+        checkPutAllFailoverColocated(true, 5, 2);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testPutAllFailoverColocatedTwoBackupsSwap() throws Exception {
+        memMode = GridTestUtils.TestMemoryMode.SWAP;
+
+        checkPutAllFailoverColocated(false, 5, 2);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testPutAllFailoverColocatedNearEnabledTwoBackupsOffheapTiered() throws Exception {
+        memMode = GridTestUtils.TestMemoryMode.OFFHEAP_TIERED;
+
+        checkPutAllFailoverColocated(true, 5, 2);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testPutAllFailoverColocatedNearEnabledTwoBackupsOffheapTieredSwap() throws Exception {
+        memMode = GridTestUtils.TestMemoryMode.OFFHEAP_TIERED_SWAP;
+
+        checkPutAllFailoverColocated(true, 5, 2);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testPutAllFailoverColocatedNearEnabledTwoBackupsOffheapEvict() throws Exception {
+        memMode = GridTestUtils.TestMemoryMode.OFFHEAP_EVICT;
+
+        checkPutAllFailoverColocated(true, 5, 2);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testPutAllFailoverColocatedNearEnabledTwoBackupsOffheapEvictSwap() throws Exception {
+        memMode = GridTestUtils.TestMemoryMode.OFFHEAP_EVICT_SWAP;
+
+        checkPutAllFailoverColocated(true, 5, 2);
+    }
+
     /** {@inheritDoc} */
     @Override protected long getTestTimeout() {
         return super.getTestTimeout() * 5;
@@ -696,6 +754,8 @@ public class GridCachePutAllFailoverSelfTest extends GridCommonAbstractTest {
 
             cacheCfg.setWriteSynchronizationMode(FULL_SYNC);
 
+            GridTestUtils.setMemoryMode(cfg, cacheCfg, memMode, 1000, 10 * 1024);
+
             cfg.setCacheConfiguration(cacheCfg);
         }
         else

http://git-wip-us.apache.org/repos/asf/ignite/blob/a7490a6e/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheTestEntryEx.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheTestEntryEx.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheTestEntryEx.java
index d9510e6..1fef4d5 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheTestEntryEx.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheTestEntryEx.java
@@ -803,6 +803,12 @@ public class GridCacheTestEntryEx extends GridMetadataAwareAdapter implements Gr
     }
 
     /** {@inheritDoc} */
+    @Override public boolean offheapSwapEvict(byte[] vb, GridCacheVersion evictVer, GridCacheVersion obsoleteVer)
+        throws IgniteCheckedException, GridCacheEntryRemovedException {
+        return false;
+    }
+
+    /** {@inheritDoc} */
     @Override public CacheObject unswap(boolean needVal) throws IgniteCheckedException {
         return null;
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/a7490a6e/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/GridCacheAbstractNodeRestartSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/GridCacheAbstractNodeRestartSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/GridCacheAbstractNodeRestartSelfTest.java
index 757dce8..6f0565b 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/GridCacheAbstractNodeRestartSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/GridCacheAbstractNodeRestartSelfTest.java
@@ -36,11 +36,13 @@ import org.apache.ignite.IgniteException;
 import org.apache.ignite.IgniteTransactions;
 import org.apache.ignite.cache.CacheAtomicityMode;
 import org.apache.ignite.cache.CacheRebalanceMode;
+import org.apache.ignite.configuration.CacheConfiguration;
 import org.apache.ignite.configuration.IgniteConfiguration;
 import org.apache.ignite.spi.communication.tcp.TcpCommunicationSpi;
 import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi;
 import org.apache.ignite.spi.discovery.tcp.ipfinder.TcpDiscoveryIpFinder;
 import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder;
+import org.apache.ignite.testframework.GridTestUtils;
 import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
 import org.apache.ignite.transactions.Transaction;
 import org.apache.ignite.transactions.TransactionConcurrency;
@@ -110,6 +112,9 @@ public abstract class GridCacheAbstractNodeRestartSelfTest extends GridCommonAbs
     private int retries = DFLT_RETRIES;
 
     /** */
+    private GridTestUtils.TestMemoryMode memMode = GridTestUtils.TestMemoryMode.HEAP;
+
+    /** */
     private static final TcpDiscoveryIpFinder ipFinder = new TcpDiscoveryVmIpFinder(true);
 
     /** {@inheritDoc} */
@@ -129,11 +134,23 @@ public abstract class GridCacheAbstractNodeRestartSelfTest extends GridCommonAbs
 
         c.setDiscoverySpi(disco);
 
+        CacheConfiguration ccfg = cacheConfiguration();
+
+        GridTestUtils.setMemoryMode(c, ccfg, memMode, 100, 1024);
+
+        c.setCacheConfiguration(ccfg);
+
         return c;
     }
 
+    /**
+     * @return Cache configuration.
+     */
+    protected abstract CacheConfiguration cacheConfiguration();
+
     /** {@inheritDoc} */
     @Override protected void beforeTestsStarted() throws Exception {
+        // No-op.
     }
 
     /** {@inheritDoc} */
@@ -280,7 +297,7 @@ public abstract class GridCacheAbstractNodeRestartSelfTest extends GridCommonAbs
 
         long duration = 30000;
 
-        checkRestartWithTx(duration, 1, 1);
+        checkRestartWithTx(duration, 1, 1, 3);
     }
 
     /**
@@ -310,7 +327,7 @@ public abstract class GridCacheAbstractNodeRestartSelfTest extends GridCommonAbs
 
         long duration = 30000;
 
-        checkRestartWithTx(duration, 1, 1);
+        checkRestartWithTx(duration, 1, 1, 3);
     }
 
     /**
@@ -340,7 +357,7 @@ public abstract class GridCacheAbstractNodeRestartSelfTest extends GridCommonAbs
 
         long duration = 60000;
 
-        checkRestartWithTx(duration, 2, 2);
+        checkRestartWithTx(duration, 2, 2, 3);
     }
 
     /**
@@ -361,6 +378,59 @@ public abstract class GridCacheAbstractNodeRestartSelfTest extends GridCommonAbs
     /**
      * @throws Exception If failed.
      */
+    public void testRestartWithPutFourNodesOneBackupsSwap() throws Throwable {
+        restartWithPutFourNodesOneBackupsWithMemoryMode(GridTestUtils.TestMemoryMode.SWAP);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testRestartWithPutFourNodesOneBackupsOffheapTiered() throws Throwable {
+        restartWithPutFourNodesOneBackupsWithMemoryMode(GridTestUtils.TestMemoryMode.OFFHEAP_TIERED);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testRestartWithPutFourNodesOneBackupsOffheapTieredSwap() throws Throwable {
+        restartWithPutFourNodesOneBackupsWithMemoryMode(GridTestUtils.TestMemoryMode.OFFHEAP_TIERED_SWAP);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testRestartWithPutFourNodesOneBackupsOffheapEvict() throws Throwable {
+        restartWithPutFourNodesOneBackupsWithMemoryMode(GridTestUtils.TestMemoryMode.OFFHEAP_EVICT);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testRestartWithPutFourNodesOneBackupsOffheapEvictSwap() throws Throwable {
+        restartWithPutFourNodesOneBackupsWithMemoryMode(GridTestUtils.TestMemoryMode.OFFHEAP_EVICT_SWAP);
+    }
+
+    /**
+     * @param memMode Memory mode.
+     * @throws Throwable If failed.
+     */
+    private void restartWithPutFourNodesOneBackupsWithMemoryMode(GridTestUtils.TestMemoryMode memMode)
+        throws Throwable {
+        backups = 1;
+        nodeCnt = 4;
+        keyCnt = 100_000;
+        partitions = 29;
+        rebalancMode = ASYNC;
+        this.memMode = memMode;
+
+        long duration = 30_000;
+
+        checkRestartWithPut(duration, 2, 2);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
     public void testRestartWithTxFourNodesOneBackups() throws Throwable {
         backups = 1;
         nodeCnt = 4;
@@ -370,7 +440,59 @@ public abstract class GridCacheAbstractNodeRestartSelfTest extends GridCommonAbs
 
         long duration = 60000;
 
-        checkRestartWithTx(duration, 2, 2);
+        checkRestartWithTx(duration, 2, 2, 3);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testRestartWithTxFourNodesOneBackupsSwap() throws Throwable {
+        restartWithTxFourNodesOneBackupsWithMemoryMode(GridTestUtils.TestMemoryMode.SWAP);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testRestartWithTxFourNodesOneBackupsOffheapTiered() throws Throwable {
+        restartWithTxFourNodesOneBackupsWithMemoryMode(GridTestUtils.TestMemoryMode.OFFHEAP_TIERED);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testRestartWithTxFourNodesOneBackupsOffheapTieredSwap() throws Throwable {
+        restartWithTxFourNodesOneBackupsWithMemoryMode(GridTestUtils.TestMemoryMode.OFFHEAP_TIERED_SWAP);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testRestartWithTxFourNodesOneBackupsOffheapEvict() throws Throwable {
+        restartWithTxFourNodesOneBackupsWithMemoryMode(GridTestUtils.TestMemoryMode.OFFHEAP_EVICT);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testRestartWithTxFourNodesOneBackupsOffheapEvictSwap() throws Throwable {
+        restartWithTxFourNodesOneBackupsWithMemoryMode(GridTestUtils.TestMemoryMode.OFFHEAP_EVICT_SWAP);
+    }
+
+    /**
+     * @param memMode Memory mode.
+     * @throws Throwable If failed.
+     */
+    private void restartWithTxFourNodesOneBackupsWithMemoryMode(GridTestUtils.TestMemoryMode memMode) throws Throwable {
+        backups = 1;
+        nodeCnt = 4;
+        keyCnt = 100_000;
+        partitions = 29;
+        rebalancMode = ASYNC;
+        this.memMode = memMode;
+
+        long duration = 30_000;
+
+        checkRestartWithTx(duration, 2, 2, 100);
     }
 
     /**
@@ -400,7 +522,7 @@ public abstract class GridCacheAbstractNodeRestartSelfTest extends GridCommonAbs
 
         long duration = 90000;
 
-        checkRestartWithTx(duration, 3, 3);
+        checkRestartWithTx(duration, 3, 3, 3);
     }
 
     /**
@@ -430,7 +552,7 @@ public abstract class GridCacheAbstractNodeRestartSelfTest extends GridCommonAbs
 
         long duration = 90000;
 
-        checkRestartWithTx(duration, 4, 4);
+        checkRestartWithTx(duration, 4, 4, 3);
     }
 
     /**
@@ -460,7 +582,7 @@ public abstract class GridCacheAbstractNodeRestartSelfTest extends GridCommonAbs
 
         long duration = 90000;
 
-        checkRestartWithTx(duration, 5, 5);
+        checkRestartWithTx(duration, 5, 5, 3);
     }
 
     /**
@@ -499,7 +621,7 @@ public abstract class GridCacheAbstractNodeRestartSelfTest extends GridCommonAbs
      * @param restartThreads Restart threads count.
      * @throws Exception If failed.
      */
-    public void checkRestartWithPut(long duration, int putThreads, int restartThreads) throws Throwable {
+    private void checkRestartWithPut(long duration, int putThreads, int restartThreads) throws Throwable {
         final long endTime = System.currentTimeMillis() + duration;
 
         final AtomicReference<Throwable> err = new AtomicReference<>();
@@ -613,9 +735,13 @@ public abstract class GridCacheAbstractNodeRestartSelfTest extends GridCommonAbs
      * @param duration Test duration.
      * @param putThreads Put threads count.
      * @param restartThreads Restart threads count.
+     * @param txKeys Keys per transaction.
      * @throws Exception If failed.
      */
-    public void checkRestartWithTx(long duration, int putThreads, int restartThreads) throws Throwable {
+    private void checkRestartWithTx(long duration,
+        int putThreads,
+        int restartThreads,
+        final int txKeys) throws Throwable {
         if (atomicityMode() == ATOMIC)
             return;
 
@@ -634,8 +760,6 @@ public abstract class GridCacheAbstractNodeRestartSelfTest extends GridCommonAbs
 
             final CyclicBarrier barrier = new CyclicBarrier(putThreads + restartThreads);
 
-            final int txKeys = 3;
-
             for (int i = 0; i < putThreads; i++) {
                 final int gridIdx = i;
 
@@ -699,6 +823,7 @@ public abstract class GridCacheAbstractNodeRestartSelfTest extends GridCommonAbs
 
                                 if (c % logFreq == 0) {
                                     info(">>> Tx iteration finished [cnt=" + c +
+                                        ", cacheSize=" + cache.localSize() +
                                         ", keys=" + keys +
                                         ", locNodeId=" + locNodeId + ']');
                                 }
@@ -773,7 +898,7 @@ public abstract class GridCacheAbstractNodeRestartSelfTest extends GridCommonAbs
      * @param restartThreads Restart threads count.
      * @throws Exception If failed.
      */
-    public void checkRestartWithTxPutAll(long duration, int putThreads, int restartThreads) throws Throwable {
+    private void checkRestartWithTxPutAll(long duration, int putThreads, int restartThreads) throws Throwable {
         if (atomicityMode() == ATOMIC)
             return;
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/a7490a6e/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtAtomicRemoveFailureTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtAtomicRemoveFailureTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtAtomicRemoveFailureTest.java
index b7f52ba..bee94a2 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtAtomicRemoveFailureTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtAtomicRemoveFailureTest.java
@@ -17,11 +17,13 @@
 
 package org.apache.ignite.internal.processors.cache.distributed.dht;
 
+import org.apache.ignite.cache.CacheAtomicWriteOrderMode;
 import org.apache.ignite.cache.CacheAtomicityMode;
 import org.apache.ignite.cache.CacheMode;
-import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.configuration.NearCacheConfiguration;
 import org.apache.ignite.internal.processors.cache.GridCacheAbstractRemoveFailureTest;
 
+import static org.apache.ignite.cache.CacheAtomicWriteOrderMode.CLOCK;
 import static org.apache.ignite.cache.CacheAtomicityMode.ATOMIC;
 import static org.apache.ignite.cache.CacheMode.PARTITIONED;
 
@@ -40,12 +42,12 @@ public class GridCacheDhtAtomicRemoveFailureTest extends GridCacheAbstractRemove
     }
 
     /** {@inheritDoc} */
-    @Override protected CacheConfiguration cacheConfiguration(String gridName) throws Exception {
-        CacheConfiguration cfg = super.cacheConfiguration(gridName);
-
-        cfg.setNearConfiguration(null);
-        cfg.setBackups(1);
+    @Override protected CacheAtomicWriteOrderMode atomicWriteOrderMode() {
+        return CLOCK;
+    }
 
-        return cfg;
+    /** {@inheritDoc} */
+    @Override protected NearCacheConfiguration nearCache() {
+        return null;
     }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/a7490a6e/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtRemoveFailureTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtRemoveFailureTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtRemoveFailureTest.java
index 1c351ff..a13ba30 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtRemoveFailureTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtRemoveFailureTest.java
@@ -17,10 +17,12 @@
 
 package org.apache.ignite.internal.processors.cache.distributed.dht;
 
+import org.apache.ignite.cache.CacheAtomicityMode;
 import org.apache.ignite.cache.CacheMode;
-import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.configuration.NearCacheConfiguration;
 import org.apache.ignite.internal.processors.cache.GridCacheAbstractRemoveFailureTest;
 
+import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL;
 import static org.apache.ignite.cache.CacheMode.PARTITIONED;
 
 /**
@@ -33,12 +35,12 @@ public class GridCacheDhtRemoveFailureTest extends GridCacheAbstractRemoveFailur
     }
 
     /** {@inheritDoc} */
-    @Override protected CacheConfiguration cacheConfiguration(String gridName) throws Exception {
-        CacheConfiguration cfg = super.cacheConfiguration(gridName);
-
-        cfg.setNearConfiguration(null);
-        cfg.setBackups(1);
+    @Override protected CacheAtomicityMode atomicityMode() {
+        return TRANSACTIONAL;
+    }
 
-        return cfg;
+    /** {@inheritDoc} */
+    @Override protected NearCacheConfiguration nearCache() {
+        return null;
     }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/a7490a6e/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheTxNodeFailureSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheTxNodeFailureSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheTxNodeFailureSelfTest.java
index dc78003..4302486 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheTxNodeFailureSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheTxNodeFailureSelfTest.java
@@ -199,7 +199,7 @@ public class GridCacheTxNodeFailureSelfTest extends GridCommonAbstractTest {
         try {
             final Ignite ignite = ignite(0);
 
-            final IgniteCache<Object, Object> cache = ignite.cache(null);
+            final IgniteCache<Object, Object> cache = ignite.cache(null).withNoRetries();
 
             final int key = generateKey(ignite, backup);
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/a7490a6e/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/IgniteCacheCrossCacheTxFailoverTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/IgniteCacheCrossCacheTxFailoverTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/IgniteCacheCrossCacheTxFailoverTest.java
index 4c52e6f..7425e23 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/IgniteCacheCrossCacheTxFailoverTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/IgniteCacheCrossCacheTxFailoverTest.java
@@ -40,7 +40,7 @@ import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
 import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi;
 import org.apache.ignite.spi.discovery.tcp.ipfinder.TcpDiscoveryIpFinder;
 import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder;
-import org.apache.ignite.testframework.GridTestUtils;
+import org.apache.ignite.spi.swapspace.file.FileSwapSpaceSpi;
 import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
 import org.apache.ignite.transactions.Transaction;
 import org.apache.ignite.transactions.TransactionConcurrency;
@@ -51,6 +51,9 @@ import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL;
 import static org.apache.ignite.cache.CacheMode.PARTITIONED;
 import static org.apache.ignite.cache.CacheMode.REPLICATED;
 import static org.apache.ignite.cache.CacheWriteSynchronizationMode.FULL_SYNC;
+import static org.apache.ignite.testframework.GridTestUtils.TestMemoryMode;
+import static org.apache.ignite.testframework.GridTestUtils.runMultiThreadedAsync;
+import static org.apache.ignite.testframework.GridTestUtils.setMemoryMode;
 import static org.apache.ignite.transactions.TransactionConcurrency.OPTIMISTIC;
 import static org.apache.ignite.transactions.TransactionConcurrency.PESSIMISTIC;
 import static org.apache.ignite.transactions.TransactionIsolation.REPEATABLE_READ;
@@ -86,6 +89,8 @@ public class IgniteCacheCrossCacheTxFailoverTest extends GridCommonAbstractTest
         if (gridName.equals(getTestGridName(GRID_CNT - 1)))
             cfg.setClientMode(true);
 
+        cfg.setSwapSpaceSpi(new FileSwapSpaceSpi());
+
         return cfg;
     }
 
@@ -107,9 +112,13 @@ public class IgniteCacheCrossCacheTxFailoverTest extends GridCommonAbstractTest
      * @param name Cache name.
      * @param cacheMode Cache mode.
      * @param parts Number of partitions.
+     * @param memMode Memory mode.
      * @return Cache configuration.
      */
-    private CacheConfiguration cacheConfiguration(String name, CacheMode cacheMode, int parts) {
+    private CacheConfiguration cacheConfiguration(String name,
+        CacheMode cacheMode,
+        int parts,
+        TestMemoryMode memMode) {
         CacheConfiguration ccfg = new CacheConfiguration();
 
         ccfg.setName(name);
@@ -122,6 +131,8 @@ public class IgniteCacheCrossCacheTxFailoverTest extends GridCommonAbstractTest
 
         ccfg.setAffinity(new RendezvousAffinityFunction(false, parts));
 
+        setMemoryMode(null, ccfg, memMode, 100, 1024);
+
         return ccfg;
     }
 
@@ -134,49 +145,63 @@ public class IgniteCacheCrossCacheTxFailoverTest extends GridCommonAbstractTest
      * @throws Exception If failed.
      */
     public void testCrossCachePessimisticTxFailover() throws Exception {
-        crossCacheTxFailover(PARTITIONED, true, PESSIMISTIC, REPEATABLE_READ);
+        crossCacheTxFailover(PARTITIONED, true, PESSIMISTIC, REPEATABLE_READ, TestMemoryMode.HEAP);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testCrossCachePessimisticTxFailoverOffheapSwap() throws Exception {
+        crossCacheTxFailover(PARTITIONED, true, PESSIMISTIC, REPEATABLE_READ, TestMemoryMode.OFFHEAP_EVICT_SWAP);
     }
 
     /**
      * @throws Exception If failed.
      */
     public void testCrossCachePessimisticTxFailoverDifferentAffinity() throws Exception {
-        crossCacheTxFailover(PARTITIONED, false, PESSIMISTIC, REPEATABLE_READ);
+        crossCacheTxFailover(PARTITIONED, false, PESSIMISTIC, REPEATABLE_READ, TestMemoryMode.HEAP);
     }
 
     /**
      * @throws Exception If failed.
      */
     public void testCrossCacheOptimisticTxFailover() throws Exception {
-        crossCacheTxFailover(PARTITIONED, true, OPTIMISTIC, REPEATABLE_READ);
+        crossCacheTxFailover(PARTITIONED, true, OPTIMISTIC, REPEATABLE_READ, TestMemoryMode.HEAP);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testCrossCacheOptimisticTxFailoverOffheapSwap() throws Exception {
+        crossCacheTxFailover(PARTITIONED, true, OPTIMISTIC, REPEATABLE_READ, TestMemoryMode.OFFHEAP_EVICT_SWAP);
     }
 
     /**
      * @throws Exception If failed.
      */
     public void testCrossCacheOptimisticTxFailoverDifferentAffinity() throws Exception {
-        crossCacheTxFailover(PARTITIONED, false, OPTIMISTIC, REPEATABLE_READ);
+        crossCacheTxFailover(PARTITIONED, false, OPTIMISTIC, REPEATABLE_READ, TestMemoryMode.HEAP);
     }
 
     /**
      * @throws Exception If failed.
      */
     public void testCrossCachePessimisticTxFailoverReplicated() throws Exception {
-        crossCacheTxFailover(REPLICATED, true, PESSIMISTIC, REPEATABLE_READ);
+        crossCacheTxFailover(REPLICATED, true, PESSIMISTIC, REPEATABLE_READ, TestMemoryMode.HEAP);
     }
 
     /**
      * @throws Exception If failed.
      */
     public void testCrossCacheOptimisticTxFailoverReplicated() throws Exception {
-        crossCacheTxFailover(REPLICATED, true, OPTIMISTIC, REPEATABLE_READ);
+        crossCacheTxFailover(REPLICATED, true, OPTIMISTIC, REPEATABLE_READ, TestMemoryMode.HEAP);
     }
 
     /**
      * @throws Exception If failed.
      */
     public void testCrossCachePessimisticTxFailoverDifferentAffinityReplicated() throws Exception {
-        crossCacheTxFailover(PARTITIONED, false, PESSIMISTIC, REPEATABLE_READ);
+        crossCacheTxFailover(PARTITIONED, false, PESSIMISTIC, REPEATABLE_READ, TestMemoryMode.HEAP);
     }
 
     /**
@@ -184,23 +209,25 @@ public class IgniteCacheCrossCacheTxFailoverTest extends GridCommonAbstractTest
      * @param sameAff If {@code false} uses different number of partitions for caches.
      * @param concurrency Transaction concurrency.
      * @param isolation Transaction isolation.
+     * @param memMode Memory mode.
      * @throws Exception If failed.
      */
     private void crossCacheTxFailover(CacheMode cacheMode,
         boolean sameAff,
         final TransactionConcurrency concurrency,
-        final TransactionIsolation isolation) throws Exception {
+        final TransactionIsolation isolation,
+        TestMemoryMode memMode) throws Exception {
         IgniteKernal ignite0 = (IgniteKernal)ignite(0);
 
         final AtomicBoolean stop = new AtomicBoolean();
 
         try {
-            ignite0.createCache(cacheConfiguration(CACHE1, cacheMode, 256));
-            ignite0.createCache(cacheConfiguration(CACHE2, cacheMode, sameAff ? 256 : 128));
+            ignite0.createCache(cacheConfiguration(CACHE1, cacheMode, 256, memMode));
+            ignite0.createCache(cacheConfiguration(CACHE2, cacheMode, sameAff ? 256 : 128, memMode));
 
             final AtomicInteger threadIdx = new AtomicInteger();
 
-            IgniteInternalFuture<?> fut = GridTestUtils.runMultiThreadedAsync(new Callable<Void>() {
+            IgniteInternalFuture<?> fut = runMultiThreadedAsync(new Callable<Void>() {
                 @Override public Void call() throws Exception {
                     int idx = threadIdx.getAndIncrement();
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/a7490a6e/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/IgniteCachePutRetryAbstractSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/IgniteCachePutRetryAbstractSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/IgniteCachePutRetryAbstractSelfTest.java
index 8a72bb9..943caeb 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/IgniteCachePutRetryAbstractSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/IgniteCachePutRetryAbstractSelfTest.java
@@ -24,35 +24,54 @@ import java.util.Map;
 import java.util.Set;
 import java.util.concurrent.Callable;
 import java.util.concurrent.atomic.AtomicBoolean;
+import javax.cache.Cache;
+import javax.cache.configuration.Factory;
+import javax.cache.integration.CacheLoaderException;
+import javax.cache.integration.CacheWriterException;
 import javax.cache.processor.EntryProcessorResult;
 import javax.cache.processor.MutableEntry;
 import org.apache.ignite.IgniteCache;
 import org.apache.ignite.cache.CacheAtomicWriteOrderMode;
+import org.apache.ignite.cache.CacheAtomicityMode;
 import org.apache.ignite.cache.CacheEntryProcessor;
 import org.apache.ignite.cache.CachePartialUpdateException;
+import org.apache.ignite.cache.store.CacheStore;
+import org.apache.ignite.cache.store.CacheStoreAdapter;
 import org.apache.ignite.configuration.AtomicConfiguration;
 import org.apache.ignite.configuration.CacheConfiguration;
 import org.apache.ignite.configuration.IgniteConfiguration;
 import org.apache.ignite.internal.IgniteInternalFuture;
 import org.apache.ignite.internal.IgniteKernal;
 import org.apache.ignite.internal.cluster.ClusterTopologyCheckedException;
-import org.apache.ignite.internal.processors.cache.GridCacheAbstractSelfTest;
 import org.apache.ignite.internal.util.typedef.X;
 import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.spi.communication.tcp.TcpCommunicationSpi;
+import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi;
+import org.apache.ignite.spi.discovery.tcp.ipfinder.TcpDiscoveryIpFinder;
+import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder;
+import org.apache.ignite.spi.swapspace.file.FileSwapSpaceSpi;
 import org.apache.ignite.testframework.GridTestUtils;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
 
 import static org.apache.ignite.cache.CacheAtomicWriteOrderMode.CLOCK;
 import static org.apache.ignite.cache.CacheAtomicityMode.ATOMIC;
 import static org.apache.ignite.cache.CacheRebalanceMode.SYNC;
+import static org.apache.ignite.cache.CacheWriteSynchronizationMode.FULL_SYNC;
+import static org.apache.ignite.testframework.GridTestUtils.TestMemoryMode;
+import static org.apache.ignite.testframework.GridTestUtils.runAsync;
 
 /**
  *
  */
-public abstract class IgniteCachePutRetryAbstractSelfTest extends GridCacheAbstractSelfTest {
-    /** {@inheritDoc} */
-    @Override protected int gridCount() {
-        return 4;
-    }
+public abstract class IgniteCachePutRetryAbstractSelfTest extends GridCommonAbstractTest {
+    /** IP finder. */
+    private static final TcpDiscoveryIpFinder IP_FINDER = new TcpDiscoveryVmIpFinder(true);
+
+    /** */
+    private static final long DURATION = 60_000;
+
+    /** */
+    protected static final int GRID_CNT = 4;
 
     /**
      * @return Keys count for the test.
@@ -61,14 +80,29 @@ public abstract class IgniteCachePutRetryAbstractSelfTest extends GridCacheAbstr
         return 10_000;
     }
 
-    /** {@inheritDoc} */
-    @Override protected CacheConfiguration cacheConfiguration(String gridName) throws Exception {
-        CacheConfiguration cfg = super.cacheConfiguration(gridName);
+    /**
+     * @param memMode Memory mode.
+     * @param store If {@code true} adds cache store.
+     * @return Cache configuration.
+     * @throws Exception If failed.
+     */
+    @SuppressWarnings("unchecked")
+    protected CacheConfiguration cacheConfiguration(TestMemoryMode memMode, boolean store) throws Exception {
+        CacheConfiguration cfg = new CacheConfiguration();
 
+        cfg.setAtomicityMode(atomicityMode());
+        cfg.setWriteSynchronizationMode(FULL_SYNC);
         cfg.setAtomicWriteOrderMode(writeOrderMode());
         cfg.setBackups(1);
         cfg.setRebalanceMode(SYNC);
 
+        if (store) {
+            cfg.setCacheStoreFactory(new TestStoreFactory());
+            cfg.setWriteThrough(true);
+        }
+
+        GridTestUtils.setMemoryMode(null, cfg, memMode, 100, 1024);
+
         return cfg;
     }
 
@@ -76,15 +110,47 @@ public abstract class IgniteCachePutRetryAbstractSelfTest extends GridCacheAbstr
     @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
         IgniteConfiguration cfg = super.getConfiguration(gridName);
 
+        ((TcpDiscoverySpi)cfg.getDiscoverySpi()).setIpFinder(IP_FINDER);
+
+        ((TcpCommunicationSpi)cfg.getCommunicationSpi()).setSharedMemoryPort(-1);
+
         AtomicConfiguration acfg = new AtomicConfiguration();
 
         acfg.setBackups(1);
 
         cfg.setAtomicConfiguration(acfg);
 
+        cfg.setSwapSpaceSpi(new FileSwapSpaceSpi());
+
         return cfg;
     }
 
+    /** {@inheritDoc} */
+    @Override protected void beforeTestsStarted() throws Exception {
+        super.beforeTestsStarted();
+
+        startGridsMultiThreaded(GRID_CNT);
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTestsStopped() throws Exception {
+        super.afterTestsStopped();
+
+        stopAllGrids();
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTest() throws Exception {
+        super.afterTest();
+
+        ignite(0).destroyCache(null);
+    }
+
+    /**
+     * @return Cache atomicity mode.
+     */
+    protected abstract CacheAtomicityMode atomicityMode();
+
     /**
      * @return Write order mode.
      */
@@ -96,47 +162,79 @@ public abstract class IgniteCachePutRetryAbstractSelfTest extends GridCacheAbstr
      * @throws Exception If failed.
      */
     public void testPut() throws Exception {
-        checkRetry(Test.PUT);
+        checkRetry(Test.PUT, TestMemoryMode.HEAP, false);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testPutStoreEnabled() throws Exception {
+        checkRetry(Test.PUT, TestMemoryMode.HEAP, true);
     }
 
     /**
      * @throws Exception If failed.
      */
     public void testPutAll() throws Exception {
-        checkRetry(Test.PUT_ALL);
+        checkRetry(Test.PUT_ALL, TestMemoryMode.HEAP, false);
     }
 
     /**
      * @throws Exception If failed.
      */
     public void testPutAsync() throws Exception {
-        checkRetry(Test.PUT_ASYNC);
+        checkRetry(Test.PUT_ASYNC, TestMemoryMode.HEAP, false);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testPutAsyncStoreEnabled() throws Exception {
+        checkRetry(Test.PUT_ASYNC, TestMemoryMode.HEAP, true);
     }
 
     /**
      * @throws Exception If failed.
      */
     public void testInvoke() throws Exception {
-        checkRetry(Test.INVOKE);
+        checkRetry(Test.INVOKE, TestMemoryMode.HEAP, false);
     }
 
     /**
      * @throws Exception If failed.
      */
     public void testInvokeAll() throws Exception {
-        checkRetry(Test.INVOKE_ALL);
+        checkRetry(Test.INVOKE_ALL, TestMemoryMode.HEAP, false);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testInvokeAllOffheapSwap() throws Exception {
+        checkRetry(Test.INVOKE_ALL, TestMemoryMode.OFFHEAP_EVICT_SWAP, false);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testInvokeAllOffheapTiered() throws Exception {
+        checkRetry(Test.INVOKE_ALL, TestMemoryMode.OFFHEAP_TIERED, false);
     }
 
     /**
      * @param test Test type.
+     * @param memMode Memory mode.
+     * @param store If {@code true} uses cache with store.
      * @throws Exception If failed.
      */
-    private void checkRetry(Test test) throws Exception {
+    private void checkRetry(Test test, TestMemoryMode memMode, boolean store) throws Exception {
+        ignite(0).createCache(cacheConfiguration(memMode, store));
+
         final AtomicBoolean finished = new AtomicBoolean();
 
         int keysCnt = keysCount();
 
-        IgniteInternalFuture<Object> fut = GridTestUtils.runAsync(new Callable<Object>() {
+        IgniteInternalFuture<Object> fut = runAsync(new Callable<Object>() {
             @Override public Object call() throws Exception {
                 while (!finished.get()) {
                     stopGrid(3);
@@ -155,10 +253,12 @@ public abstract class IgniteCachePutRetryAbstractSelfTest extends GridCacheAbstr
         int iter = 0;
 
         try {
-            if (atomicityMode() == ATOMIC)
-                assertEquals(writeOrderMode(), cache.getConfiguration(CacheConfiguration.class).getAtomicWriteOrderMode());
+            if (atomicityMode() == ATOMIC) {
+                assertEquals(writeOrderMode(),
+                    cache.getConfiguration(CacheConfiguration.class).getAtomicWriteOrderMode());
+            }
 
-            long stopTime = System.currentTimeMillis() + 60_000;
+            long stopTime = System.currentTimeMillis() + DURATION;
 
             switch (test) {
                 case PUT: {
@@ -288,7 +388,7 @@ public abstract class IgniteCachePutRetryAbstractSelfTest extends GridCacheAbstr
         for (int i = 0; i < keysCnt; i++)
             assertEquals((Integer)iter, cache.get(i));
 
-        for (int i = 0; i < gridCount(); i++) {
+        for (int i = 0; i < GRID_CNT; i++) {
             IgniteKernal ignite = (IgniteKernal)grid(i);
 
             Collection<?> futs = ignite.context().cache().context().mvcc().atomicFutures();
@@ -316,9 +416,11 @@ public abstract class IgniteCachePutRetryAbstractSelfTest extends GridCacheAbstr
      * @throws Exception If failed.
      */
     private void checkFailsWithNoRetries(boolean async) throws Exception {
+        ignite(0).createCache(cacheConfiguration(TestMemoryMode.HEAP, false));
+
         final AtomicBoolean finished = new AtomicBoolean();
 
-        IgniteInternalFuture<Object> fut = GridTestUtils.runAsync(new Callable<Object>() {
+        IgniteInternalFuture<Object> fut = runAsync(new Callable<Object>() {
             @Override public Object call() throws Exception {
                 while (!finished.get()) {
                     stopGrid(3);
@@ -428,4 +530,26 @@ public abstract class IgniteCachePutRetryAbstractSelfTest extends GridCacheAbstr
             return old == null ? 0 : old;
         }
     }
+
+    /**
+     *
+     */
+    private static class TestStoreFactory implements Factory<CacheStore> {
+        /** {@inheritDoc} */
+        @Override public CacheStore create() {
+            return new CacheStoreAdapter() {
+                @Override public Object load(Object key) throws CacheLoaderException {
+                    return null;
+                }
+
+                @Override public void write(Cache.Entry entry) throws CacheWriterException {
+                    // No-op.
+                }
+
+                @Override public void delete(Object key) throws CacheWriterException {
+                    // No-op.
+                }
+            };
+        }
+    }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/a7490a6e/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/IgniteCachePutRetryAtomicSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/IgniteCachePutRetryAtomicSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/IgniteCachePutRetryAtomicSelfTest.java
index 9d84609..3d7c7d7 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/IgniteCachePutRetryAtomicSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/IgniteCachePutRetryAtomicSelfTest.java
@@ -47,6 +47,8 @@ public class IgniteCachePutRetryAtomicSelfTest extends IgniteCachePutRetryAbstra
      * @throws Exception If failed.
      */
     public void testPutInsideTransaction() throws Exception {
+        ignite(0).createCache(cacheConfiguration(GridTestUtils.TestMemoryMode.HEAP, false));
+
         CacheConfiguration<Integer, Integer> ccfg = new CacheConfiguration<>();
 
         ccfg.setName("tx-cache");

http://git-wip-us.apache.org/repos/asf/ignite/blob/a7490a6e/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 7c66efc..f61faf2 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
@@ -31,12 +31,13 @@ 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.configuration.NearCacheConfiguration;
 import org.apache.ignite.internal.IgniteInternalFuture;
 import org.apache.ignite.internal.util.typedef.internal.U;
-import org.apache.ignite.testframework.GridTestUtils;
 
 import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL;
+import static org.apache.ignite.testframework.GridTestUtils.TestMemoryMode;
+import static org.apache.ignite.testframework.GridTestUtils.runAsync;
+import static org.apache.ignite.testframework.GridTestUtils.runMultiThreadedAsync;
 
 /**
  *
@@ -50,11 +51,6 @@ public class IgniteCachePutRetryTransactionalSelfTest extends IgniteCachePutRetr
         return TRANSACTIONAL;
     }
 
-    /** {@inheritDoc} */
-    @Override protected NearCacheConfiguration nearConfiguration() {
-        return null;
-    }
-
     /**
      * @throws Exception If failed.
      */
@@ -63,7 +59,7 @@ public class IgniteCachePutRetryTransactionalSelfTest extends IgniteCachePutRetr
 
         IgniteAtomicLong atomic = ignite(0).atomicLong("TestAtomic", 0, true);
 
-        IgniteInternalFuture<Object> fut = GridTestUtils.runAsync(new Callable<Object>() {
+        IgniteInternalFuture<Object> fut = runAsync(new Callable<Object>() {
             @Override public Object call() throws Exception {
                 while (!finished.get()) {
                     stopGrid(3);
@@ -92,15 +88,42 @@ public class IgniteCachePutRetryTransactionalSelfTest extends IgniteCachePutRetr
         }
     }
 
-    /** {@inheritDoc} */
-    @SuppressWarnings("unchecked")
+    /**
+     * @throws Exception If failed.
+     */
     public void testExplicitTransactionRetries() throws Exception {
+        explicitTransactionRetries(TestMemoryMode.HEAP, false);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testExplicitTransactionRetriesStoreEnabled() throws Exception {
+        explicitTransactionRetries(TestMemoryMode.HEAP, true);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testExplicitTransactionRetriesOffheapSwap() throws Exception {
+        explicitTransactionRetries(TestMemoryMode.OFFHEAP_EVICT_SWAP, false);
+    }
+
+    /**
+     * @param memMode Memory mode.
+     * @param store If {@code true} uses cache with store.
+     * @throws Exception If failed.
+     */
+    @SuppressWarnings("unchecked")
+    public void explicitTransactionRetries(TestMemoryMode memMode, boolean store) throws Exception {
+        ignite(0).createCache(cacheConfiguration(memMode, store));
+
         final AtomicInteger idx = new AtomicInteger();
         int threads = 8;
 
         final AtomicReferenceArray<Exception> err = new AtomicReferenceArray<>(threads);
 
-        IgniteInternalFuture<Long> fut = GridTestUtils.runMultiThreadedAsync(new Callable<Object>() {
+        IgniteInternalFuture<Long> fut = runMultiThreadedAsync(new Callable<Object>() {
             @Override public Object call() throws Exception {
                 int th = idx.getAndIncrement();
                 int base = th * FACTOR;
@@ -115,8 +138,7 @@ public class IgniteCachePutRetryTransactionalSelfTest extends IgniteCachePutRetr
                         if (i > 0 && i % 500 == 0)
                             info("Done: " + i);
                     }
-                }
-                catch (Exception e) {
+                } catch (Exception e) {
                     err.set(th, e);
                 }
 
@@ -142,7 +164,7 @@ public class IgniteCachePutRetryTransactionalSelfTest extends IgniteCachePutRetr
         }
 
         // Verify contents of the cache.
-        for (int g = 0; g < gridCount(); g++) {
+        for (int g = 0; g < GRID_CNT; g++) {
             IgniteCache<Object, Object> cache = ignite(g).cache(null);
 
             for (int th = 0; th < threads; th++) {

http://git-wip-us.apache.org/repos/asf/ignite/blob/a7490a6e/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridCacheAtomicInvalidPartitionHandlingSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridCacheAtomicInvalidPartitionHandlingSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridCacheAtomicInvalidPartitionHandlingSelfTest.java
index c598e97..3913957 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridCacheAtomicInvalidPartitionHandlingSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridCacheAtomicInvalidPartitionHandlingSelfTest.java
@@ -43,6 +43,7 @@ import org.apache.ignite.internal.IgniteKernal;
 import org.apache.ignite.internal.managers.communication.GridIoMessage;
 import org.apache.ignite.internal.processors.cache.GridCacheAdapter;
 import org.apache.ignite.internal.processors.cache.GridCacheEntryEx;
+import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtInvalidPartitionException;
 import org.apache.ignite.internal.processors.cache.version.GridCacheVersion;
 import org.apache.ignite.internal.util.lang.GridAbsPredicate;
 import org.apache.ignite.internal.util.typedef.internal.CU;
@@ -65,6 +66,7 @@ import static org.apache.ignite.cache.CacheRebalanceMode.SYNC;
 import static org.apache.ignite.cache.CacheWriteSynchronizationMode.FULL_ASYNC;
 import static org.apache.ignite.cache.CacheWriteSynchronizationMode.FULL_SYNC;
 import static org.apache.ignite.cache.CacheWriteSynchronizationMode.PRIMARY_SYNC;
+import static org.apache.ignite.testframework.GridTestUtils.TestMemoryMode;
 
 /**
  * Test GridDhtInvalidPartitionException handling in ATOMIC cache during restarts.
@@ -83,19 +85,26 @@ public class GridCacheAtomicInvalidPartitionHandlingSelfTest extends GridCommonA
     /** Write sync. */
     private CacheWriteSynchronizationMode writeSync;
 
+    /** Memory mode. */
+    private TestMemoryMode memMode;
+
     /** {@inheritDoc} */
     @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
         IgniteConfiguration cfg = super.getConfiguration(gridName);
 
         cfg.setDiscoverySpi(new TcpDiscoverySpi().setIpFinder(IP_FINDER).setForceServerMode(true));
 
-        cfg.setCacheConfiguration(cacheConfiguration());
+        CacheConfiguration ccfg = cacheConfiguration();
+
+        cfg.setCacheConfiguration(ccfg);
 
         cfg.setCommunicationSpi(new DelayCommunicationSpi());
 
         if (testClientNode() && getTestGridName(0).equals(gridName))
             cfg.setClientMode(true);
 
+        GridTestUtils.setMemoryMode(cfg, ccfg, memMode, 100, 1024);
+
         return cfg;
     }
 
@@ -137,53 +146,99 @@ public class GridCacheAtomicInvalidPartitionHandlingSelfTest extends GridCommonA
      * @throws Exception If failed.
      */
     public void testClockFullSync() throws Exception {
-        checkRestarts(CLOCK, FULL_SYNC);
+        checkRestarts(CLOCK, FULL_SYNC, TestMemoryMode.HEAP);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testClockFullSyncSwap() throws Exception {
+        checkRestarts(CLOCK, FULL_SYNC, TestMemoryMode.SWAP);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testClockFullSyncOffheapTiered() throws Exception {
+        checkRestarts(CLOCK, FULL_SYNC, TestMemoryMode.OFFHEAP_TIERED);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testClockFullSyncOffheapSwap() throws Exception {
+        checkRestarts(CLOCK, FULL_SYNC, TestMemoryMode.OFFHEAP_EVICT_SWAP);
     }
 
     /**
      * @throws Exception If failed.
      */
     public void testClockPrimarySync() throws Exception {
-        checkRestarts(CLOCK, PRIMARY_SYNC);
+        checkRestarts(CLOCK, PRIMARY_SYNC, TestMemoryMode.HEAP);
     }
 
     /**
      * @throws Exception If failed.
      */
     public void testClockFullAsync() throws Exception {
-        checkRestarts(CLOCK, FULL_ASYNC);
+        checkRestarts(CLOCK, FULL_ASYNC, TestMemoryMode.HEAP);
     }
 
     /**
      * @throws Exception If failed.
      */
     public void testPrimaryFullSync() throws Exception {
-        checkRestarts(PRIMARY, FULL_SYNC);
+        checkRestarts(PRIMARY, FULL_SYNC, TestMemoryMode.HEAP);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testPrimaryFullSyncSwap() throws Exception {
+        checkRestarts(PRIMARY, FULL_SYNC, TestMemoryMode.SWAP);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testPrimaryFullSyncOffheapTiered() throws Exception {
+        checkRestarts(PRIMARY, FULL_SYNC, TestMemoryMode.OFFHEAP_TIERED);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testPrimaryFullSyncOffheapSwap() throws Exception {
+        checkRestarts(PRIMARY, FULL_SYNC, TestMemoryMode.OFFHEAP_EVICT_SWAP);
     }
 
     /**
      * @throws Exception If failed.
      */
     public void testPrimaryPrimarySync() throws Exception {
-        checkRestarts(PRIMARY, PRIMARY_SYNC);
+        checkRestarts(PRIMARY, PRIMARY_SYNC, TestMemoryMode.HEAP);
     }
 
     /**
      * @throws Exception If failed.
      */
     public void testPrimaryFullAsync() throws Exception {
-        checkRestarts(PRIMARY, FULL_ASYNC);
+        checkRestarts(PRIMARY, FULL_ASYNC, TestMemoryMode.HEAP);
     }
 
     /**
      * @param writeOrder Write order to check.
      * @param writeSync Write synchronization mode to check.
+     * @param memMode Memory mode.
      * @throws Exception If failed.
      */
-    private void checkRestarts(CacheAtomicWriteOrderMode writeOrder, CacheWriteSynchronizationMode writeSync)
+    private void checkRestarts(CacheAtomicWriteOrderMode writeOrder,
+        CacheWriteSynchronizationMode writeSync,
+        TestMemoryMode memMode)
         throws Exception {
         this.writeOrder = writeOrder;
         this.writeSync = writeSync;
+        this.memMode = memMode;
 
         final int gridCnt = 6;
 
@@ -227,16 +282,16 @@ public class GridCacheAtomicInvalidPartitionHandlingSelfTest extends GridCommonA
                         for (int i = 0; i < gridCnt; i++) {
                             ClusterNode locNode = grid(i).localNode();
 
-                            GridCacheAdapter<Object, Object> c = ((IgniteKernal)grid(i)).internalCache();
+                            IgniteCache<Object, Object> cache = grid(i).cache(null);
 
-                            GridCacheEntryEx entry = c.peekEx(key);
+                            Object val = cache.localPeek(key);
 
                             if (affNodes.contains(locNode)) {
-                                if (entry == null)
+                                if (val == null)
                                     return false;
                             }
                             else
-                                assertNull(entry);
+                                assertNull(val);
                         }
 
                         it.remove();
@@ -323,7 +378,20 @@ public class GridCacheAtomicInvalidPartitionHandlingSelfTest extends GridCommonA
 
                     GridCacheAdapter<Object, Object> c = ((IgniteKernal)grid(i)).internalCache();
 
-                    GridCacheEntryEx entry = c.peekEx(k);
+                    GridCacheEntryEx entry = null;
+
+                    if (memMode == TestMemoryMode.HEAP)
+                        entry = c.peekEx(k);
+                    else {
+                        try {
+                            entry = c.entryEx(k);
+
+                            entry.unswap();
+                        }
+                        catch (GridDhtInvalidPartitionException e) {
+                            // Skip key.
+                        }
+                    }
 
                     for (int r = 0; r < 10; r++) {
                         try {
@@ -383,7 +451,7 @@ public class GridCacheAtomicInvalidPartitionHandlingSelfTest extends GridCommonA
      */
     private static class DelayCommunicationSpi extends TcpCommunicationSpi {
         /** {@inheritDoc} */
-        @Override public void sendMessage(ClusterNode node, Message msg, IgniteInClosure<IgniteException> ackClosure)
+        @Override public void sendMessage(ClusterNode node, Message msg, IgniteInClosure<IgniteException> ackC)
             throws IgniteSpiException {
             try {
                 if (delayMessage((GridIoMessage)msg))
@@ -393,7 +461,7 @@ public class GridCacheAtomicInvalidPartitionHandlingSelfTest extends GridCommonA
                 throw new IgniteSpiException(e);
             }
 
-            super.sendMessage(node, msg, ackClosure);
+            super.sendMessage(node, msg, ackC);
         }
 
         /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/a7490a6e/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridCacheAtomicPrimaryWriteOrderRemoveFailureTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridCacheAtomicPrimaryWriteOrderRemoveFailureTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridCacheAtomicPrimaryWriteOrderRemoveFailureTest.java
index 844bde3..2fd8a82 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridCacheAtomicPrimaryWriteOrderRemoveFailureTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridCacheAtomicPrimaryWriteOrderRemoveFailureTest.java
@@ -17,9 +17,9 @@
 
 package org.apache.ignite.internal.processors.cache.distributed.dht.atomic;
 
+import org.apache.ignite.cache.CacheAtomicWriteOrderMode;
 import org.apache.ignite.cache.CacheAtomicityMode;
 import org.apache.ignite.cache.CacheMode;
-import org.apache.ignite.configuration.CacheConfiguration;
 import org.apache.ignite.configuration.NearCacheConfiguration;
 import org.apache.ignite.internal.processors.cache.GridCacheAbstractRemoveFailureTest;
 
@@ -42,17 +42,12 @@ public class GridCacheAtomicPrimaryWriteOrderRemoveFailureTest extends GridCache
     }
 
     /** {@inheritDoc} */
-    @Override protected NearCacheConfiguration nearConfiguration() {
-        return null;
+    @Override protected CacheAtomicWriteOrderMode atomicWriteOrderMode() {
+        return PRIMARY;
     }
 
     /** {@inheritDoc} */
-    @Override protected CacheConfiguration cacheConfiguration(String gridName) throws Exception {
-        CacheConfiguration cfg = super.cacheConfiguration(gridName);
-
-        cfg.setBackups(1);
-        cfg.setAtomicWriteOrderMode(PRIMARY);
-
-        return cfg;
+    @Override protected NearCacheConfiguration nearCache() {
+        return null;
     }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/a7490a6e/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridCacheAtomicRemoveFailureTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridCacheAtomicRemoveFailureTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridCacheAtomicRemoveFailureTest.java
index 9966e18..a21cd35 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridCacheAtomicRemoveFailureTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridCacheAtomicRemoveFailureTest.java
@@ -17,9 +17,9 @@
 
 package org.apache.ignite.internal.processors.cache.distributed.dht.atomic;
 
+import org.apache.ignite.cache.CacheAtomicWriteOrderMode;
 import org.apache.ignite.cache.CacheAtomicityMode;
 import org.apache.ignite.cache.CacheMode;
-import org.apache.ignite.configuration.CacheConfiguration;
 import org.apache.ignite.configuration.NearCacheConfiguration;
 import org.apache.ignite.internal.processors.cache.GridCacheAbstractRemoveFailureTest;
 
@@ -42,17 +42,12 @@ public class GridCacheAtomicRemoveFailureTest extends GridCacheAbstractRemoveFai
     }
 
     /** {@inheritDoc} */
-    @Override protected NearCacheConfiguration nearConfiguration() {
-        return null;
+    @Override protected CacheAtomicWriteOrderMode atomicWriteOrderMode() {
+        return CLOCK;
     }
 
     /** {@inheritDoc} */
-    @Override protected CacheConfiguration cacheConfiguration(String gridName) throws Exception {
-        CacheConfiguration cfg = super.cacheConfiguration(gridName);
-
-        cfg.setBackups(1);
-        cfg.setAtomicWriteOrderMode(CLOCK);
-
-        return cfg;
+    @Override protected NearCacheConfiguration nearCache() {
+        return null;
     }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/a7490a6e/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheAtomicNearRemoveFailureTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheAtomicNearRemoveFailureTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheAtomicNearRemoveFailureTest.java
index a6fc2b9..d9c6e01 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheAtomicNearRemoveFailureTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheAtomicNearRemoveFailureTest.java
@@ -17,9 +17,9 @@
 
 package org.apache.ignite.internal.processors.cache.distributed.near;
 
+import org.apache.ignite.cache.CacheAtomicWriteOrderMode;
 import org.apache.ignite.cache.CacheAtomicityMode;
 import org.apache.ignite.cache.CacheMode;
-import org.apache.ignite.configuration.CacheConfiguration;
 import org.apache.ignite.configuration.NearCacheConfiguration;
 import org.apache.ignite.internal.processors.cache.GridCacheAbstractRemoveFailureTest;
 
@@ -41,13 +41,12 @@ public class GridCacheAtomicNearRemoveFailureTest extends GridCacheAbstractRemov
     }
 
     /** {@inheritDoc} */
-    @Override protected CacheConfiguration cacheConfiguration(String gridName) throws Exception {
-        CacheConfiguration cfg = super.cacheConfiguration(gridName);
-
-        cfg.setNearConfiguration(new NearCacheConfiguration());
-        cfg.setBackups(1);
-        cfg.setAtomicWriteOrderMode(CLOCK);
+    @Override protected CacheAtomicWriteOrderMode atomicWriteOrderMode() {
+        return CLOCK;
+    }
 
-        return cfg;
+    /** {@inheritDoc} */
+    @Override protected NearCacheConfiguration nearCache() {
+        return new NearCacheConfiguration();
     }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/a7490a6e/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheAtomicPrimaryWriteOrderNearRemoveFailureTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheAtomicPrimaryWriteOrderNearRemoveFailureTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheAtomicPrimaryWriteOrderNearRemoveFailureTest.java
index 6362bc8..f6e7e2f 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheAtomicPrimaryWriteOrderNearRemoveFailureTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheAtomicPrimaryWriteOrderNearRemoveFailureTest.java
@@ -17,9 +17,9 @@
 
 package org.apache.ignite.internal.processors.cache.distributed.near;
 
+import org.apache.ignite.cache.CacheAtomicWriteOrderMode;
 import org.apache.ignite.cache.CacheAtomicityMode;
 import org.apache.ignite.cache.CacheMode;
-import org.apache.ignite.configuration.CacheConfiguration;
 import org.apache.ignite.configuration.NearCacheConfiguration;
 import org.apache.ignite.internal.processors.cache.GridCacheAbstractRemoveFailureTest;
 
@@ -41,13 +41,12 @@ public class GridCacheAtomicPrimaryWriteOrderNearRemoveFailureTest extends GridC
     }
 
     /** {@inheritDoc} */
-    @Override protected CacheConfiguration cacheConfiguration(String gridName) throws Exception {
-        CacheConfiguration cfg = super.cacheConfiguration(gridName);
-
-        cfg.setNearConfiguration(new NearCacheConfiguration());
-        cfg.setBackups(1);
-        cfg.setAtomicWriteOrderMode(PRIMARY);
+    @Override protected CacheAtomicWriteOrderMode atomicWriteOrderMode() {
+        return PRIMARY;
+    }
 
-        return cfg;
+    /** {@inheritDoc} */
+    @Override protected NearCacheConfiguration nearCache() {
+        return new NearCacheConfiguration();
     }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/a7490a6e/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheNearRemoveFailureTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheNearRemoveFailureTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheNearRemoveFailureTest.java
index 1cf3da1..a7e8eb9 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheNearRemoveFailureTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheNearRemoveFailureTest.java
@@ -17,11 +17,12 @@
 
 package org.apache.ignite.internal.processors.cache.distributed.near;
 
+import org.apache.ignite.cache.CacheAtomicityMode;
 import org.apache.ignite.cache.CacheMode;
-import org.apache.ignite.configuration.CacheConfiguration;
 import org.apache.ignite.configuration.NearCacheConfiguration;
 import org.apache.ignite.internal.processors.cache.GridCacheAbstractRemoveFailureTest;
 
+import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL;
 import static org.apache.ignite.cache.CacheMode.PARTITIONED;
 
 /**
@@ -34,12 +35,12 @@ public class GridCacheNearRemoveFailureTest extends GridCacheAbstractRemoveFailu
     }
 
     /** {@inheritDoc} */
-    @Override protected CacheConfiguration cacheConfiguration(String gridName) throws Exception {
-        CacheConfiguration cfg = super.cacheConfiguration(gridName);
-
-        cfg.setNearConfiguration(new NearCacheConfiguration());
-        cfg.setBackups(1);
+    @Override protected CacheAtomicityMode atomicityMode() {
+        return TRANSACTIONAL;
+    }
 
-        return cfg;
+    /** {@inheritDoc} */
+    @Override protected NearCacheConfiguration nearCache() {
+        return new NearCacheConfiguration();
     }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/a7490a6e/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedNodeRestartTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedNodeRestartTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedNodeRestartTest.java
index 265c185..684d6e4 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedNodeRestartTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedNodeRestartTest.java
@@ -36,6 +36,11 @@ public class GridCachePartitionedNodeRestartTest extends GridCacheAbstractNodeRe
 
         c.getTransactionConfiguration().setDefaultTxConcurrency(PESSIMISTIC);
 
+        return c;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected CacheConfiguration cacheConfiguration() {
         CacheConfiguration cc = defaultCacheConfiguration();
 
         cc.setName(CACHE_NAME);
@@ -49,9 +54,7 @@ public class GridCachePartitionedNodeRestartTest extends GridCacheAbstractNodeRe
         cc.setAffinity(new RendezvousAffinityFunction(false, partitions));
         cc.setBackups(backups);
 
-        c.setCacheConfiguration(cc);
-
-        return c;
+        return cc;
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/a7490a6e/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedOptimisticTxNodeRestartTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedOptimisticTxNodeRestartTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedOptimisticTxNodeRestartTest.java
index 0e81828..a458aa7 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedOptimisticTxNodeRestartTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedOptimisticTxNodeRestartTest.java
@@ -44,6 +44,11 @@ public class GridCachePartitionedOptimisticTxNodeRestartTest extends GridCacheAb
 
         c.getTransactionConfiguration().setDefaultTxConcurrency(OPTIMISTIC);
 
+        return c;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected CacheConfiguration cacheConfiguration() {
         CacheConfiguration cc = defaultCacheConfiguration();
 
         cc.setName(CACHE_NAME);
@@ -57,9 +62,7 @@ public class GridCachePartitionedOptimisticTxNodeRestartTest extends GridCacheAb
 
         cc.setNearConfiguration(nearEnabled() ? new NearCacheConfiguration() : null);
 
-        c.setCacheConfiguration(cc);
-
-        return c;
+        return cc;
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/a7490a6e/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/replicated/GridCacheReplicatedNodeRestartSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/replicated/GridCacheReplicatedNodeRestartSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/replicated/GridCacheReplicatedNodeRestartSelfTest.java
index 5be228d..7e3e7e5 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/replicated/GridCacheReplicatedNodeRestartSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/replicated/GridCacheReplicatedNodeRestartSelfTest.java
@@ -30,9 +30,7 @@ import static org.apache.ignite.cache.CacheWriteSynchronizationMode.FULL_SYNC;
  */
 public class GridCacheReplicatedNodeRestartSelfTest extends GridCacheAbstractNodeRestartSelfTest {
     /** {@inheritDoc} */
-    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
-        IgniteConfiguration c = super.getConfiguration(gridName);
-
+    @Override protected CacheConfiguration cacheConfiguration() {
         CacheConfiguration cc = defaultCacheConfiguration();
 
         cc.setNearConfiguration(null);
@@ -51,9 +49,7 @@ public class GridCacheReplicatedNodeRestartSelfTest extends GridCacheAbstractNod
 
         cc.setRebalanceBatchSize(20);
 
-        c.setCacheConfiguration(cc);
-
-        return c;
+        return cc;
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/a7490a6e/modules/core/src/test/java/org/apache/ignite/internal/util/offheap/GridOffHeapMapAbstractSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/util/offheap/GridOffHeapMapAbstractSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/util/offheap/GridOffHeapMapAbstractSelfTest.java
index 4bda4f4..4cda0c2 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/util/offheap/GridOffHeapMapAbstractSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/util/offheap/GridOffHeapMapAbstractSelfTest.java
@@ -42,7 +42,7 @@ public abstract class GridOffHeapMapAbstractSelfTest extends GridCommonAbstractT
     private static final Random RAND = new Random();
 
     /** Unsafe map. */
-    private GridOffHeapMap<String> map;
+    private GridOffHeapMap map;
 
     /** */
     protected float load = 0.75f;
@@ -86,7 +86,7 @@ public abstract class GridOffHeapMapAbstractSelfTest extends GridCommonAbstractT
     /**
      * @return New map.
      */
-    protected abstract <K> GridOffHeapMap<K> newMap();
+    protected abstract GridOffHeapMap newMap();
 
     /**
      *
@@ -551,6 +551,10 @@ public abstract class GridOffHeapMapAbstractSelfTest extends GridCommonAbstractT
 
                 evictCnt.incrementAndGet();
             }
+
+            @Override public boolean removeEvicted() {
+                return true;
+            }
         };
 
         map = newMap();
@@ -587,6 +591,10 @@ public abstract class GridOffHeapMapAbstractSelfTest extends GridCommonAbstractT
             @Override public void onEvict(int part, int hash, byte[] k, byte[] v) {
                 evictCnt.incrementAndGet();
             }
+
+            @Override public boolean removeEvicted() {
+                return true;
+            }
         };
 
         map = newMap();
@@ -622,6 +630,10 @@ public abstract class GridOffHeapMapAbstractSelfTest extends GridCommonAbstractT
             @Override public void onEvict(int part, int hash, byte[] k, byte[] v) {
                 evictCnt.incrementAndGet();
             }
+
+            @Override public boolean removeEvicted() {
+                return true;
+            }
         };
 
         map = newMap();

http://git-wip-us.apache.org/repos/asf/ignite/blob/a7490a6e/modules/core/src/test/java/org/apache/ignite/internal/util/offheap/GridOffHeapMapPerformanceAbstractTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/util/offheap/GridOffHeapMapPerformanceAbstractTest.java b/modules/core/src/test/java/org/apache/ignite/internal/util/offheap/GridOffHeapMapPerformanceAbstractTest.java
index d1a1b20..f7388e8 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/util/offheap/GridOffHeapMapPerformanceAbstractTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/util/offheap/GridOffHeapMapPerformanceAbstractTest.java
@@ -39,7 +39,7 @@ public abstract class GridOffHeapMapPerformanceAbstractTest extends GridCommonAb
         new HashMap<>(LOAD_CNT);
 
     /** Unsafe map. */
-    private GridOffHeapMap<String> map;
+    private GridOffHeapMap map;
 
     /** */
     protected float load = 0.75f;
@@ -91,7 +91,7 @@ public abstract class GridOffHeapMapPerformanceAbstractTest extends GridCommonAb
     /**
      * @return New map.
      */
-    protected abstract <K> GridOffHeapMap<K> newMap();
+    protected abstract GridOffHeapMap newMap();
 
     /**
      * @param key Key.

http://git-wip-us.apache.org/repos/asf/ignite/blob/a7490a6e/modules/core/src/test/java/org/apache/ignite/internal/util/offheap/GridOffHeapPartitionedMapAbstractSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/util/offheap/GridOffHeapPartitionedMapAbstractSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/util/offheap/GridOffHeapPartitionedMapAbstractSelfTest.java
index 03fcd4a..c40c10f 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/util/offheap/GridOffHeapPartitionedMapAbstractSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/util/offheap/GridOffHeapPartitionedMapAbstractSelfTest.java
@@ -882,6 +882,10 @@ public abstract class GridOffHeapPartitionedMapAbstractSelfTest extends GridComm
 
                 evictCnt.incrementAndGet();
             }
+
+            @Override public boolean removeEvicted() {
+                return true;
+            }
         };
 
         map = newMap();
@@ -921,6 +925,10 @@ public abstract class GridOffHeapPartitionedMapAbstractSelfTest extends GridComm
                 @Override public void onEvict(int part, int hash, byte[] k, byte[] v) {
                     evictCnt.incrementAndGet();
                 }
+
+                @Override public boolean removeEvicted() {
+                    return true;
+                }
             };
 
             map = newMap();
@@ -957,6 +965,10 @@ public abstract class GridOffHeapPartitionedMapAbstractSelfTest extends GridComm
             @Override public void onEvict(int part, int hash, byte[] k, byte[] v) {
                 evictCnt.incrementAndGet();
             }
+
+            @Override public boolean removeEvicted() {
+                return true;
+            }
         };
 
         map = newMap();
@@ -1009,6 +1021,10 @@ public abstract class GridOffHeapPartitionedMapAbstractSelfTest extends GridComm
 
                 evicted.set(key);
             }
+
+            @Override public boolean removeEvicted() {
+                return true;
+            }
         };
 
         map = newMap();
@@ -1072,6 +1088,10 @@ public abstract class GridOffHeapPartitionedMapAbstractSelfTest extends GridComm
 
                 evicted.set(key);
             }
+
+            @Override public boolean removeEvicted() {
+                return true;
+            }
         };
 
         map = newMap();

http://git-wip-us.apache.org/repos/asf/ignite/blob/a7490a6e/modules/core/src/test/java/org/apache/ignite/internal/util/offheap/unsafe/GridUnsafeMapPerformanceTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/util/offheap/unsafe/GridUnsafeMapPerformanceTest.java b/modules/core/src/test/java/org/apache/ignite/internal/util/offheap/unsafe/GridUnsafeMapPerformanceTest.java
index e758246..58ad494 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/util/offheap/unsafe/GridUnsafeMapPerformanceTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/util/offheap/unsafe/GridUnsafeMapPerformanceTest.java
@@ -26,7 +26,7 @@ import org.apache.ignite.internal.util.offheap.GridOffHeapMapPerformanceAbstract
  */
 public class GridUnsafeMapPerformanceTest extends GridOffHeapMapPerformanceAbstractTest {
     /** {@inheritDoc} */
-    @Override protected <K> GridOffHeapMap<K> newMap() {
+    @Override protected GridOffHeapMap newMap() {
         return GridOffHeapMapFactory.unsafeMap(concurrency, load, initCap, mem, lruStripes, evictClo);
     }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/a7490a6e/modules/core/src/test/java/org/apache/ignite/internal/util/offheap/unsafe/GridUnsafeMapSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/util/offheap/unsafe/GridUnsafeMapSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/util/offheap/unsafe/GridUnsafeMapSelfTest.java
index 43fdb34..0e36f3d 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/util/offheap/unsafe/GridUnsafeMapSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/util/offheap/unsafe/GridUnsafeMapSelfTest.java
@@ -26,7 +26,7 @@ import org.apache.ignite.internal.util.offheap.GridOffHeapMapFactory;
  */
 public class GridUnsafeMapSelfTest extends GridOffHeapMapAbstractSelfTest {
     /** {@inheritDoc} */
-    @Override protected <K> GridOffHeapMap<K> newMap() {
+    @Override protected GridOffHeapMap newMap() {
         return GridOffHeapMapFactory.unsafeMap(concurrency, load, initCap, mem, lruStripes, evictLsnr);
     }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/a7490a6e/modules/core/src/test/java/org/apache/ignite/loadtests/offheap/GridOffHeapMapPerformanceAbstractTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/loadtests/offheap/GridOffHeapMapPerformanceAbstractTest.java b/modules/core/src/test/java/org/apache/ignite/loadtests/offheap/GridOffHeapMapPerformanceAbstractTest.java
index ed37306..4064482 100644
--- a/modules/core/src/test/java/org/apache/ignite/loadtests/offheap/GridOffHeapMapPerformanceAbstractTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/loadtests/offheap/GridOffHeapMapPerformanceAbstractTest.java
@@ -42,7 +42,7 @@ public abstract class GridOffHeapMapPerformanceAbstractTest extends GridCommonAb
         new HashMap<>(LOAD_CNT);
 
     /** Unsafe map. */
-    private GridOffHeapMap<String> map;
+    private GridOffHeapMap map;
 
     /** */
     protected float load = 0.75f;
@@ -94,7 +94,7 @@ public abstract class GridOffHeapMapPerformanceAbstractTest extends GridCommonAb
     /**
      * @return New map.
      */
-    protected abstract <K> GridOffHeapMap<K> newMap();
+    protected abstract GridOffHeapMap newMap();
 
     /**
      * @param key Key.

http://git-wip-us.apache.org/repos/asf/ignite/blob/a7490a6e/modules/core/src/test/java/org/apache/ignite/loadtests/offheap/unsafe/GridUnsafeMapPerformanceTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/loadtests/offheap/unsafe/GridUnsafeMapPerformanceTest.java b/modules/core/src/test/java/org/apache/ignite/loadtests/offheap/unsafe/GridUnsafeMapPerformanceTest.java
index 1486a9c..af691b4 100644
--- a/modules/core/src/test/java/org/apache/ignite/loadtests/offheap/unsafe/GridUnsafeMapPerformanceTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/loadtests/offheap/unsafe/GridUnsafeMapPerformanceTest.java
@@ -26,7 +26,7 @@ import org.apache.ignite.internal.util.offheap.GridOffHeapMapPerformanceAbstract
  */
 public class GridUnsafeMapPerformanceTest extends GridOffHeapMapPerformanceAbstractTest {
     /** {@inheritDoc} */
-    @Override protected <K> GridOffHeapMap<K> newMap() {
+    @Override protected GridOffHeapMap newMap() {
         return GridOffHeapMapFactory.unsafeMap(concurrency, load, initCap, mem, lruStripes, evictClo);
     }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/a7490a6e/modules/core/src/test/java/org/apache/ignite/testframework/GridTestUtils.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testframework/GridTestUtils.java b/modules/core/src/test/java/org/apache/ignite/testframework/GridTestUtils.java
index b585a8f..edf7c52 100644
--- a/modules/core/src/test/java/org/apache/ignite/testframework/GridTestUtils.java
+++ b/modules/core/src/test/java/org/apache/ignite/testframework/GridTestUtils.java
@@ -59,7 +59,11 @@ import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.IgniteException;
 import org.apache.ignite.IgniteLogger;
 import org.apache.ignite.Ignition;
+import org.apache.ignite.cache.CacheMemoryMode;
+import org.apache.ignite.cache.eviction.lru.LruEvictionPolicy;
 import org.apache.ignite.cluster.ClusterNode;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
 import org.apache.ignite.internal.IgniteEx;
 import org.apache.ignite.internal.IgniteInternalFuture;
 import org.apache.ignite.internal.IgniteInterruptedCheckedException;
@@ -83,6 +87,7 @@ import org.apache.ignite.internal.util.typedef.X;
 import org.apache.ignite.internal.util.typedef.internal.LT;
 import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.lang.IgnitePredicate;
+import org.apache.ignite.spi.swapspace.file.FileSwapSpaceSpi;
 import org.apache.ignite.ssl.SslContextFactory;
 import org.apache.ignite.testframework.config.GridTestProperties;
 import org.jetbrains.annotations.NotNull;
@@ -1582,4 +1587,116 @@ public final class GridTestUtils {
 
         suite.addTestSuite(test);
     }
+
+    /**
+     * Sets cache configuration parameters according to test memory mode.
+     *
+     * @param cfg Ignite configuration.
+     * @param ccfg Cache configuration.
+     * @param testMode Test memory mode.
+     * @param maxHeapCnt Maximum number of entries in heap (used if test mode involves eviction from heap).
+     * @param maxOffheapSize Maximum offheap memory size (used if test mode involves eviction from offheap to swap).
+     */
+    public static void setMemoryMode(IgniteConfiguration cfg, CacheConfiguration ccfg,
+        TestMemoryMode testMode,
+        int maxHeapCnt,
+        long maxOffheapSize) {
+        assert testMode != null;
+        assert ccfg != null;
+
+        CacheMemoryMode memMode;
+        boolean swap = false;
+        boolean evictionPlc = false;
+        long offheapMaxMem = -1L;
+
+        switch (testMode) {
+            case HEAP: {
+                memMode = CacheMemoryMode.ONHEAP_TIERED;
+                swap = false;
+
+                break;
+            }
+
+            case SWAP: {
+                memMode = CacheMemoryMode.ONHEAP_TIERED;
+                evictionPlc = true;
+                swap = true;
+
+                break;
+            }
+
+            case OFFHEAP_TIERED: {
+                memMode = CacheMemoryMode.OFFHEAP_TIERED;
+                offheapMaxMem = 0;
+
+                break;
+            }
+
+            case OFFHEAP_TIERED_SWAP: {
+                assert maxOffheapSize > 0 : maxOffheapSize;
+
+                memMode = CacheMemoryMode.OFFHEAP_TIERED;
+                offheapMaxMem = maxOffheapSize;
+                swap = true;
+
+                break;
+            }
+
+            case OFFHEAP_EVICT: {
+                memMode = CacheMemoryMode.ONHEAP_TIERED;
+                evictionPlc = true;
+                offheapMaxMem = 0;
+
+                break;
+            }
+
+            case OFFHEAP_EVICT_SWAP: {
+                assert maxOffheapSize > 0 : maxOffheapSize;
+
+                memMode = CacheMemoryMode.ONHEAP_TIERED;
+                swap = true;
+                evictionPlc = true;
+                offheapMaxMem = maxOffheapSize;
+
+                break;
+            }
+
+            default:
+                throw new IllegalArgumentException("Invalid mode: " + testMode);
+        }
+
+        ccfg.setMemoryMode(memMode);
+        ccfg.setSwapEnabled(swap);
+
+        if (swap && cfg != null)
+            cfg.setSwapSpaceSpi(new FileSwapSpaceSpi());
+
+        if (evictionPlc) {
+            LruEvictionPolicy plc = new LruEvictionPolicy();
+
+            plc.setMaxSize(maxHeapCnt);
+
+            ccfg.setEvictionPolicy(plc);
+        }
+
+        ccfg.setOffHeapMaxMemory(offheapMaxMem);
+    }
+
+    /**
+     *
+     */
+    public enum TestMemoryMode {
+        /** Heap only. */
+        HEAP,
+        /** Evict from heap to swap with eviction policy. */
+        SWAP,
+        /** Always evict to offheap, no swap. */
+        OFFHEAP_TIERED,
+        /** Always evict to offheap + evict from offheap to swap when max offheap memory limit is reached. */
+        OFFHEAP_TIERED_SWAP,
+        /** Evict to offheap with eviction policy, no swap. */
+        OFFHEAP_EVICT,
+        /** Evict to offheap with eviction policy + evict from offheap to swap when max offheap memory limit is reached. */
+        OFFHEAP_EVICT_SWAP,
+    }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/a7490a6e/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheFailoverTestSuite.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheFailoverTestSuite.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheFailoverTestSuite.java
index eaeb7b3..abc8765 100644
--- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheFailoverTestSuite.java
+++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheFailoverTestSuite.java
@@ -91,12 +91,6 @@ public class IgniteCacheFailoverTestSuite extends TestSuite {
         suite.addTestSuite(IgniteCacheTxNearDisabledPutGetRestartTest.class);
         suite.addTestSuite(IgniteCacheTxNearDisabledFairAffinityPutGetRestartTest.class);
 
-        suite.addTestSuite(IgniteCachePutRetryAtomicSelfTest.class);
-        suite.addTestSuite(IgniteCachePutRetryAtomicPrimaryWriteOrderSelfTest.class);
-        suite.addTestSuite(IgniteCachePutRetryTransactionalSelfTest.class);
-
-        suite.addTestSuite(IgniteCacheSslStartStopSelfTest.class);
-
         suite.addTestSuite(IgniteCacheSizeFailoverTest.class);
 
         suite.addTestSuite(IgniteCacheTopologySafeGetSelfTest.class);


[27/55] [abbrv] ignite git commit: removed link to IgniteCache.withKeepPortable() method in documentation

Posted by ag...@apache.org.
removed link to IgniteCache.withKeepPortable() method in documentation


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

Branch: refs/heads/ignite-1171
Commit: eb7d2b0b1fe88ce82af43f3bcb7420f2bfcfb638
Parents: dc44a2a
Author: Denis Magda <dm...@gridgain.com>
Authored: Tue Sep 15 12:25:41 2015 +0300
Committer: Denis Magda <dm...@gridgain.com>
Committed: Tue Sep 15 12:25:41 2015 +0300

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


http://git-wip-us.apache.org/repos/asf/ignite/blob/eb7d2b0b/modules/core/src/main/java/org/apache/ignite/internal/portable/api/IgnitePortables.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/api/IgnitePortables.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/api/IgnitePortables.java
index 56f3768..6a534e9 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/portable/api/IgnitePortables.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/portable/api/IgnitePortables.java
@@ -51,7 +51,7 @@ import org.jetbrains.annotations.Nullable;
  * (assuming that class definitions are present in the classpath).
  * <p>
  * To work with the portable format directly, user should create a special cache projection
- * using {@link IgniteCache#withKeepPortable()} method and then retrieve individual fields as needed:
+ * using IgniteCache.withKeepPortable() method and then retrieve individual fields as needed:
  * <pre name=code class=java>
  * IgniteCache&lt;PortableObject, PortableObject&gt; prj = cache.withKeepPortable();
  *


[34/55] [abbrv] ignite git commit: IGNITE-1376: Fixed test clear routine.

Posted by ag...@apache.org.
IGNITE-1376: Fixed test clear routine.


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

Branch: refs/heads/ignite-1171
Commit: 3f75c1c4427dc32d990560d0b8dc80a191f5f666
Parents: 367d805
Author: vozerov-gridgain <vo...@gridgain.com>
Authored: Tue Sep 15 15:31:42 2015 +0300
Committer: vozerov-gridgain <vo...@gridgain.com>
Committed: Tue Sep 15 15:31:42 2015 +0300

----------------------------------------------------------------------
 .../processors/igfs/IgfsFileWorkerBatch.java    | 15 ++++-
 .../processors/igfs/IgfsAbstractSelfTest.java   | 60 +++++++++-----------
 .../igfs/IgfsDualAbstractSelfTest.java          | 10 +++-
 3 files changed, 49 insertions(+), 36 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/3f75c1c4/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsFileWorkerBatch.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsFileWorkerBatch.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsFileWorkerBatch.java
index 6bea187..130846b 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsFileWorkerBatch.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsFileWorkerBatch.java
@@ -49,6 +49,9 @@ public abstract class IgfsFileWorkerBatch implements Runnable {
     /** Output stream to the file. */
     private final OutputStream out;
 
+    /** Cancel flag. */
+    private volatile boolean cancelled;
+
     /** Finishing flag. */
     private volatile boolean finishing;
 
@@ -108,6 +111,13 @@ public abstract class IgfsFileWorkerBatch implements Runnable {
     }
 
     /**
+     * @return {@code True} if batch write was terminated abruptly due to explicit cancellation.
+     */
+    boolean cancelled() {
+        return cancelled;
+    }
+
+    /**
      * Process the batch.
      */
     @SuppressWarnings("unchecked")
@@ -124,8 +134,11 @@ public abstract class IgfsFileWorkerBatch implements Runnable {
 
                         break;
                     }
-                    else if (data == CANCEL_MARKER)
+                    else if (data == CANCEL_MARKER) {
+                        cancelled = true;
+
                         throw new IgniteCheckedException("Write to file was cancelled due to node stop.");
+                    }
                     else if (data != null) {
                         try {
                             out.write(data);

http://git-wip-us.apache.org/repos/asf/ignite/blob/3f75c1c4/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsAbstractSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsAbstractSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsAbstractSelfTest.java
index 7513bb8..957363f 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsAbstractSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsAbstractSelfTest.java
@@ -188,8 +188,8 @@ public abstract class IgfsAbstractSelfTest extends IgfsCommonAbstractTest {
     /**
      * Constructor.
      *
-     * @param mode
-     * @param memoryMode
+     * @param mode IGFS mode.
+     * @param memoryMode Memory mode.
      */
     protected IgfsAbstractSelfTest(IgfsMode mode, CacheMemoryMode memoryMode) {
         assert mode != null && mode != PROXY;
@@ -201,9 +201,10 @@ public abstract class IgfsAbstractSelfTest extends IgfsCommonAbstractTest {
     }
 
     /**
+     * Data chunk.
      *
-     * @param length
-     * @return
+     * @param length Length.
+     * @return Data chunk.
      */
     static byte[] createChunk(int length) {
         byte[] chunk = new byte[length];
@@ -255,6 +256,7 @@ public abstract class IgfsAbstractSelfTest extends IgfsCommonAbstractTest {
      * @return Started grid instance.
      * @throws Exception If failed.
      */
+    @SuppressWarnings("unchecked")
     protected Ignite startGridWithIgfs(String gridName, String igfsName, IgfsMode mode,
         @Nullable IgfsSecondaryFileSystem secondaryFs, @Nullable IgfsIpcEndpointConfiguration restCfg) throws Exception {
         FileSystemConfiguration igfsCfg = new FileSystemConfiguration();
@@ -309,9 +311,10 @@ public abstract class IgfsAbstractSelfTest extends IgfsCommonAbstractTest {
     }
 
     /**
+     * Prepare cache configuration.
      *
-     * @param dataCacheCfg
-     * @param metaCacheCfg
+     * @param dataCacheCfg Data cache configuration.
+     * @param metaCacheCfg Meta cache configuration.
      */
     protected void prepareCacheConfigurations(CacheConfiguration dataCacheCfg, CacheConfiguration metaCacheCfg) {
         // Noop
@@ -782,9 +785,9 @@ public abstract class IgfsAbstractSelfTest extends IgfsCommonAbstractTest {
             boolean ok = igfs.delete(SUBDIR, false);
 
             assertFalse(ok);
-        } catch (IgfsDirectoryNotEmptyException idnee) {
-            // ok, expected
-            U.debug("Expected: " + idnee);
+        }
+        catch (IgfsDirectoryNotEmptyException ignore) {
+            // No-op, expected.
         }
 
         checkExist(igfs, igfsSecondary, SUBDIR, SUBSUBDIR, FILE);
@@ -841,7 +844,7 @@ public abstract class IgfsAbstractSelfTest extends IgfsCommonAbstractTest {
 
     /**
      * Ensure that formatting is not propagated to the secondary file system.
-     * 
+     *
      * TODO: IGNITE-586.
      *
      * @throws Exception If failed.
@@ -850,7 +853,7 @@ public abstract class IgfsAbstractSelfTest extends IgfsCommonAbstractTest {
     public void testFormat() throws Exception {
         // Test works too long and fails.
         fail("https://issues.apache.org/jira/browse/IGNITE-586");
-        
+
         IgniteKernal grid = (IgniteKernal)G.ignite("grid");
         GridCacheAdapter cache = grid.internalCache("dataCache");
 
@@ -2249,6 +2252,7 @@ public abstract class IgfsAbstractSelfTest extends IgfsCommonAbstractTest {
         }
     }
 
+    @SuppressWarnings("EmptyTryBlock")
     public void create(UniversalFileSystemAdapter uni, @Nullable IgfsPath[] dirs, @Nullable IgfsPath[] files) throws Exception {
         if (dirs != null) {
             for (IgfsPath dir : dirs)
@@ -2257,8 +2261,8 @@ public abstract class IgfsAbstractSelfTest extends IgfsCommonAbstractTest {
 
         if (files != null) {
             for (IgfsPath file : files)
-                try (OutputStream os = uni.openOutputStream(file.toString(), false)) {
-                    // noop
+                try (OutputStream ignore = uni.openOutputStream(file.toString(), false)) {
+                    // No-op
                 }
         }
     }
@@ -2291,12 +2295,12 @@ public abstract class IgfsAbstractSelfTest extends IgfsCommonAbstractTest {
     /**
      * Create the file in the given IGFS and write provided data chunks to it.
      *
+     * @param uni File system adapter.
      * @param file File.
-     * @param overwrite Overwrite flag.
      * @param chunks Data chunks.
      * @throws IOException In case of IO exception.
      */
-    protected static void createFile(UniversalFileSystemAdapter uni, IgfsPath file, boolean overwrite, @Nullable byte[]... chunks)
+    protected static void createFile(UniversalFileSystemAdapter uni, IgfsPath file, @Nullable byte[]... chunks)
         throws IOException {
         OutputStream os = null;
 
@@ -2658,7 +2662,14 @@ public abstract class IgfsAbstractSelfTest extends IgfsCommonAbstractTest {
 
         for (Map.Entry<IgfsPath, IgfsFileWorkerBatch> entry : workerMap.entrySet()) {
             entry.getValue().cancel();
-            entry.getValue().await();
+
+            try {
+                entry.getValue().await();
+            }
+            catch (IgniteCheckedException e) {
+                if (!entry.getValue().cancelled())
+                    throw e;
+            }
         }
 
         // Clear igfs.
@@ -2675,21 +2686,6 @@ public abstract class IgfsAbstractSelfTest extends IgfsCommonAbstractTest {
     public static void clear(UniversalFileSystemAdapter uni) throws Exception {
         IgfsEx igfsEx = uni.getAdapter(IgfsEx.class);
 
-        if (igfsEx != null) {
-            Field workerMapFld = IgfsImpl.class.getDeclaredField("workerMap");
-
-            workerMapFld.setAccessible(true);
-
-            // Wait for all workers to finish.
-            Map<IgfsPath, IgfsFileWorkerBatch> workerMap = (Map<IgfsPath, IgfsFileWorkerBatch>)workerMapFld.get(igfs);
-
-            for (Map.Entry<IgfsPath, IgfsFileWorkerBatch> entry : workerMap.entrySet()) {
-                entry.getValue().cancel();
-                entry.getValue().await();
-            }
-        }
-
-        // Clear the filesystem:
-        uni.format();
+        clear(igfsEx);
     }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/3f75c1c4/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsDualAbstractSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsDualAbstractSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsDualAbstractSelfTest.java
index 0141173..683054b 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsDualAbstractSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsDualAbstractSelfTest.java
@@ -1129,7 +1129,7 @@ public abstract class IgfsDualAbstractSelfTest extends IgfsAbstractSelfTest {
         create(igfsSecondary, paths(DIR, SUBDIR), null);
         create(igfs, null, null);
 
-        createFile(igfsSecondary, FILE, true, chunk);
+        createFile(igfsSecondary, FILE, chunk);
 
         checkFileContent(igfs, FILE, chunk);
     }
@@ -1139,6 +1139,7 @@ public abstract class IgfsDualAbstractSelfTest extends IgfsAbstractSelfTest {
      *
      * @throws Exception If failed.
      */
+    @SuppressWarnings({"ResultOfMethodCallIgnored", "ThrowableResultOfMethodCallIgnored"})
     public void testOpenNoPrefetch() throws Exception {
         create(igfsSecondary, paths(DIR, SUBDIR), paths(FILE));
 
@@ -1216,6 +1217,7 @@ public abstract class IgfsDualAbstractSelfTest extends IgfsAbstractSelfTest {
      *
      * @throws Exception If failed.
      */
+    @SuppressWarnings("ResultOfMethodCallIgnored")
     public void testOpenPrefetch() throws Exception {
         create(igfsSecondary, paths(DIR, SUBDIR), paths(FILE));
 
@@ -1251,6 +1253,8 @@ public abstract class IgfsDualAbstractSelfTest extends IgfsAbstractSelfTest {
 
         IgfsFileInfo info = meta.info(meta.fileId(FILE));
 
+        assert info != null;
+
         IgfsBlockKey key = new IgfsBlockKey(info.id(), info.affinityKey(), info.evictExclude(), 2);
 
         IgniteCache<IgfsBlockKey, byte[]> dataCache = igfs.context().kernalContext().cache().jcache(
@@ -1344,7 +1348,7 @@ public abstract class IgfsDualAbstractSelfTest extends IgfsAbstractSelfTest {
 
         igfsSecondaryFileSystem.update(SUBDIR, props);
 
-        createFile(igfsSecondary, FILE, true, /*BLOCK_SIZE,*/ chunk);
+        createFile(igfsSecondary, FILE, /*BLOCK_SIZE,*/ chunk);
 
         appendFile(igfs, FILE, chunk);
 
@@ -1371,7 +1375,7 @@ public abstract class IgfsDualAbstractSelfTest extends IgfsAbstractSelfTest {
         igfsSecondaryFileSystem.update(DIR, propsDir);
         igfsSecondaryFileSystem.update(SUBDIR, propsSubDir);
 
-        createFile(igfsSecondary, FILE, true, /*BLOCK_SIZE,*/ chunk);
+        createFile(igfsSecondary, FILE, /*BLOCK_SIZE,*/ chunk);
 
         appendFile(igfs, FILE, chunk);
 


[49/55] [abbrv] 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/f8577cab
Tree: http://git-wip-us.apache.org/repos/asf/ignite/tree/f8577cab
Diff: http://git-wip-us.apache.org/repos/asf/ignite/diff/f8577cab

Branch: refs/heads/ignite-1171
Commit: f8577cab29faa87dd337295e3dbac68f0b94ac63
Parents: 8e25204 7a71b20
Author: Yakov Zhdanov <yz...@gridgain.com>
Authored: Tue Sep 15 17:09:37 2015 +0300
Committer: Yakov Zhdanov <yz...@gridgain.com>
Committed: Tue Sep 15 17:09:37 2015 +0300

----------------------------------------------------------------------
 .../ignite/internal/processors/igfs/IgfsAbstractSelfTest.java   | 5 ++++-
 1 file changed, 4 insertions(+), 1 deletion(-)
----------------------------------------------------------------------



[16/55] [abbrv] ignite git commit: IGNITE-1197 - Fixed unswap iterator

Posted by ag...@apache.org.
IGNITE-1197 - Fixed unswap iterator


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

Branch: refs/heads/ignite-1171
Commit: cb9b76620167cc8b71b333615e6406dd98dc6d7a
Parents: 866fb41
Author: Yakov Zhdanov <yz...@gridgain.com>
Authored: Mon Sep 14 16:34:28 2015 -0700
Committer: Alexey Goncharuk <al...@gmail.com>
Committed: Mon Sep 14 16:36:04 2015 -0700

----------------------------------------------------------------------
 .../distributed/dht/GridDhtLocalPartition.java  | 63 ++++++++++++++------
 1 file changed, 44 insertions(+), 19 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/cb9b7662/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtLocalPartition.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtLocalPartition.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtLocalPartition.java
index 215a1b5..a58451f 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtLocalPartition.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtLocalPartition.java
@@ -21,6 +21,7 @@ import java.util.Collection;
 import java.util.HashMap;
 import java.util.Iterator;
 import java.util.Map;
+import java.util.NoSuchElementException;
 import java.util.Set;
 import java.util.concurrent.ConcurrentMap;
 import java.util.concurrent.CopyOnWriteArrayList;
@@ -707,38 +708,62 @@ public class GridDhtLocalPartition implements Comparable<GridDhtLocalPartition>,
 
         return new Iterator<GridDhtCacheEntry>() {
             /** */
-            GridDhtCacheEntry lastEntry;
+            private GridDhtCacheEntry lastEntry;
 
-            @Override public boolean hasNext() {
-                return it.hasNext();
+            {
+                lastEntry = advance();
             }
 
-            @Override public GridDhtCacheEntry next() {
-                Map.Entry<byte[], GridCacheSwapEntry> entry = it.next();
+            private GridDhtCacheEntry advance() {
+                if (it.hasNext()) {
+                    Map.Entry<byte[], GridCacheSwapEntry> entry = it.next();
 
-                byte[] keyBytes = entry.getKey();
+                    byte[] keyBytes = entry.getKey();
 
-                while (true) {
-                    try {
-                        KeyCacheObject key = cctx.toCacheKeyObject(keyBytes);
+                    while (true) {
+                        try {
+                            KeyCacheObject key = cctx.toCacheKeyObject(keyBytes);
 
-                        lastEntry = (GridDhtCacheEntry)cctx.cache().entryEx(key, false);
+                            lastEntry = (GridDhtCacheEntry)cctx.cache().entryEx(key, false);
 
-                        lastEntry.unswap(true);
+                            lastEntry.unswap(true);
 
-                        return lastEntry;
-                    }
-                    catch (GridCacheEntryRemovedException e) {
-                        if (log.isDebugEnabled())
-                            log.debug("Got removed entry: " + lastEntry);
-                    }
-                    catch (IgniteCheckedException e) {
-                        throw new CacheException(e);
+                            return lastEntry;
+                        }
+                        catch (GridCacheEntryRemovedException e) {
+                            if (log.isDebugEnabled())
+                                log.debug("Got removed entry: " + lastEntry);
+                        }
+                        catch (IgniteCheckedException e) {
+                            throw new CacheException(e);
+                        }
+                        catch (GridDhtInvalidPartitionException e) {
+                            if (log.isDebugEnabled())
+                                log.debug("Got invalid partition exception: " + e);
+
+                            return null;
+                        }
                     }
                 }
+
+                return null;
+            }
+
+            @Override public boolean hasNext() {
+                return lastEntry != null;
+            }
+
+            @Override public GridDhtCacheEntry next() {
+                if (lastEntry == null)
+                    throw new NoSuchElementException();
+
+                return lastEntry;
             }
 
             @Override public void remove() {
+                if (lastEntry == null)
+                    throw new NoSuchElementException();
+
                 map.remove(lastEntry.key(), lastEntry);
             }
         };


[31/55] [abbrv] ignite git commit: Release notes updated

Posted by ag...@apache.org.
Release notes updated


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

Branch: refs/heads/ignite-1171
Commit: f9776134a7820b7ee82c60a11f62b4a0ab9ec4d1
Parents: a7490a6
Author: Anton Vinogradov <av...@apache.org>
Authored: Tue Sep 15 13:01:20 2015 +0300
Committer: Anton Vinogradov <av...@apache.org>
Committed: Tue Sep 15 13:01:20 2015 +0300

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


http://git-wip-us.apache.org/repos/asf/ignite/blob/f9776134/RELEASE_NOTES.txt
----------------------------------------------------------------------
diff --git a/RELEASE_NOTES.txt b/RELEASE_NOTES.txt
index 3f3ac7b..d7097fd 100644
--- a/RELEASE_NOTES.txt
+++ b/RELEASE_NOTES.txt
@@ -6,6 +6,7 @@ Apache Ignite In-Memory Data Fabric 1.4
 * Added SSL support to communication and discovery.
 * Added support for log4j2.
 * Added versioned entry to cache API.
+* Added client node based JDBC driver implementation.
 * Fixed IGNITE_HOME resolution with JBoss.
 * Fixed configuration file resolution on classpath.
 * Fixed MBean registration for caches.


[13/55] [abbrv] ignite git commit: Merge branch 'ignite-1.4' of https://git-wip-us.apache.org/repos/asf/ignite into ignite-1.4-main

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


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

Branch: refs/heads/ignite-1171
Commit: e5f16818c20c774c2dcccfacbd85040a071edce9
Parents: 71379a8 c01313d
Author: Denis Magda <dm...@gridgain.com>
Authored: Mon Sep 14 17:36:15 2015 +0300
Committer: Denis Magda <dm...@gridgain.com>
Committed: Mon Sep 14 17:36:15 2015 +0300

----------------------------------------------------------------------
 .../processors/platform/services/PlatformAbstractService.java     | 3 ++-
 1 file changed, 2 insertions(+), 1 deletion(-)
----------------------------------------------------------------------



[18/55] [abbrv] ignite git commit: IGNITE-1197 - Reverted fix.

Posted by ag...@apache.org.
IGNITE-1197 - Reverted fix.


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

Branch: refs/heads/ignite-1171
Commit: c70680a8be837258ae3e10d034f1a53522d6f0f8
Parents: cb9b766
Author: Alexey Goncharuk <al...@gmail.com>
Authored: Mon Sep 14 18:53:29 2015 -0700
Committer: Alexey Goncharuk <al...@gmail.com>
Committed: Mon Sep 14 18:53:29 2015 -0700

----------------------------------------------------------------------
 .../distributed/dht/GridDhtLocalPartition.java  | 62 ++++++--------------
 1 file changed, 19 insertions(+), 43 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/c70680a8/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtLocalPartition.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtLocalPartition.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtLocalPartition.java
index a58451f..c5f15cd 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtLocalPartition.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtLocalPartition.java
@@ -708,62 +708,38 @@ public class GridDhtLocalPartition implements Comparable<GridDhtLocalPartition>,
 
         return new Iterator<GridDhtCacheEntry>() {
             /** */
-            private GridDhtCacheEntry lastEntry;
+            GridDhtCacheEntry lastEntry;
 
-            {
-                lastEntry = advance();
+            @Override public boolean hasNext() {
+                return it.hasNext();
             }
 
-            private GridDhtCacheEntry advance() {
-                if (it.hasNext()) {
-                    Map.Entry<byte[], GridCacheSwapEntry> entry = it.next();
-
-                    byte[] keyBytes = entry.getKey();
+            @Override public GridDhtCacheEntry next() {
+                Map.Entry<byte[], GridCacheSwapEntry> entry = it.next();
 
-                    while (true) {
-                        try {
-                            KeyCacheObject key = cctx.toCacheKeyObject(keyBytes);
+                byte[] keyBytes = entry.getKey();
 
-                            lastEntry = (GridDhtCacheEntry)cctx.cache().entryEx(key, false);
+                while (true) {
+                    try {
+                        KeyCacheObject key = cctx.toCacheKeyObject(keyBytes);
 
-                            lastEntry.unswap(true);
+                        lastEntry = (GridDhtCacheEntry)cctx.cache().entryEx(key, false);
 
-                            return lastEntry;
-                        }
-                        catch (GridCacheEntryRemovedException e) {
-                            if (log.isDebugEnabled())
-                                log.debug("Got removed entry: " + lastEntry);
-                        }
-                        catch (IgniteCheckedException e) {
-                            throw new CacheException(e);
-                        }
-                        catch (GridDhtInvalidPartitionException e) {
-                            if (log.isDebugEnabled())
-                                log.debug("Got invalid partition exception: " + e);
+                        lastEntry.unswap(true);
 
-                            return null;
-                        }
+                        return lastEntry;
+                    }
+                    catch (GridCacheEntryRemovedException e) {
+                        if (log.isDebugEnabled())
+                            log.debug("Got removed entry: " + lastEntry);
+                    }
+                    catch (IgniteCheckedException e) {
+                        throw new CacheException(e);
                     }
                 }
-
-                return null;
-            }
-
-            @Override public boolean hasNext() {
-                return lastEntry != null;
-            }
-
-            @Override public GridDhtCacheEntry next() {
-                if (lastEntry == null)
-                    throw new NoSuchElementException();
-
-                return lastEntry;
             }
 
             @Override public void remove() {
-                if (lastEntry == null)
-                    throw new NoSuchElementException();
-
                 map.remove(lastEntry.key(), lastEntry);
             }
         };


[23/55] [abbrv] ignite git commit: IGNITE-1392: Fixed.

Posted by ag...@apache.org.
IGNITE-1392: Fixed.


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

Branch: refs/heads/ignite-1171
Commit: 2db59e68913ad05aaeb2bbb3351d84f773cd6f3e
Parents: f8b798d
Author: vozerov-gridgain <vo...@gridgain.com>
Authored: Tue Sep 15 09:40:36 2015 +0300
Committer: vozerov-gridgain <vo...@gridgain.com>
Committed: Tue Sep 15 09:40:36 2015 +0300

----------------------------------------------------------------------
 examples/config/filesystem/README.txt       | 2 +-
 examples/config/filesystem/example-igfs.xml | 7 -------
 2 files changed, 1 insertion(+), 8 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/2db59e68/examples/config/filesystem/README.txt
----------------------------------------------------------------------
diff --git a/examples/config/filesystem/README.txt b/examples/config/filesystem/README.txt
index c01881c..4f6ae88 100644
--- a/examples/config/filesystem/README.txt
+++ b/examples/config/filesystem/README.txt
@@ -2,7 +2,7 @@ FileSystem Configuration Example
 --------------------------------
 
 This folder contains configuration files for IgniteFs examples located in
-org.apache.ignite.examples.fs package.
+org.apache.ignite.examples.igfs package.
 
 - example-igfs.xml file is used to start Apache Ignite nodes with IgniteFS configured
 - core-site.xml file is used to run Hadoop FS driver over IgniteFs

http://git-wip-us.apache.org/repos/asf/ignite/blob/2db59e68/examples/config/filesystem/example-igfs.xml
----------------------------------------------------------------------
diff --git a/examples/config/filesystem/example-igfs.xml b/examples/config/filesystem/example-igfs.xml
index 41ac865..0a0b1b4 100644
--- a/examples/config/filesystem/example-igfs.xml
+++ b/examples/config/filesystem/example-igfs.xml
@@ -76,13 +76,6 @@
                     <property name="prefetchBlocks" value="32"/>
 
                     <!--
-                        This will disable IPC endpoint binding thus preventing Hadoop clients from connecting to IgniteFs.
-                        With this option set to false the only possible way to interact with IgniteFs is through
-                        Ignite API.
-                    -->
-                    <property name="ipcEndpointEnabled" value="false"/>
-
-                    <!--
                         Example of configured IPC loopback endpoint.
                     -->
                     <!--


[40/55] [abbrv] 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/3e762e25
Tree: http://git-wip-us.apache.org/repos/asf/ignite/tree/3e762e25
Diff: http://git-wip-us.apache.org/repos/asf/ignite/diff/3e762e25

Branch: refs/heads/ignite-1171
Commit: 3e762e25e91f3223fcd2a78bcd80651784e4c32e
Parents: adf5ef6 6e19979
Author: vozerov-gridgain <vo...@gridgain.com>
Authored: Tue Sep 15 16:23:33 2015 +0300
Committer: vozerov-gridgain <vo...@gridgain.com>
Committed: Tue Sep 15 16:23:33 2015 +0300

----------------------------------------------------------------------
 .../processors/cache/GridCacheProcessor.java    |  10 +-
 .../dht/GridClientPartitionTopology.java        | 104 +++++++++++--------
 .../dht/GridDhtPartitionTopology.java           |   4 +-
 .../dht/GridDhtPartitionTopologyImpl.java       |   7 +-
 .../ignite/internal/util/IgniteUtils.java       |  16 +++
 5 files changed, 87 insertions(+), 54 deletions(-)
----------------------------------------------------------------------



[28/55] [abbrv] ignite git commit: ignite-971 Fixed offheap to swap eviction, added failover tests with swap/offheap, added retries for tx 'check backup' rollback.

Posted by ag...@apache.org.
http://git-wip-us.apache.org/repos/asf/ignite/blob/a7490a6e/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheFailoverTestSuite3.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheFailoverTestSuite3.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheFailoverTestSuite3.java
new file mode 100644
index 0000000..318db9e
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheFailoverTestSuite3.java
@@ -0,0 +1,62 @@
+/*
+ * 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.testsuites;
+
+import junit.framework.TestSuite;
+import org.apache.ignite.internal.processors.cache.GridCacheIncrementTransformTest;
+import org.apache.ignite.internal.processors.cache.IgniteCacheTopologySafeGetSelfTest;
+import org.apache.ignite.internal.processors.cache.distributed.*;
+import org.apache.ignite.internal.processors.cache.distributed.dht.*;
+import org.apache.ignite.internal.processors.cache.distributed.dht.atomic.*;
+import org.apache.ignite.internal.processors.cache.distributed.near.GridCacheAtomicNearRemoveFailureTest;
+import org.apache.ignite.internal.processors.cache.distributed.near.GridCacheAtomicPrimaryWriteOrderNearRemoveFailureTest;
+import org.apache.ignite.internal.processors.cache.distributed.near.GridCacheNearRemoveFailureTest;
+import org.apache.ignite.spi.communication.tcp.IgniteCacheSslStartStopSelfTest;
+import org.apache.ignite.testframework.GridTestUtils;
+
+import java.util.Set;
+
+/**
+ * Test suite.
+ */
+public class IgniteCacheFailoverTestSuite3 extends TestSuite {
+    /**
+     * @return Ignite Cache Failover test suite.
+     * @throws Exception Thrown in case of the failure.
+     */
+    public static TestSuite suite() throws Exception {
+        return suite(null);
+    }
+
+    /**
+     * @param ignoredTests Tests don't include in the execution.
+     * @return Test suite.
+     * @throws Exception Thrown in case of the failure.
+     */
+    public static TestSuite suite(Set<Class> ignoredTests) throws Exception {
+        TestSuite suite = new TestSuite("Cache Failover Test Suite3");
+
+        suite.addTestSuite(IgniteCachePutRetryAtomicSelfTest.class);
+        suite.addTestSuite(IgniteCachePutRetryAtomicPrimaryWriteOrderSelfTest.class);
+        suite.addTestSuite(IgniteCachePutRetryTransactionalSelfTest.class);
+
+        suite.addTestSuite(IgniteCacheSslStartStopSelfTest.class);
+
+        return suite;
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/a7490a6e/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheRestartTestSuite.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheRestartTestSuite.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheRestartTestSuite.java
index 869f25c..34ef79b 100644
--- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheRestartTestSuite.java
+++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheRestartTestSuite.java
@@ -18,18 +18,13 @@
 package org.apache.ignite.testsuites;
 
 import junit.framework.TestSuite;
-import org.apache.ignite.internal.processors.cache.GridCachePutAllFailoverSelfTest;
-import org.apache.ignite.internal.processors.cache.IgniteCacheAtomicPutAllFailoverSelfTest;
-import org.apache.ignite.internal.processors.cache.IgniteCachePutAllRestartTest;
-import org.apache.ignite.internal.processors.cache.distributed.IgniteCacheAtomicNodeRestartTest;
 import org.apache.ignite.internal.processors.cache.distributed.dht.GridCachePartitionedNearDisabledOptimisticTxNodeRestartTest;
 import org.apache.ignite.internal.processors.cache.distributed.near.GridCachePartitionedNodeRestartTest;
 import org.apache.ignite.internal.processors.cache.distributed.near.GridCachePartitionedOptimisticTxNodeRestartTest;
 import org.apache.ignite.internal.processors.cache.distributed.replicated.GridCacheReplicatedNodeRestartSelfTest;
-import org.apache.ignite.internal.processors.cache.distributed.replicated.IgniteCacheAtomicReplicatedNodeRestartSelfTest;
 
 /**
- * In-Memory Data Grid stability test suite on changing topology.
+ * Cache stability test suite on changing topology.
  */
 public class IgniteCacheRestartTestSuite extends TestSuite {
     /**
@@ -44,13 +39,6 @@ public class IgniteCacheRestartTestSuite extends TestSuite {
         suite.addTestSuite(GridCacheReplicatedNodeRestartSelfTest.class);
         suite.addTestSuite(GridCachePartitionedNearDisabledOptimisticTxNodeRestartTest.class);
 
-        suite.addTestSuite(IgniteCacheAtomicNodeRestartTest.class);
-        suite.addTestSuite(IgniteCacheAtomicReplicatedNodeRestartSelfTest.class);
-
-        suite.addTestSuite(IgniteCacheAtomicPutAllFailoverSelfTest.class);
-        suite.addTestSuite(IgniteCachePutAllRestartTest.class);
-        suite.addTestSuite(GridCachePutAllFailoverSelfTest.class);
-
         return suite;
     }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/a7490a6e/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheRestartTestSuite2.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheRestartTestSuite2.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheRestartTestSuite2.java
new file mode 100644
index 0000000..c9e9467
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheRestartTestSuite2.java
@@ -0,0 +1,47 @@
+/*
+ * 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.testsuites;
+
+import junit.framework.TestSuite;
+import org.apache.ignite.internal.processors.cache.GridCachePutAllFailoverSelfTest;
+import org.apache.ignite.internal.processors.cache.IgniteCacheAtomicPutAllFailoverSelfTest;
+import org.apache.ignite.internal.processors.cache.IgniteCachePutAllRestartTest;
+import org.apache.ignite.internal.processors.cache.distributed.IgniteCacheAtomicNodeRestartTest;
+import org.apache.ignite.internal.processors.cache.distributed.replicated.IgniteCacheAtomicReplicatedNodeRestartSelfTest;
+
+/**
+ * Cache stability test suite on changing topology.
+ */
+public class IgniteCacheRestartTestSuite2 extends TestSuite {
+    /**
+     * @return Suite.
+     * @throws Exception If failed.
+     */
+    public static TestSuite suite() throws Exception {
+        TestSuite suite = new TestSuite("Cache Restart Test Suite2");
+
+        suite.addTestSuite(IgniteCacheAtomicNodeRestartTest.class);
+        suite.addTestSuite(IgniteCacheAtomicReplicatedNodeRestartSelfTest.class);
+
+        suite.addTestSuite(IgniteCacheAtomicPutAllFailoverSelfTest.class);
+        suite.addTestSuite(IgniteCachePutAllRestartTest.class);
+        suite.addTestSuite(GridCachePutAllFailoverSelfTest.class);
+
+        return suite;
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/a7490a6e/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2AbstractKeyValueRow.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2AbstractKeyValueRow.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2AbstractKeyValueRow.java
index 63e2bc3..07c49a5 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2AbstractKeyValueRow.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2AbstractKeyValueRow.java
@@ -182,9 +182,11 @@ public abstract class GridH2AbstractKeyValueRow extends GridH2Row {
     /** {@inheritDoc} */
     @Override public Value getValue(int col) {
         if (col < DEFAULT_COLUMNS_COUNT) {
-            Value v = peekValue(col);
+            Value v;
 
             if (col == VAL_COL) {
+                v = syncValue(0);
+
                 long start = 0;
                 int attempt = 0;
 
@@ -206,11 +208,15 @@ public abstract class GridH2AbstractKeyValueRow extends GridH2Row {
                         Object valObj = desc.readFromSwap(k);
 
                         if (valObj != null) {
-                            Value upd = desc.wrap(valObj, desc.valueType());
+                            // Even if we've found valObj in swap, it is may be some new value,
+                            // while the needed value was already unswapped, so we have to recheck it.
+                            if ((v = WeakValue.unwrap(syncValue(0))) == null && (v = getOffheapValue(VAL_COL)) == null) {
+                                Value upd = desc.wrap(valObj, desc.valueType());
 
-                            v = updateWeakValue(upd);
+                                v = updateWeakValue(upd);
 
-                            return v == null ? upd : v;
+                                return v == null ? upd : v;
+                            }
                         }
                         else {
                             // If nothing found in swap then we should be already unswapped.
@@ -230,18 +236,21 @@ public abstract class GridH2AbstractKeyValueRow extends GridH2Row {
                             ". This can happen due to a long GC pause.");
                 }
             }
-
-            if (v == null) {
+            else {
                 assert col == KEY_COL : col;
 
-                v = getOffheapValue(KEY_COL);
+                v = peekValue(KEY_COL);
 
-                assert v != null : v;
+                if (v == null) {
+                    v = getOffheapValue(KEY_COL);
 
-                setValue(KEY_COL, v);
+                    assert v != null;
 
-                if (peekValue(VAL_COL) == null)
-                    cache();
+                    setValue(KEY_COL, v);
+
+                    if (peekValue(VAL_COL) == null)
+                        cache();
+                }
             }
 
             assert !(v instanceof WeakValue) : v;

http://git-wip-us.apache.org/repos/asf/ignite/blob/a7490a6e/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheOffHeapAndSwapSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheOffHeapAndSwapSelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheOffHeapAndSwapSelfTest.java
index 8f453a5..eb0880a 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheOffHeapAndSwapSelfTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheOffHeapAndSwapSelfTest.java
@@ -249,6 +249,10 @@ public class GridCacheOffHeapAndSwapSelfTest extends GridCommonAbstractTest {
             assertEquals(ENTRY_CNT - i - 1, cache.localSize(CachePeekMode.ONHEAP));
         }
 
+        log.info("Cache size [onheap=" + cache.localSize(CachePeekMode.ONHEAP) +
+            ", offheap=" + cache.localSize(CachePeekMode.OFFHEAP) +
+            ", swap=" + cache.localSize(CachePeekMode.SWAP) + ']');
+
         // Ensure that part of entries located in off-heap memory and part is swapped.
         assertEquals(0, cache.localSize(CachePeekMode.ONHEAP));
         assertTrue(cache.localSize(CachePeekMode.OFFHEAP) > 0);

http://git-wip-us.apache.org/repos/asf/ignite/blob/a7490a6e/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheQueryMultiThreadedSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheQueryMultiThreadedSelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheQueryMultiThreadedSelfTest.java
index b02b37e..be644e2 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheQueryMultiThreadedSelfTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheQueryMultiThreadedSelfTest.java
@@ -32,7 +32,6 @@ import org.apache.ignite.IgniteCache;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.cache.CacheMemoryMode;
 import org.apache.ignite.cache.CachePeekMode;
-import org.apache.ignite.cache.CacheWriteSynchronizationMode;
 import org.apache.ignite.cache.eviction.lru.LruEvictionPolicy;
 import org.apache.ignite.cache.query.ScanQuery;
 import org.apache.ignite.cache.query.SqlQuery;
@@ -55,6 +54,7 @@ import org.jetbrains.annotations.Nullable;
 
 import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL;
 import static org.apache.ignite.cache.CacheMode.PARTITIONED;
+import static org.apache.ignite.cache.CacheWriteSynchronizationMode.FULL_SYNC;
 
 /**
  * Multi-threaded tests for cache queries.
@@ -111,7 +111,7 @@ public class IgniteCacheQueryMultiThreadedSelfTest extends GridCommonAbstractTes
 
         cacheCfg.setCacheMode(PARTITIONED);
         cacheCfg.setAtomicityMode(TRANSACTIONAL);
-        cacheCfg.setWriteSynchronizationMode(CacheWriteSynchronizationMode.FULL_SYNC);
+        cacheCfg.setWriteSynchronizationMode(FULL_SYNC);
         cacheCfg.setSwapEnabled(true);
         cacheCfg.setBackups(1);
 
@@ -139,6 +139,11 @@ public class IgniteCacheQueryMultiThreadedSelfTest extends GridCommonAbstractTes
         return cacheCfg;
     }
 
+    /** {@inheritDoc} */
+    @Override protected long getTestTimeout() {
+        return DURATION + 60_000;
+    }
+
     /**
      *
      */

http://git-wip-us.apache.org/repos/asf/ignite/blob/a7490a6e/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheQueryOffheapEvictsMultiThreadedSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheQueryOffheapEvictsMultiThreadedSelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheQueryOffheapEvictsMultiThreadedSelfTest.java
index 909fd74..d7d2b5a 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheQueryOffheapEvictsMultiThreadedSelfTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheQueryOffheapEvictsMultiThreadedSelfTest.java
@@ -22,11 +22,6 @@ package org.apache.ignite.internal.processors.cache;
  */
 public class IgniteCacheQueryOffheapEvictsMultiThreadedSelfTest extends IgniteCacheQueryOffheapMultiThreadedSelfTest {
     /** {@inheritDoc} */
-    @Override protected void beforeTestsStarted() throws Exception {
-        fail("IGNITE-959");
-    }
-
-    /** {@inheritDoc} */
     @Override protected boolean evictsEnabled() {
         return true;
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/a7490a6e/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/IgniteCacheClientQueryReplicatedNodeRestartSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/IgniteCacheClientQueryReplicatedNodeRestartSelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/IgniteCacheClientQueryReplicatedNodeRestartSelfTest.java
index 7e72292..5dbb12c 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/IgniteCacheClientQueryReplicatedNodeRestartSelfTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/IgniteCacheClientQueryReplicatedNodeRestartSelfTest.java
@@ -315,7 +315,7 @@ public class IgniteCacheClientQueryReplicatedNodeRestartSelfTest extends GridCom
                         info("Executed queries: " + c);
                 }
             }
-        }, qryThreadNum);
+        }, qryThreadNum, "query-thread");
 
         final AtomicInteger restartCnt = new AtomicInteger();
 
@@ -334,10 +334,14 @@ public class IgniteCacheClientQueryReplicatedNodeRestartSelfTest extends GridCom
                     }
                     while (!locks.compareAndSet(g, 0, -1));
 
+                    log.info("Stop node: " + g);
+
                     stopGrid(g);
 
                     Thread.sleep(rnd.nextInt(nodeLifeTime));
 
+                    log.info("Start node: " + g);
+
                     startGrid(g);
 
                     Thread.sleep(rnd.nextInt(nodeLifeTime));
@@ -352,7 +356,7 @@ public class IgniteCacheClientQueryReplicatedNodeRestartSelfTest extends GridCom
 
                 return true;
             }
-        }, restartThreadsNum);
+        }, restartThreadsNum, "restart-thread");
 
         Thread.sleep(duration);
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/a7490a6e/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/IgniteCacheQueryNodeRestartSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/IgniteCacheQueryNodeRestartSelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/IgniteCacheQueryNodeRestartSelfTest.java
index 32da55a..45a249d 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/IgniteCacheQueryNodeRestartSelfTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/IgniteCacheQueryNodeRestartSelfTest.java
@@ -149,7 +149,7 @@ public class IgniteCacheQueryNodeRestartSelfTest extends GridCacheAbstractSelfTe
                         info("Executed queries: " + c);
                 }
             }
-        }, qryThreadNum);
+        }, qryThreadNum, "query-thread");
 
         final AtomicInteger restartCnt = new AtomicInteger();
 
@@ -178,7 +178,7 @@ public class IgniteCacheQueryNodeRestartSelfTest extends GridCacheAbstractSelfTe
 
                 return true;
             }
-        }, 1);
+        }, 1, "restart-thread");
 
         Thread.sleep(duration);
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/a7490a6e/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/IgniteCacheQueryNodeRestartSelfTest2.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/IgniteCacheQueryNodeRestartSelfTest2.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/IgniteCacheQueryNodeRestartSelfTest2.java
index 9e903d1..1276405 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/IgniteCacheQueryNodeRestartSelfTest2.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/IgniteCacheQueryNodeRestartSelfTest2.java
@@ -277,7 +277,7 @@ public class IgniteCacheQueryNodeRestartSelfTest2 extends GridCommonAbstractTest
                         info("Executed queries: " + c);
                 }
             }
-        }, qryThreadNum);
+        }, qryThreadNum, "query-thread");
 
         final AtomicInteger restartCnt = new AtomicInteger();
 
@@ -296,10 +296,14 @@ public class IgniteCacheQueryNodeRestartSelfTest2 extends GridCommonAbstractTest
                     }
                     while (!locks.compareAndSet(g, 0, -1));
 
+                    log.info("Stop node: " + g);
+
                     stopGrid(g);
 
                     Thread.sleep(rnd.nextInt(nodeLifeTime));
 
+                    log.info("Start node: " + g);
+
                     startGrid(g);
 
                     Thread.sleep(rnd.nextInt(nodeLifeTime));
@@ -314,7 +318,7 @@ public class IgniteCacheQueryNodeRestartSelfTest2 extends GridCommonAbstractTest
 
                 return true;
             }
-        }, restartThreadsNum);
+        }, restartThreadsNum, "restart-thread");
 
         Thread.sleep(duration);
 


[02/55] [abbrv] ignite git commit: ignite-1462: hid portable API in 1.4 release

Posted by ag...@apache.org.
http://git-wip-us.apache.org/repos/asf/ignite/blob/71379a80/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/distributed/dht/GridCacheAtomicPartitionedOnlyPortableDataStreamerMultithreadedSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/distributed/dht/GridCacheAtomicPartitionedOnlyPortableDataStreamerMultithreadedSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/distributed/dht/GridCacheAtomicPartitionedOnlyPortableDataStreamerMultithreadedSelfTest.java
deleted file mode 100644
index 3f8cd1c..0000000
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/distributed/dht/GridCacheAtomicPartitionedOnlyPortableDataStreamerMultithreadedSelfTest.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.processors.cache.portable.distributed.dht;
-
-import org.apache.ignite.cache.CacheAtomicityMode;
-import org.apache.ignite.cache.CacheMode;
-import org.apache.ignite.configuration.NearCacheConfiguration;
-import org.apache.ignite.internal.processors.cache.portable.GridCachePortableObjectsAbstractDataStreamerSelfTest;
-
-import static org.apache.ignite.cache.CacheAtomicityMode.ATOMIC;
-import static org.apache.ignite.cache.CacheMode.PARTITIONED;
-
-/**
- *
- */
-public class GridCacheAtomicPartitionedOnlyPortableDataStreamerMultithreadedSelfTest extends
-    GridCachePortableObjectsAbstractDataStreamerSelfTest {
-    /** {@inheritDoc} */
-    @Override protected CacheMode cacheMode() {
-        return PARTITIONED;
-    }
-
-    /** {@inheritDoc} */
-    @Override protected CacheAtomicityMode atomicityMode() {
-        return ATOMIC;
-    }
-
-    /** {@inheritDoc} */
-    @Override protected NearCacheConfiguration nearConfiguration() {
-        return null;
-    }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/71379a80/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/distributed/dht/GridCacheAtomicPartitionedOnlyPortableMultiNodeSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/distributed/dht/GridCacheAtomicPartitionedOnlyPortableMultiNodeSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/distributed/dht/GridCacheAtomicPartitionedOnlyPortableMultiNodeSelfTest.java
deleted file mode 100644
index a53a5ea..0000000
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/distributed/dht/GridCacheAtomicPartitionedOnlyPortableMultiNodeSelfTest.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.processors.cache.portable.distributed.dht;
-
-/**
- *
- */
-public class GridCacheAtomicPartitionedOnlyPortableMultiNodeSelfTest extends
-    GridCacheAtomicPartitionedOnlyPortableMultithreadedSelfTest {
-    /** {@inheritDoc} */
-    @Override protected int gridCount() {
-        return 4;
-    }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/71379a80/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/distributed/dht/GridCacheAtomicPartitionedOnlyPortableMultithreadedSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/distributed/dht/GridCacheAtomicPartitionedOnlyPortableMultithreadedSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/distributed/dht/GridCacheAtomicPartitionedOnlyPortableMultithreadedSelfTest.java
deleted file mode 100644
index 8f3a05f..0000000
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/distributed/dht/GridCacheAtomicPartitionedOnlyPortableMultithreadedSelfTest.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.processors.cache.portable.distributed.dht;
-
-import org.apache.ignite.cache.CacheAtomicityMode;
-import org.apache.ignite.cache.CacheMode;
-import org.apache.ignite.configuration.NearCacheConfiguration;
-import org.apache.ignite.internal.processors.cache.portable.GridCachePortableObjectsAbstractMultiThreadedSelfTest;
-
-import static org.apache.ignite.cache.CacheAtomicityMode.ATOMIC;
-import static org.apache.ignite.cache.CacheMode.PARTITIONED;
-
-/**
- *
- */
-public class GridCacheAtomicPartitionedOnlyPortableMultithreadedSelfTest extends
-    GridCachePortableObjectsAbstractMultiThreadedSelfTest {
-    /** {@inheritDoc} */
-    @Override protected CacheMode cacheMode() {
-        return PARTITIONED;
-    }
-
-    /** {@inheritDoc} */
-    @Override protected CacheAtomicityMode atomicityMode() {
-        return ATOMIC;
-    }
-
-    /** {@inheritDoc} */
-    @Override protected NearCacheConfiguration nearConfiguration() {
-        return null;
-    }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/71379a80/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/distributed/dht/GridCacheMemoryModePortableSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/distributed/dht/GridCacheMemoryModePortableSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/distributed/dht/GridCacheMemoryModePortableSelfTest.java
deleted file mode 100644
index ab6b0dd..0000000
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/distributed/dht/GridCacheMemoryModePortableSelfTest.java
+++ /dev/null
@@ -1,36 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.ignite.internal.processors.cache.portable.distributed.dht;
-
-import org.apache.ignite.configuration.IgniteConfiguration;
-import org.apache.ignite.internal.processors.cache.GridCacheMemoryModeSelfTest;
-import org.apache.ignite.marshaller.portable.PortableMarshaller;
-
-/**
- * Memory models test.
- */
-public class GridCacheMemoryModePortableSelfTest extends GridCacheMemoryModeSelfTest {
-    /** {@inheritDoc} */
-    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
-        IgniteConfiguration cfg = super.getConfiguration(gridName);
-
-        cfg.setMarshaller(new PortableMarshaller());
-
-        return cfg;
-    }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/71379a80/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/distributed/dht/GridCacheOffHeapTieredAtomicPortableSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/distributed/dht/GridCacheOffHeapTieredAtomicPortableSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/distributed/dht/GridCacheOffHeapTieredAtomicPortableSelfTest.java
deleted file mode 100644
index c845257..0000000
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/distributed/dht/GridCacheOffHeapTieredAtomicPortableSelfTest.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.processors.cache.portable.distributed.dht;
-
-import java.util.Arrays;
-import org.apache.ignite.configuration.IgniteConfiguration;
-import org.apache.ignite.internal.processors.cache.GridCacheOffHeapTieredAtomicSelfTest;
-import org.apache.ignite.marshaller.portable.PortableMarshaller;
-
-/**
- *
- */
-public class GridCacheOffHeapTieredAtomicPortableSelfTest extends GridCacheOffHeapTieredAtomicSelfTest {
-    /** {@inheritDoc} */
-    @Override protected boolean portableEnabled() {
-        return true;
-    }
-
-    /** {@inheritDoc} */
-    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
-        // Enable portables.
-        IgniteConfiguration cfg = super.getConfiguration(gridName);
-
-        PortableMarshaller marsh = new PortableMarshaller();
-
-        marsh.setClassNames(Arrays.asList(TestValue.class.getName()));
-
-        cfg.setMarshaller(marsh);
-
-        return cfg;
-    }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/71379a80/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/distributed/dht/GridCacheOffHeapTieredEvictionAtomicPortableSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/distributed/dht/GridCacheOffHeapTieredEvictionAtomicPortableSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/distributed/dht/GridCacheOffHeapTieredEvictionAtomicPortableSelfTest.java
deleted file mode 100644
index 1a0d601..0000000
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/distributed/dht/GridCacheOffHeapTieredEvictionAtomicPortableSelfTest.java
+++ /dev/null
@@ -1,95 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.ignite.internal.processors.cache.portable.distributed.dht;
-
-import java.util.Arrays;
-import org.apache.ignite.configuration.IgniteConfiguration;
-import org.apache.ignite.internal.processors.cache.GridCacheOffHeapTieredEvictionAtomicSelfTest;
-import org.apache.ignite.marshaller.portable.PortableMarshaller;
-import org.apache.ignite.portable.PortableObject;
-
-/**
- *
- */
-public class GridCacheOffHeapTieredEvictionAtomicPortableSelfTest extends GridCacheOffHeapTieredEvictionAtomicSelfTest {
-    /** {@inheritDoc} */
-    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
-        // Enable portables.
-        IgniteConfiguration cfg = super.getConfiguration(gridName);
-
-        PortableMarshaller marsh = new PortableMarshaller();
-
-        marsh.setClassNames(Arrays.asList(TestValue.class.getName()));
-
-        cfg.setMarshaller(marsh);
-
-        return cfg;
-    }
-
-    /** {@inheritDoc} */
-    @Override protected TestPredicate testPredicate(String expVal, boolean acceptNull) {
-        return new PortableValuePredicate(expVal, acceptNull);
-    }
-
-    /** {@inheritDoc} */
-    @Override protected TestProcessor testClosure(String expVal, boolean acceptNull) {
-        return new PortableValueClosure(expVal, acceptNull);
-    }
-
-    /**
-     *
-     */
-    @SuppressWarnings("PackageVisibleInnerClass")
-    static class PortableValuePredicate extends TestPredicate {
-        /**
-         * @param expVal Expected value.
-         * @param acceptNull If {@code true} value can be null;
-         */
-        PortableValuePredicate(String expVal, boolean acceptNull) {
-            super(expVal, acceptNull);
-        }
-
-        /** {@inheritDoc} */
-        @Override public void checkValue(Object val) {
-            PortableObject obj = (PortableObject)val;
-
-            assertEquals(expVal, obj.field("val"));
-        }
-    }
-
-    /**
-     *
-     */
-    @SuppressWarnings("PackageVisibleInnerClass")
-    static class PortableValueClosure extends TestProcessor {
-        /**
-         * @param expVal Expected value.
-         * @param acceptNull If {@code true} value can be null;
-         */
-        PortableValueClosure(String expVal, boolean acceptNull) {
-            super(expVal, acceptNull);
-        }
-
-        /** {@inheritDoc} */
-        @Override public void checkValue(Object val) {
-            PortableObject obj = (PortableObject)val;
-
-            assertEquals(expVal, obj.field("val"));
-        }
-    }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/71379a80/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/distributed/dht/GridCacheOffHeapTieredEvictionPortableSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/distributed/dht/GridCacheOffHeapTieredEvictionPortableSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/distributed/dht/GridCacheOffHeapTieredEvictionPortableSelfTest.java
deleted file mode 100644
index 60eed45..0000000
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/distributed/dht/GridCacheOffHeapTieredEvictionPortableSelfTest.java
+++ /dev/null
@@ -1,95 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.ignite.internal.processors.cache.portable.distributed.dht;
-
-import java.util.Arrays;
-import org.apache.ignite.configuration.IgniteConfiguration;
-import org.apache.ignite.internal.processors.cache.GridCacheOffHeapTieredEvictionSelfTest;
-import org.apache.ignite.marshaller.portable.PortableMarshaller;
-import org.apache.ignite.portable.PortableObject;
-
-/**
- *
- */
-public class GridCacheOffHeapTieredEvictionPortableSelfTest extends GridCacheOffHeapTieredEvictionSelfTest {
-    /** {@inheritDoc} */
-    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
-        // Enable portables.
-        IgniteConfiguration cfg = super.getConfiguration(gridName);
-
-        PortableMarshaller marsh = new PortableMarshaller();
-
-        marsh.setClassNames(Arrays.asList(TestValue.class.getName()));
-
-        cfg.setMarshaller(marsh);
-
-        return cfg;
-    }
-
-    /** {@inheritDoc} */
-    @Override protected TestPredicate testPredicate(String expVal, boolean acceptNull) {
-        return new PortableValuePredicate(expVal, acceptNull);
-    }
-
-    /** {@inheritDoc} */
-    @Override protected TestProcessor testClosure(String expVal, boolean acceptNull) {
-        return new PortableValueClosure(expVal, acceptNull);
-    }
-
-    /**
-     *
-     */
-    @SuppressWarnings("PackageVisibleInnerClass")
-    static class PortableValuePredicate extends TestPredicate {
-        /**
-         * @param expVal Expected value.
-         * @param acceptNull If {@code true} value can be null;
-         */
-        PortableValuePredicate(String expVal, boolean acceptNull) {
-            super(expVal, acceptNull);
-        }
-
-        /** {@inheritDoc} */
-        @Override public void checkValue(Object val) {
-            PortableObject obj = (PortableObject)val;
-
-            assertEquals(expVal, obj.field("val"));
-        }
-    }
-
-    /**
-     *
-     */
-    @SuppressWarnings("PackageVisibleInnerClass")
-    static class PortableValueClosure extends TestProcessor {
-        /**
-         * @param expVal Expected value.
-         * @param acceptNull If {@code true} value can be null;
-         */
-        PortableValueClosure(String expVal, boolean acceptNull) {
-            super(expVal, acceptNull);
-        }
-
-        /** {@inheritDoc} */
-        @Override public void checkValue(Object val) {
-            PortableObject obj = (PortableObject)val;
-
-            assertEquals(expVal, obj.field("val"));
-        }
-    }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/71379a80/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/distributed/dht/GridCacheOffHeapTieredPortableSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/distributed/dht/GridCacheOffHeapTieredPortableSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/distributed/dht/GridCacheOffHeapTieredPortableSelfTest.java
deleted file mode 100644
index 6170e39..0000000
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/distributed/dht/GridCacheOffHeapTieredPortableSelfTest.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.processors.cache.portable.distributed.dht;
-
-import java.util.Arrays;
-import org.apache.ignite.configuration.IgniteConfiguration;
-import org.apache.ignite.internal.processors.cache.GridCacheOffHeapTieredSelfTest;
-import org.apache.ignite.marshaller.portable.PortableMarshaller;
-
-/**
- *
- */
-public class GridCacheOffHeapTieredPortableSelfTest extends GridCacheOffHeapTieredSelfTest {
-    /** {@inheritDoc} */
-    @Override protected boolean portableEnabled() {
-        return true;
-    }
-
-    /** {@inheritDoc} */
-    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
-        // Enable portables.
-        IgniteConfiguration cfg = super.getConfiguration(gridName);
-
-        PortableMarshaller marsh = new PortableMarshaller();
-
-        marsh.setClassNames(Arrays.asList(TestValue.class.getName()));
-
-        cfg.setMarshaller(marsh);
-
-        return cfg;
-    }
-}
\ No newline at end of file

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

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

http://git-wip-us.apache.org/repos/asf/ignite/blob/71379a80/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/distributed/dht/GridCachePortableObjectsAtomicNearDisabledOffheapTieredSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/distributed/dht/GridCachePortableObjectsAtomicNearDisabledOffheapTieredSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/distributed/dht/GridCachePortableObjectsAtomicNearDisabledOffheapTieredSelfTest.java
deleted file mode 100644
index a5c28f3..0000000
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/distributed/dht/GridCachePortableObjectsAtomicNearDisabledOffheapTieredSelfTest.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.processors.cache.portable.distributed.dht;
-
-/**
- *
- */
-public class GridCachePortableObjectsAtomicNearDisabledOffheapTieredSelfTest
-    extends GridCachePortableObjectsAtomicNearDisabledSelfTest {
-    /** {@inheritDoc} */
-    @Override protected boolean offheapTiered() {
-        return true;
-    }
-}
\ No newline at end of file

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

http://git-wip-us.apache.org/repos/asf/ignite/blob/71379a80/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/distributed/dht/GridCachePortableObjectsAtomicOffheapTieredSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/distributed/dht/GridCachePortableObjectsAtomicOffheapTieredSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/distributed/dht/GridCachePortableObjectsAtomicOffheapTieredSelfTest.java
deleted file mode 100644
index 8e04fa1..0000000
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/distributed/dht/GridCachePortableObjectsAtomicOffheapTieredSelfTest.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.processors.cache.portable.distributed.dht;
-
-/**
- *
- */
-public class GridCachePortableObjectsAtomicOffheapTieredSelfTest extends GridCachePortableObjectsAtomicSelfTest {
-    /** {@inheritDoc} */
-    @Override protected boolean offheapTiered() {
-        return true;
-    }
-}
\ No newline at end of file

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

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

http://git-wip-us.apache.org/repos/asf/ignite/blob/71379a80/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/distributed/dht/GridCachePortableObjectsPartitionedNearDisabledSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/distributed/dht/GridCachePortableObjectsPartitionedNearDisabledSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/distributed/dht/GridCachePortableObjectsPartitionedNearDisabledSelfTest.java
deleted file mode 100644
index df55de7..0000000
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/distributed/dht/GridCachePortableObjectsPartitionedNearDisabledSelfTest.java
+++ /dev/null
@@ -1,51 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.ignite.internal.processors.cache.portable.distributed.dht;
-
-import org.apache.ignite.cache.CacheAtomicityMode;
-import org.apache.ignite.cache.CacheMode;
-import org.apache.ignite.configuration.NearCacheConfiguration;
-import org.apache.ignite.internal.processors.cache.portable.GridCachePortableObjectsAbstractSelfTest;
-
-import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL;
-import static org.apache.ignite.cache.CacheMode.PARTITIONED;
-
-/**
- * Test for portable objects stored in cache.
- */
-public class GridCachePortableObjectsPartitionedNearDisabledSelfTest extends GridCachePortableObjectsAbstractSelfTest {
-    /** {@inheritDoc} */
-    @Override protected CacheMode cacheMode() {
-        return PARTITIONED;
-    }
-
-    /** {@inheritDoc} */
-    @Override protected CacheAtomicityMode atomicityMode() {
-        return TRANSACTIONAL;
-    }
-
-    /** {@inheritDoc} */
-    @Override protected NearCacheConfiguration nearConfiguration() {
-        return null;
-    }
-
-    /** {@inheritDoc} */
-    @Override protected int gridCount() {
-        return 3;
-    }
-}
\ No newline at end of file

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

http://git-wip-us.apache.org/repos/asf/ignite/blob/71379a80/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/distributed/dht/GridCachePortableObjectsPartitionedSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/distributed/dht/GridCachePortableObjectsPartitionedSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/distributed/dht/GridCachePortableObjectsPartitionedSelfTest.java
deleted file mode 100644
index 8c248be..0000000
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/distributed/dht/GridCachePortableObjectsPartitionedSelfTest.java
+++ /dev/null
@@ -1,51 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.ignite.internal.processors.cache.portable.distributed.dht;
-
-import org.apache.ignite.cache.CacheAtomicityMode;
-import org.apache.ignite.cache.CacheMode;
-import org.apache.ignite.configuration.NearCacheConfiguration;
-import org.apache.ignite.internal.processors.cache.portable.GridCachePortableObjectsAbstractSelfTest;
-
-import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL;
-import static org.apache.ignite.cache.CacheMode.PARTITIONED;
-
-/**
- * Test for portable objects stored in cache.
- */
-public class GridCachePortableObjectsPartitionedSelfTest extends GridCachePortableObjectsAbstractSelfTest {
-    /** {@inheritDoc} */
-    @Override protected CacheMode cacheMode() {
-        return PARTITIONED;
-    }
-
-    /** {@inheritDoc} */
-    @Override protected CacheAtomicityMode atomicityMode() {
-        return TRANSACTIONAL;
-    }
-
-    /** {@inheritDoc} */
-    @Override protected NearCacheConfiguration nearConfiguration() {
-        return new NearCacheConfiguration();
-    }
-
-    /** {@inheritDoc} */
-    @Override protected int gridCount() {
-        return 3;
-    }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/71379a80/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/distributed/dht/GridCachePortablesNearPartitionedByteArrayValuesSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/distributed/dht/GridCachePortablesNearPartitionedByteArrayValuesSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/distributed/dht/GridCachePortablesNearPartitionedByteArrayValuesSelfTest.java
deleted file mode 100644
index d984756..0000000
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/distributed/dht/GridCachePortablesNearPartitionedByteArrayValuesSelfTest.java
+++ /dev/null
@@ -1,41 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.ignite.internal.processors.cache.portable.distributed.dht;
-
-import org.apache.ignite.configuration.IgniteConfiguration;
-import org.apache.ignite.internal.processors.cache.distributed.near.GridCacheAbstractNearPartitionedByteArrayValuesSelfTest;
-import org.apache.ignite.marshaller.portable.PortableMarshaller;
-
-/**
- *
- */
-public class GridCachePortablesNearPartitionedByteArrayValuesSelfTest
-    extends GridCacheAbstractNearPartitionedByteArrayValuesSelfTest {
-    /** {@inheritDoc} */
-    @Override protected boolean peerClassLoading() {
-        return false;
-    }
-
-    /** {@inheritDoc} */
-    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
-        IgniteConfiguration cfg = super.getConfiguration(gridName);
-
-        cfg.setMarshaller(new PortableMarshaller());
-
-        return cfg;
-    }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/71379a80/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/distributed/dht/GridCachePortablesPartitionedOnlyByteArrayValuesSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/distributed/dht/GridCachePortablesPartitionedOnlyByteArrayValuesSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/distributed/dht/GridCachePortablesPartitionedOnlyByteArrayValuesSelfTest.java
deleted file mode 100644
index 5830b12..0000000
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/distributed/dht/GridCachePortablesPartitionedOnlyByteArrayValuesSelfTest.java
+++ /dev/null
@@ -1,42 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.ignite.internal.processors.cache.portable.distributed.dht;
-
-import org.apache.ignite.configuration.IgniteConfiguration;
-import org.apache.ignite.internal.processors.cache.distributed.dht.GridCacheAbstractPartitionedOnlyByteArrayValuesSelfTest;
-import org.apache.ignite.marshaller.portable.PortableMarshaller;
-
-/**
- *
- */
-public class GridCachePortablesPartitionedOnlyByteArrayValuesSelfTest
-    extends GridCacheAbstractPartitionedOnlyByteArrayValuesSelfTest {
-    /** {@inheritDoc} */
-    @Override protected boolean peerClassLoading() {
-        return false;
-    }
-
-    /** {@inheritDoc} */
-    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
-        IgniteConfiguration cfg = super.getConfiguration(gridName);
-
-        cfg.setMarshaller(new PortableMarshaller());
-
-        return cfg;
-    }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/71379a80/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/distributed/replicated/GridCachePortableObjectsReplicatedSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/distributed/replicated/GridCachePortableObjectsReplicatedSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/distributed/replicated/GridCachePortableObjectsReplicatedSelfTest.java
deleted file mode 100644
index 953fbfa..0000000
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/distributed/replicated/GridCachePortableObjectsReplicatedSelfTest.java
+++ /dev/null
@@ -1,51 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.ignite.internal.processors.cache.portable.distributed.replicated;
-
-import org.apache.ignite.cache.CacheAtomicityMode;
-import org.apache.ignite.cache.CacheMode;
-import org.apache.ignite.configuration.NearCacheConfiguration;
-import org.apache.ignite.internal.processors.cache.portable.GridCachePortableObjectsAbstractSelfTest;
-
-import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL;
-import static org.apache.ignite.cache.CacheMode.REPLICATED;
-
-/**
- * Test for portable objects stored in cache.
- */
-public class GridCachePortableObjectsReplicatedSelfTest extends GridCachePortableObjectsAbstractSelfTest {
-    /** {@inheritDoc} */
-    @Override protected CacheMode cacheMode() {
-        return REPLICATED;
-    }
-
-    /** {@inheritDoc} */
-    @Override protected CacheAtomicityMode atomicityMode() {
-        return TRANSACTIONAL;
-    }
-
-    /** {@inheritDoc} */
-    @Override protected NearCacheConfiguration nearConfiguration() {
-        return null;
-    }
-
-    /** {@inheritDoc} */
-    @Override protected int gridCount() {
-        return 3;
-    }
-}
\ No newline at end of file

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

http://git-wip-us.apache.org/repos/asf/ignite/blob/71379a80/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/local/GridCachePortableObjectsLocalOffheapTieredSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/local/GridCachePortableObjectsLocalOffheapTieredSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/local/GridCachePortableObjectsLocalOffheapTieredSelfTest.java
deleted file mode 100644
index 53713ce..0000000
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/local/GridCachePortableObjectsLocalOffheapTieredSelfTest.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.processors.cache.portable.local;
-
-/**
- *
- */
-public class GridCachePortableObjectsLocalOffheapTieredSelfTest extends GridCachePortableObjectsLocalSelfTest {
-    /** {@inheritDoc} */
-    @Override protected boolean offheapTiered() {
-        return true;
-    }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/71379a80/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/local/GridCachePortableObjectsLocalSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/local/GridCachePortableObjectsLocalSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/local/GridCachePortableObjectsLocalSelfTest.java
deleted file mode 100644
index 1a87865..0000000
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/local/GridCachePortableObjectsLocalSelfTest.java
+++ /dev/null
@@ -1,51 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.ignite.internal.processors.cache.portable.local;
-
-import org.apache.ignite.cache.CacheAtomicityMode;
-import org.apache.ignite.cache.CacheMode;
-import org.apache.ignite.configuration.NearCacheConfiguration;
-import org.apache.ignite.internal.processors.cache.portable.GridCachePortableObjectsAbstractSelfTest;
-
-import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL;
-import static org.apache.ignite.cache.CacheMode.LOCAL;
-
-/**
- * Test for portable objects stored in cache.
- */
-public class GridCachePortableObjectsLocalSelfTest extends GridCachePortableObjectsAbstractSelfTest {
-    /** {@inheritDoc} */
-    @Override protected CacheMode cacheMode() {
-        return LOCAL;
-    }
-
-    /** {@inheritDoc} */
-    @Override protected CacheAtomicityMode atomicityMode() {
-        return TRANSACTIONAL;
-    }
-
-    /** {@inheritDoc} */
-    @Override protected NearCacheConfiguration nearConfiguration() {
-        return null;
-    }
-
-    /** {@inheritDoc} */
-    @Override protected int gridCount() {
-        return 1;
-    }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/71379a80/modules/core/src/test/java/org/apache/ignite/testframework/junits/IgniteMock.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testframework/junits/IgniteMock.java b/modules/core/src/test/java/org/apache/ignite/testframework/junits/IgniteMock.java
index 1e4c828..964753d 100644
--- a/modules/core/src/test/java/org/apache/ignite/testframework/junits/IgniteMock.java
+++ b/modules/core/src/test/java/org/apache/ignite/testframework/junits/IgniteMock.java
@@ -35,7 +35,6 @@ import org.apache.ignite.IgniteEvents;
 import org.apache.ignite.IgniteFileSystem;
 import org.apache.ignite.IgniteLogger;
 import org.apache.ignite.IgniteMessaging;
-import org.apache.ignite.IgnitePortables;
 import org.apache.ignite.IgniteQueue;
 import org.apache.ignite.IgniteScheduler;
 import org.apache.ignite.IgniteServices;
@@ -271,11 +270,6 @@ public class IgniteMock implements Ignite {
     }
 
     /** {@inheritDoc} */
-    @Override public IgnitePortables portables() {
-        return null;
-    }
-
-    /** {@inheritDoc} */
     @Override public void close() {}
 
     @Nullable @Override public IgniteAtomicSequence atomicSequence(String name, long initVal, boolean create) {
@@ -339,4 +333,4 @@ public class IgniteMock implements Ignite {
     public void setStaticCfg(IgniteConfiguration staticCfg) {
         this.staticCfg = staticCfg;
     }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/71379a80/modules/core/src/test/java/org/apache/ignite/testframework/junits/multijvm/IgniteCacheProcessProxy.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testframework/junits/multijvm/IgniteCacheProcessProxy.java b/modules/core/src/test/java/org/apache/ignite/testframework/junits/multijvm/IgniteCacheProcessProxy.java
index 2b448f8..dfbb0ae 100644
--- a/modules/core/src/test/java/org/apache/ignite/testframework/junits/multijvm/IgniteCacheProcessProxy.java
+++ b/modules/core/src/test/java/org/apache/ignite/testframework/junits/multijvm/IgniteCacheProcessProxy.java
@@ -153,11 +153,6 @@ public class IgniteCacheProcessProxy<K, V> implements IgniteCache<K, V> {
     }
 
     /** {@inheritDoc} */
-    @Override public <K1, V1> IgniteCache<K1, V1> withKeepPortable() {
-        throw new UnsupportedOperationException("Method should be supported.");
-    }
-
-    /** {@inheritDoc} */
     @Override public void loadCache(@Nullable IgniteBiPredicate<K, V> p, @Nullable Object... args) throws CacheException {
         throw new UnsupportedOperationException("Method should be supported.");
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/71379a80/modules/core/src/test/java/org/apache/ignite/testframework/junits/multijvm/IgniteProcessProxy.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testframework/junits/multijvm/IgniteProcessProxy.java b/modules/core/src/test/java/org/apache/ignite/testframework/junits/multijvm/IgniteProcessProxy.java
index 3522407..ec7dab7 100644
--- a/modules/core/src/test/java/org/apache/ignite/testframework/junits/multijvm/IgniteProcessProxy.java
+++ b/modules/core/src/test/java/org/apache/ignite/testframework/junits/multijvm/IgniteProcessProxy.java
@@ -41,7 +41,7 @@ import org.apache.ignite.IgniteFileSystem;
 import org.apache.ignite.IgniteIllegalStateException;
 import org.apache.ignite.IgniteLogger;
 import org.apache.ignite.IgniteMessaging;
-import org.apache.ignite.IgnitePortables;
+import org.apache.ignite.internal.portable.api.IgnitePortables;
 import org.apache.ignite.IgniteQueue;
 import org.apache.ignite.IgniteScheduler;
 import org.apache.ignite.IgniteServices;

http://git-wip-us.apache.org/repos/asf/ignite/blob/71379a80/modules/core/src/test/java/org/apache/ignite/testsuites/IgnitePortableCacheFullApiTestSuite.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgnitePortableCacheFullApiTestSuite.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgnitePortableCacheFullApiTestSuite.java
deleted file mode 100644
index d7dda61..0000000
--- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgnitePortableCacheFullApiTestSuite.java
+++ /dev/null
@@ -1,37 +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.testsuites;
-
-import junit.framework.TestSuite;
-import org.apache.ignite.marshaller.portable.PortableMarshaller;
-import org.apache.ignite.testframework.config.GridTestProperties;
-
-/**
- * Cache full API suite with portable marshaller.
- */
-public class IgnitePortableCacheFullApiTestSuite extends TestSuite {
-    /**
-     * @return Suite.
-     * @throws Exception In case of error.
-     */
-    public static TestSuite suite() throws Exception {
-        GridTestProperties.setProperty(GridTestProperties.MARSH_CLASS_NAME, PortableMarshaller.class.getName());
-
-        return IgniteCacheFullApiSelfTestSuite.suite();
-    }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/71379a80/modules/core/src/test/java/org/apache/ignite/testsuites/IgnitePortableCacheTestSuite.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgnitePortableCacheTestSuite.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgnitePortableCacheTestSuite.java
deleted file mode 100644
index db20e48..0000000
--- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgnitePortableCacheTestSuite.java
+++ /dev/null
@@ -1,103 +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.testsuites;
-
-import java.util.HashSet;
-import junit.framework.TestSuite;
-import org.apache.ignite.internal.processors.cache.GridCacheAffinityRoutingSelfTest;
-import org.apache.ignite.internal.processors.cache.GridCacheEntryMemorySizeSelfTest;
-import org.apache.ignite.internal.processors.cache.GridCacheMvccSelfTest;
-import org.apache.ignite.internal.processors.cache.GridCacheOffHeapTieredAtomicSelfTest;
-import org.apache.ignite.internal.processors.cache.GridCacheOffHeapTieredEvictionAtomicSelfTest;
-import org.apache.ignite.internal.processors.cache.GridCacheOffHeapTieredEvictionSelfTest;
-import org.apache.ignite.internal.processors.cache.GridCacheOffHeapTieredSelfTest;
-import org.apache.ignite.internal.processors.cache.expiry.IgniteCacheAtomicLocalExpiryPolicyTest;
-import org.apache.ignite.internal.processors.cache.expiry.IgniteCacheExpiryPolicyTestSuite;
-import org.apache.ignite.internal.processors.cache.portable.GridPortableCacheEntryMemorySizeSelfTest;
-import org.apache.ignite.internal.processors.cache.portable.datastreaming.DataStreamProcessorPortableSelfTest;
-import org.apache.ignite.internal.processors.cache.portable.datastreaming.GridDataStreamerImplSelfTest;
-import org.apache.ignite.internal.processors.cache.portable.distributed.dht.GridCacheAffinityRoutingPortableSelfTest;
-import org.apache.ignite.internal.processors.cache.portable.distributed.dht.GridCacheAtomicPartitionedOnlyPortableDataStreamerMultiNodeSelfTest;
-import org.apache.ignite.internal.processors.cache.portable.distributed.dht.GridCacheAtomicPartitionedOnlyPortableDataStreamerMultithreadedSelfTest;
-import org.apache.ignite.internal.processors.cache.portable.distributed.dht.GridCacheAtomicPartitionedOnlyPortableMultiNodeSelfTest;
-import org.apache.ignite.internal.processors.cache.portable.distributed.dht.GridCacheAtomicPartitionedOnlyPortableMultithreadedSelfTest;
-import org.apache.ignite.internal.processors.cache.portable.distributed.dht.GridCacheMemoryModePortableSelfTest;
-import org.apache.ignite.internal.processors.cache.portable.distributed.dht.GridCacheOffHeapTieredAtomicPortableSelfTest;
-import org.apache.ignite.internal.processors.cache.portable.distributed.dht.GridCacheOffHeapTieredEvictionAtomicPortableSelfTest;
-import org.apache.ignite.internal.processors.cache.portable.distributed.dht.GridCacheOffHeapTieredEvictionPortableSelfTest;
-import org.apache.ignite.internal.processors.cache.portable.distributed.dht.GridCacheOffHeapTieredPortableSelfTest;
-import org.apache.ignite.internal.processors.cache.portable.distributed.dht.GridCachePortablesNearPartitionedByteArrayValuesSelfTest;
-import org.apache.ignite.internal.processors.cache.portable.distributed.dht.GridCachePortablesPartitionedOnlyByteArrayValuesSelfTest;
-import org.apache.ignite.internal.processors.datastreamer.DataStreamProcessorSelfTest;
-import org.apache.ignite.marshaller.portable.PortableMarshaller;
-import org.apache.ignite.testframework.config.GridTestProperties;
-
-/**
- * Cache suite with portable marshaller.
- */
-public class IgnitePortableCacheTestSuite extends TestSuite {
-    /**
-     * @return Suite.
-     * @throws Exception In case of error.
-     */
-    public static TestSuite suite() throws Exception {
-        GridTestProperties.setProperty(GridTestProperties.MARSH_CLASS_NAME, PortableMarshaller.class.getName());
-
-        TestSuite suite = new TestSuite("Portable Cache Test Suite");
-
-        HashSet<Class> ignoredTests = new HashSet<>();
-
-        // Tests below have a special version for Portable Marshaller
-        ignoredTests.add(DataStreamProcessorSelfTest.class);
-        ignoredTests.add(GridCacheOffHeapTieredEvictionAtomicSelfTest.class);
-        ignoredTests.add(GridCacheOffHeapTieredEvictionSelfTest.class);
-        ignoredTests.add(GridCacheOffHeapTieredSelfTest.class);
-        ignoredTests.add(GridCacheOffHeapTieredAtomicSelfTest.class);
-        ignoredTests.add(GridCacheAffinityRoutingSelfTest.class);
-        ignoredTests.add(IgniteCacheAtomicLocalExpiryPolicyTest.class);
-        ignoredTests.add(GridCacheEntryMemorySizeSelfTest.class);
-
-        // Tests that are not ready to be used with PortableMarshaller
-        ignoredTests.add(GridCacheMvccSelfTest.class);
-
-        suite.addTest(IgniteCacheTestSuite.suite(ignoredTests));
-        suite.addTest(IgniteCacheExpiryPolicyTestSuite.suite());
-
-        suite.addTestSuite(GridCacheMemoryModePortableSelfTest.class);
-        suite.addTestSuite(GridCacheOffHeapTieredEvictionAtomicPortableSelfTest.class);
-        suite.addTestSuite(GridCacheOffHeapTieredEvictionPortableSelfTest.class);
-
-        suite.addTestSuite(GridCachePortablesPartitionedOnlyByteArrayValuesSelfTest.class);
-        suite.addTestSuite(GridCachePortablesNearPartitionedByteArrayValuesSelfTest.class);
-        suite.addTestSuite(GridCacheOffHeapTieredPortableSelfTest.class);
-        suite.addTestSuite(GridCacheOffHeapTieredAtomicPortableSelfTest.class);
-
-        suite.addTestSuite(GridDataStreamerImplSelfTest.class);
-        suite.addTestSuite(DataStreamProcessorPortableSelfTest.class);
-        suite.addTestSuite(GridCacheAtomicPartitionedOnlyPortableDataStreamerMultiNodeSelfTest.class);
-        suite.addTestSuite(GridCacheAtomicPartitionedOnlyPortableDataStreamerMultithreadedSelfTest.class);
-
-        suite.addTestSuite(GridCacheAtomicPartitionedOnlyPortableMultiNodeSelfTest.class);
-        suite.addTestSuite(GridCacheAtomicPartitionedOnlyPortableMultithreadedSelfTest.class);
-
-        suite.addTestSuite(GridCacheAffinityRoutingPortableSelfTest.class);
-        suite.addTestSuite(GridPortableCacheEntryMemorySizeSelfTest.class);
-
-        return suite;
-    }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/71379a80/modules/core/src/test/java/org/apache/ignite/testsuites/IgnitePortableObjectsTestSuite.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgnitePortableObjectsTestSuite.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgnitePortableObjectsTestSuite.java
deleted file mode 100644
index ecd25e1..0000000
--- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgnitePortableObjectsTestSuite.java
+++ /dev/null
@@ -1,92 +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.testsuites;
-
-import junit.framework.TestSuite;
-import org.apache.ignite.internal.portable.GridPortableAffinityKeySelfTest;
-import org.apache.ignite.internal.portable.GridPortableBuilderAdditionalSelfTest;
-import org.apache.ignite.internal.portable.GridPortableBuilderSelfTest;
-import org.apache.ignite.internal.portable.GridPortableBuilderStringAsCharsAdditionalSelfTest;
-import org.apache.ignite.internal.portable.GridPortableBuilderStringAsCharsSelfTest;
-import org.apache.ignite.internal.portable.GridPortableMarshallerCtxDisabledSelfTest;
-import org.apache.ignite.internal.portable.GridPortableMarshallerSelfTest;
-import org.apache.ignite.internal.portable.GridPortableMetaDataDisabledSelfTest;
-import org.apache.ignite.internal.portable.GridPortableMetaDataSelfTest;
-import org.apache.ignite.internal.portable.GridPortableWildcardsSelfTest;
-import org.apache.ignite.internal.processors.cache.portable.GridCacheClientNodePortableMetadataMultinodeTest;
-import org.apache.ignite.internal.processors.cache.portable.GridCacheClientNodePortableMetadataTest;
-import org.apache.ignite.internal.processors.cache.portable.GridCachePortableStoreObjectsSelfTest;
-import org.apache.ignite.internal.processors.cache.portable.GridCachePortableStorePortablesSelfTest;
-import org.apache.ignite.internal.processors.cache.portable.distributed.dht.GridCachePortableObjectsAtomicNearDisabledOffheapTieredSelfTest;
-import org.apache.ignite.internal.processors.cache.portable.distributed.dht.GridCachePortableObjectsAtomicNearDisabledSelfTest;
-import org.apache.ignite.internal.processors.cache.portable.distributed.dht.GridCachePortableObjectsAtomicOffheapTieredSelfTest;
-import org.apache.ignite.internal.processors.cache.portable.distributed.dht.GridCachePortableObjectsAtomicSelfTest;
-import org.apache.ignite.internal.processors.cache.portable.distributed.dht.GridCachePortableObjectsPartitionedNearDisabledOffheapTieredSelfTest;
-import org.apache.ignite.internal.processors.cache.portable.distributed.dht.GridCachePortableObjectsPartitionedNearDisabledSelfTest;
-import org.apache.ignite.internal.processors.cache.portable.distributed.dht.GridCachePortableObjectsPartitionedOffheapTieredSelfTest;
-import org.apache.ignite.internal.processors.cache.portable.distributed.dht.GridCachePortableObjectsPartitionedSelfTest;
-import org.apache.ignite.internal.processors.cache.portable.distributed.replicated.GridCachePortableObjectsReplicatedSelfTest;
-import org.apache.ignite.internal.processors.cache.portable.local.GridCachePortableObjectsAtomicLocalSelfTest;
-import org.apache.ignite.internal.processors.cache.portable.local.GridCachePortableObjectsLocalOffheapTieredSelfTest;
-import org.apache.ignite.internal.processors.cache.portable.local.GridCachePortableObjectsLocalSelfTest;
-
-/**
- * Test for portable objects stored in cache.
- */
-public class IgnitePortableObjectsTestSuite extends TestSuite {
-    /**
-     * @return Suite.
-     * @throws Exception If failed.
-     */
-    public static TestSuite suite() throws Exception {
-        TestSuite suite = new TestSuite("GridGain Portable Objects Test Suite");
-
-        suite.addTestSuite(GridPortableMarshallerSelfTest.class);
-        suite.addTestSuite(GridPortableMarshallerCtxDisabledSelfTest.class);
-        suite.addTestSuite(GridPortableBuilderSelfTest.class);
-        suite.addTestSuite(GridPortableBuilderStringAsCharsSelfTest.class);
-        suite.addTestSuite(GridPortableMetaDataSelfTest.class);
-        suite.addTestSuite(GridPortableMetaDataDisabledSelfTest.class);
-        suite.addTestSuite(GridPortableAffinityKeySelfTest.class);
-        suite.addTestSuite(GridPortableWildcardsSelfTest.class);
-        suite.addTestSuite(GridPortableBuilderAdditionalSelfTest.class);
-        suite.addTestSuite(GridPortableBuilderStringAsCharsAdditionalSelfTest.class);
-
-        suite.addTestSuite(GridCachePortableObjectsLocalSelfTest.class);
-        suite.addTestSuite(GridCachePortableObjectsAtomicLocalSelfTest.class);
-        suite.addTestSuite(GridCachePortableObjectsReplicatedSelfTest.class);
-        suite.addTestSuite(GridCachePortableObjectsPartitionedSelfTest.class);
-        suite.addTestSuite(GridCachePortableObjectsPartitionedNearDisabledSelfTest.class);
-        suite.addTestSuite(GridCachePortableObjectsAtomicSelfTest.class);
-        suite.addTestSuite(GridCachePortableObjectsAtomicNearDisabledSelfTest.class);
-
-        suite.addTestSuite(GridCachePortableObjectsLocalOffheapTieredSelfTest.class);
-        suite.addTestSuite(GridCachePortableObjectsAtomicOffheapTieredSelfTest.class);
-        suite.addTestSuite(GridCachePortableObjectsAtomicNearDisabledOffheapTieredSelfTest.class);
-        suite.addTestSuite(GridCachePortableObjectsPartitionedOffheapTieredSelfTest.class);
-        suite.addTestSuite(GridCachePortableObjectsPartitionedNearDisabledOffheapTieredSelfTest.class);
-
-        suite.addTestSuite(GridCachePortableStoreObjectsSelfTest.class);
-        suite.addTestSuite(GridCachePortableStorePortablesSelfTest.class);
-
-        suite.addTestSuite(GridCacheClientNodePortableMetadataTest.class);
-        suite.addTestSuite(GridCacheClientNodePortableMetadataMultinodeTest.class);
-
-        return suite;
-    }
-}

http://git-wip-us.apache.org/repos/asf/ignite/blob/71379a80/modules/core/src/test/portables/repo/org/apache/ignite/portable/test1/1.1/test1-1.1.jar
----------------------------------------------------------------------
diff --git a/modules/core/src/test/portables/repo/org/apache/ignite/portable/test1/1.1/test1-1.1.jar b/modules/core/src/test/portables/repo/org/apache/ignite/portable/test1/1.1/test1-1.1.jar
deleted file mode 100644
index 863350d..0000000
Binary files a/modules/core/src/test/portables/repo/org/apache/ignite/portable/test1/1.1/test1-1.1.jar and /dev/null differ

http://git-wip-us.apache.org/repos/asf/ignite/blob/71379a80/modules/core/src/test/portables/repo/org/apache/ignite/portable/test1/1.1/test1-1.1.pom
----------------------------------------------------------------------
diff --git a/modules/core/src/test/portables/repo/org/apache/ignite/portable/test1/1.1/test1-1.1.pom b/modules/core/src/test/portables/repo/org/apache/ignite/portable/test1/1.1/test1-1.1.pom
deleted file mode 100644
index c79dfbf..0000000
--- a/modules/core/src/test/portables/repo/org/apache/ignite/portable/test1/1.1/test1-1.1.pom
+++ /dev/null
@@ -1,9 +0,0 @@
-<?xml version="1.0" encoding="UTF-8"?>
-<project xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd" xmlns="http://maven.apache.org/POM/4.0.0"
-    xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance">
-  <modelVersion>4.0.0</modelVersion>
-  <groupId>org.apache.ignite.portable</groupId>
-  <artifactId>test1</artifactId>
-  <version>1.1</version>
-  <description>POM was created from install:install-file</description>
-</project>

http://git-wip-us.apache.org/repos/asf/ignite/blob/71379a80/modules/core/src/test/portables/repo/org/apache/ignite/portable/test1/maven-metadata-local.xml
----------------------------------------------------------------------
diff --git a/modules/core/src/test/portables/repo/org/apache/ignite/portable/test1/maven-metadata-local.xml b/modules/core/src/test/portables/repo/org/apache/ignite/portable/test1/maven-metadata-local.xml
deleted file mode 100644
index 33f5abf..0000000
--- a/modules/core/src/test/portables/repo/org/apache/ignite/portable/test1/maven-metadata-local.xml
+++ /dev/null
@@ -1,12 +0,0 @@
-<?xml version="1.0" encoding="UTF-8"?>
-<metadata>
-  <groupId>org.apache.ignite.portable</groupId>
-  <artifactId>test1</artifactId>
-  <versioning>
-    <release>1.1</release>
-    <versions>
-      <version>1.1</version>
-    </versions>
-    <lastUpdated>20140806090184</lastUpdated>
-  </versioning>
-</metadata>

http://git-wip-us.apache.org/repos/asf/ignite/blob/71379a80/modules/core/src/test/portables/repo/org/apache/ignite/portable/test2/1.1/test2-1.1.jar
----------------------------------------------------------------------
diff --git a/modules/core/src/test/portables/repo/org/apache/ignite/portable/test2/1.1/test2-1.1.jar b/modules/core/src/test/portables/repo/org/apache/ignite/portable/test2/1.1/test2-1.1.jar
deleted file mode 100644
index ccf4ea2..0000000
Binary files a/modules/core/src/test/portables/repo/org/apache/ignite/portable/test2/1.1/test2-1.1.jar and /dev/null differ

http://git-wip-us.apache.org/repos/asf/ignite/blob/71379a80/modules/core/src/test/portables/repo/org/apache/ignite/portable/test2/1.1/test2-1.1.pom
----------------------------------------------------------------------
diff --git a/modules/core/src/test/portables/repo/org/apache/ignite/portable/test2/1.1/test2-1.1.pom b/modules/core/src/test/portables/repo/org/apache/ignite/portable/test2/1.1/test2-1.1.pom
deleted file mode 100644
index 37621e1..0000000
--- a/modules/core/src/test/portables/repo/org/apache/ignite/portable/test2/1.1/test2-1.1.pom
+++ /dev/null
@@ -1,9 +0,0 @@
-<?xml version="1.0" encoding="UTF-8"?>
-<project xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd" xmlns="http://maven.apache.org/POM/4.0.0"
-    xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance">
-  <modelVersion>4.0.0</modelVersion>
-  <groupId>org.apache.ignite.portable</groupId>
-  <artifactId>test2</artifactId>
-  <version>1.1</version>
-  <description>POM was created from install:install-file</description>
-</project>

http://git-wip-us.apache.org/repos/asf/ignite/blob/71379a80/modules/core/src/test/portables/repo/org/apache/ignite/portable/test2/maven-metadata-local.xml
----------------------------------------------------------------------
diff --git a/modules/core/src/test/portables/repo/org/apache/ignite/portable/test2/maven-metadata-local.xml b/modules/core/src/test/portables/repo/org/apache/ignite/portable/test2/maven-metadata-local.xml
deleted file mode 100644
index 9c705ef..0000000
--- a/modules/core/src/test/portables/repo/org/apache/ignite/portable/test2/maven-metadata-local.xml
+++ /dev/null
@@ -1,12 +0,0 @@
-<?xml version="1.0" encoding="UTF-8"?>
-<metadata>
-  <groupId>org.apache.ignite.portable</groupId>
-  <artifactId>test2</artifactId>
-  <versioning>
-    <release>1.1</release>
-    <versions>
-      <version>1.1</version>
-    </versions>
-    <lastUpdated>20140806090410</lastUpdated>
-  </versioning>
-</metadata>

http://git-wip-us.apache.org/repos/asf/ignite/blob/71379a80/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopDefaultMapReducePlannerSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopDefaultMapReducePlannerSelfTest.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopDefaultMapReducePlannerSelfTest.java
index 2bdf28c..833e49e 100644
--- a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopDefaultMapReducePlannerSelfTest.java
+++ b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopDefaultMapReducePlannerSelfTest.java
@@ -43,6 +43,7 @@ import org.apache.ignite.internal.GridKernalContext;
 import org.apache.ignite.internal.IgniteEx;
 import org.apache.ignite.internal.IgniteInternalFuture;
 import org.apache.ignite.internal.cluster.IgniteClusterEx;
+import org.apache.ignite.internal.portable.api.IgnitePortables;
 import org.apache.ignite.internal.processors.cache.GridCacheUtilityKey;
 import org.apache.ignite.internal.processors.cache.IgniteInternalCache;
 import org.apache.ignite.internal.processors.igfs.IgfsBlockLocationImpl;
@@ -1024,5 +1025,10 @@ public class HadoopDefaultMapReducePlannerSelfTest extends HadoopAbstractSelfTes
         @Override public GridKernalContext context() {
             return null;
         }
+
+        /** {@inheritDoc} */
+        @Override public IgnitePortables portables() {
+            return null;
+        }
     }
 }
\ No newline at end of file


[15/55] [abbrv] ignite git commit: IGNITE-1239 - Added test for reopened ticket.

Posted by ag...@apache.org.
IGNITE-1239 - Added test for reopened ticket.


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

Branch: refs/heads/ignite-1171
Commit: 866fb41525957555231fca11c5853731b9473170
Parents: 06fdd7d
Author: Alexey Goncharuk <al...@gmail.com>
Authored: Mon Sep 14 16:09:37 2015 -0700
Committer: Alexey Goncharuk <al...@gmail.com>
Committed: Mon Sep 14 16:09:37 2015 -0700

----------------------------------------------------------------------
 ...CacheScanPartitionQueryFallbackSelfTest.java | 105 ++++++++++++++++++-
 1 file changed, 104 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/866fb415/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/CacheScanPartitionQueryFallbackSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/CacheScanPartitionQueryFallbackSelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/CacheScanPartitionQueryFallbackSelfTest.java
index cb3a3bf..df310b4 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/CacheScanPartitionQueryFallbackSelfTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/CacheScanPartitionQueryFallbackSelfTest.java
@@ -26,13 +26,19 @@ import java.util.TreeSet;
 import java.util.UUID;
 import java.util.concurrent.Callable;
 import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.ThreadLocalRandom;
 import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.concurrent.atomic.AtomicInteger;
+import javax.cache.Cache;
 import org.apache.ignite.Ignite;
+import org.apache.ignite.IgniteCache;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.IgniteException;
 import org.apache.ignite.cache.CacheAtomicityMode;
 import org.apache.ignite.cache.CacheMode;
+import org.apache.ignite.cache.affinity.rendezvous.RendezvousAffinityFunction;
+import org.apache.ignite.cache.query.QueryCursor;
+import org.apache.ignite.cache.query.ScanQuery;
 import org.apache.ignite.cluster.ClusterNode;
 import org.apache.ignite.configuration.CacheConfiguration;
 import org.apache.ignite.configuration.IgniteConfiguration;
@@ -48,6 +54,7 @@ import org.apache.ignite.internal.processors.cache.query.CacheQueryFuture;
 import org.apache.ignite.internal.processors.cache.query.GridCacheQueryAdapter;
 import org.apache.ignite.internal.processors.cache.query.GridCacheQueryRequest;
 import org.apache.ignite.internal.util.typedef.F;
+import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.lang.IgniteBiTuple;
 import org.apache.ignite.lang.IgniteClosure;
 import org.apache.ignite.lang.IgniteInClosure;
@@ -67,7 +74,7 @@ public class CacheScanPartitionQueryFallbackSelfTest extends GridCommonAbstractT
     private static final int GRID_CNT = 3;
 
     /** Keys count. */
-    private static final int KEYS_CNT = 5000;
+    private static final int KEYS_CNT = 50 * RendezvousAffinityFunction.DFLT_PARTITION_COUNT;
 
     /** Ip finder. */
     private static final TcpDiscoveryVmIpFinder IP_FINDER = new TcpDiscoveryVmIpFinder(true);
@@ -261,6 +268,79 @@ public class CacheScanPartitionQueryFallbackSelfTest extends GridCommonAbstractT
     }
 
     /**
+     * Scan should activate fallback mechanism when new nodes join topology and rebalancing happens in parallel with
+     * scan query.
+     *
+     * @throws Exception In case of error.
+     */
+    public void testScanFallbackOnRebalancingCursor() throws Exception {
+        fail("https://issues.apache.org/jira/browse/IGNITE-1239");
+
+        cacheMode = CacheMode.PARTITIONED;
+        clientMode = false;
+        backups = 1;
+        commSpiFactory = new TestFallbackOnRebalancingCommunicationSpiFactory();
+
+        try {
+            Ignite ignite = startGrids(GRID_CNT);
+
+            fillCache(ignite);
+
+            final AtomicBoolean done = new AtomicBoolean(false);
+
+            IgniteInternalFuture fut1 = multithreadedAsync(
+                new Callable<Object>() {
+                    @Override public Object call() throws Exception {
+                        for (int i = 0; i < 5; i++) {
+                            startGrid(GRID_CNT + i);
+
+                            U.sleep(500);
+                        }
+
+                        done.set(true);
+
+                        return null;
+                    }
+                }, 1);
+
+            final AtomicInteger nodeIdx = new AtomicInteger();
+
+            IgniteInternalFuture fut2 = multithreadedAsync(
+                new Callable<Object>() {
+                    @Override public Object call() throws Exception {
+                        int nodeId = nodeIdx.getAndIncrement();
+
+                        IgniteCache<Integer, Integer> cache = grid(nodeId).cache(null);
+
+                        while (!done.get()) {
+                            int part = ThreadLocalRandom.current().nextInt(ignite(nodeId).affinity(null).partitions());
+
+                            try {
+                                QueryCursor<Cache.Entry<Integer, Integer>> cur =
+                                    cache.query(new ScanQuery<Integer, Integer>(part));
+
+                                U.debug(log, "Running query [node=" + nodeId + ", part=" + part + ']');
+
+                                doTestScanQueryCursor(cur, part);
+                            }
+                            catch (ClusterGroupEmptyCheckedException e) {
+                                log.warning("Invalid partition: " + part, e);
+                            }
+                        }
+
+                        return null;
+                    }
+                }, GRID_CNT);
+
+            fut1.get();
+            fut2.get();
+        }
+        finally {
+            stopAllGrids();
+        }
+    }
+
+    /**
      * Scan should try first remote node and fallbacks to second remote node.
      *
      * @throws Exception If failed.
@@ -391,6 +471,29 @@ public class CacheScanPartitionQueryFallbackSelfTest extends GridCommonAbstractT
     }
 
     /**
+     * @param cur Query cursor.
+     * @param part Partition number.
+     */
+    protected void doTestScanQueryCursor(
+        QueryCursor<Cache.Entry<Integer, Integer>> cur, int part) throws IgniteCheckedException {
+
+        Map<Integer, Integer> map = entries.get(part);
+
+        assert map != null;
+
+        int cnt = 0;
+
+        for (Cache.Entry<Integer, Integer> e : cur) {
+
+            assertEquals(map.get(e.getKey()), e.getValue());
+
+            cnt++;
+        }
+
+        assertEquals("Invalid number of entries for partition: " + part, map.size(), cnt);
+    }
+
+    /**
      * @param cctx Cctx.
      */
     private static int anyLocalPartition(GridCacheContext<?, ?> cctx) {


[33/55] [abbrv] ignite git commit: IGNITE-1482 - Fixed incorrect cache value for replace() on changing topology.

Posted by ag...@apache.org.
IGNITE-1482 - Fixed incorrect cache value for replace() on changing topology.


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

Branch: refs/heads/ignite-1171
Commit: 367d805d10ea071532fe99c6b67cfc97cc8f2fb9
Parents: 91dd7c1
Author: sboikov <sb...@gridgain.com>
Authored: Tue Sep 15 14:54:20 2015 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Tue Sep 15 14:54:20 2015 +0300

----------------------------------------------------------------------
 .../GridDistributedTxRemoteAdapter.java         |  8 +--
 .../distributed/dht/GridDhtTxPrepareFuture.java |  2 +-
 .../IgniteCacheEntryProcessorNodeJoinTest.java  | 73 ++++++++++++++++++++
 3 files changed, 78 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/367d805d/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedTxRemoteAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedTxRemoteAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedTxRemoteAdapter.java
index c930d88..f969737 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedTxRemoteAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedTxRemoteAdapter.java
@@ -521,7 +521,7 @@ public class GridDistributedTxRemoteAdapter extends IgniteTxAdapter
                                     if (updateNearCache(cacheCtx, txEntry.key(), topVer))
                                         nearCached = cacheCtx.dht().near().peekExx(txEntry.key());
 
-                                    if (!F.isEmpty(txEntry.entryProcessors()) || !F.isEmpty(txEntry.filters()))
+                                    if (!F.isEmpty(txEntry.entryProcessors()))
                                         txEntry.cached().unswap(false);
 
                                     IgniteBiTuple<GridCacheOperation, CacheObject> res =
@@ -573,12 +573,12 @@ public class GridDistributedTxRemoteAdapter extends IgniteTxAdapter
                                         // Invalidate only for near nodes (backups cannot be invalidated).
                                         if (isSystemInvalidate() || (isInvalidate() && cacheCtx.isNear()))
                                             cached.innerRemove(this, eventNodeId(), nodeId, false, false, true, true,
-                                                topVer, txEntry.filters(), replicate ? DR_BACKUP : DR_NONE,
+                                                topVer, null, replicate ? DR_BACKUP : DR_NONE,
                                                 near() ? null : explicitVer, CU.subjectId(this, cctx),
                                                 resolveTaskName());
                                         else {
                                             cached.innerSet(this, eventNodeId(), nodeId, val, false, false,
-                                                txEntry.ttl(), true, true, topVer, txEntry.filters(),
+                                                txEntry.ttl(), true, true, topVer, null,
                                                 replicate ? DR_BACKUP : DR_NONE, txEntry.conflictExpireTime(),
                                                 near() ? null : explicitVer, CU.subjectId(this, cctx),
                                                 resolveTaskName());
@@ -598,7 +598,7 @@ public class GridDistributedTxRemoteAdapter extends IgniteTxAdapter
                                     }
                                     else if (op == DELETE) {
                                         cached.innerRemove(this, eventNodeId(), nodeId, false, false, true, true,
-                                            topVer, txEntry.filters(), replicate ? DR_BACKUP : DR_NONE,
+                                            topVer, null, replicate ? DR_BACKUP : DR_NONE,
                                             near() ? null : explicitVer, CU.subjectId(this, cctx), resolveTaskName());
 
                                         // Keep near entry up to date.

http://git-wip-us.apache.org/repos/asf/ignite/blob/367d805d/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 89fc0ae..81cc272 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
@@ -842,7 +842,7 @@ public final class GridDhtTxPrepareFuture extends GridCompoundFuture<IgniteInter
         IgniteTxEntry e,
         Map<Integer, Collection<KeyCacheObject>> map
     ) {
-        if (retVal || !F.isEmpty(e.entryProcessors())) {
+        if (retVal || !F.isEmpty(e.entryProcessors()) || !F.isEmpty(e.filters())) {
             if (map == null)
                 map = new HashMap<>();
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/367d805d/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheEntryProcessorNodeJoinTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheEntryProcessorNodeJoinTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheEntryProcessorNodeJoinTest.java
index af9477e..6b4d473 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheEntryProcessorNodeJoinTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheEntryProcessorNodeJoinTest.java
@@ -30,7 +30,9 @@ import org.apache.ignite.IgniteCache;
 import org.apache.ignite.cache.CacheAtomicityMode;
 import org.apache.ignite.configuration.CacheConfiguration;
 import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.internal.IgniteEx;
 import org.apache.ignite.internal.IgniteInternalFuture;
+import org.apache.ignite.internal.IgniteKernal;
 import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.spi.communication.tcp.TcpCommunicationSpi;
 import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi;
@@ -201,6 +203,77 @@ public class IgniteCacheEntryProcessorNodeJoinTest extends GridCommonAbstractTes
         }
     }
 
+    /**
+     * @throws Exception If failed.
+     */
+    public void testReplaceNodeJoin() throws Exception {
+        final AtomicReference<Throwable> error = new AtomicReference<>();
+        final int started = 6;
+
+        try {
+            int keys = 100;
+
+            final AtomicBoolean done = new AtomicBoolean(false);
+
+            for (int i = 0; i < keys; i++)
+                ignite(0).cache(null).put(i, 0);
+
+            IgniteInternalFuture<Long> fut = GridTestUtils.runMultiThreadedAsync(new Runnable() {
+                @Override public void run() {
+                    try {
+                        for (int i = 0; i < started; i++) {
+                            U.sleep(1_000);
+
+                            IgniteEx grid = startGrid(GRID_CNT + i);
+
+                            info("Test started grid [idx=" + (GRID_CNT + i) + ", nodeId=" + grid.localNode().id() + ']');
+                        }
+                    }
+                    catch (Exception e) {
+                        error.compareAndSet(null, e);
+                    }
+                    finally {
+                        done.set(true);
+                    }
+                }
+            }, 1, "starter");
+
+            int updVal = 0;
+
+            try {
+                while (!done.get()) {
+                    info("Will put: " + (updVal + 1));
+
+                    for (int i = 0; i < keys; i++)
+                        assertTrue("Failed [key=" + i + ", oldVal=" + updVal+ ']',
+                            ignite(0).cache(null).replace(i, updVal, updVal + 1));
+
+                    updVal++;
+                }
+            }
+            finally {
+                fut.get(getTestTimeout());
+            }
+
+            for (int i = 0; i < keys; i++) {
+                for (int g = 0; g < GRID_CNT + started; g++) {
+                    Integer val = ignite(g).<Integer, Integer>cache(null).get(i);
+
+                    GridCacheEntryEx entry = ((IgniteKernal)grid(g)).internalCache(null).peekEx(i);
+
+                    if (updVal != val)
+                        info("Invalid value for grid [g=" + g + ", entry=" + entry + ']');
+
+                    assertEquals((Integer)updVal, val);
+                }
+            }
+        }
+        finally {
+            for (int i = 0; i < started; i++)
+                stopGrid(GRID_CNT + i);
+        }
+    }
+
     /** */
     private static class Processor implements EntryProcessor<String, Set<String>, Void>, Serializable {
         /** */


[39/55] [abbrv] ignite git commit: Quick fix to Hadoop IGFS tests NPE.

Posted by ag...@apache.org.
Quick fix to Hadoop IGFS tests NPE.


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

Branch: refs/heads/ignite-1171
Commit: adf5ef6100b58beb6fb00d85123f34cb5d9e084a
Parents: 2aa292c
Author: vozerov-gridgain <vo...@gridgain.com>
Authored: Tue Sep 15 16:23:10 2015 +0300
Committer: vozerov-gridgain <vo...@gridgain.com>
Committed: Tue Sep 15 16:23:10 2015 +0300

----------------------------------------------------------------------
 .../ignite/internal/processors/igfs/IgfsAbstractSelfTest.java     | 3 ++-
 1 file changed, 2 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/adf5ef61/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsAbstractSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsAbstractSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsAbstractSelfTest.java
index 957363f..076c269 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsAbstractSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsAbstractSelfTest.java
@@ -2686,6 +2686,7 @@ public abstract class IgfsAbstractSelfTest extends IgfsCommonAbstractTest {
     public static void clear(UniversalFileSystemAdapter uni) throws Exception {
         IgfsEx igfsEx = uni.getAdapter(IgfsEx.class);
 
-        clear(igfsEx);
+        if (igfsEx != null)
+            clear(igfsEx);
     }
 }
\ No newline at end of file


[08/55] [abbrv] ignite git commit: ignite-1462: hid portable API in 1.4 release

Posted by ag...@apache.org.
http://git-wip-us.apache.org/repos/asf/ignite/blob/71379a80/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/store/CacheOsStoreManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/store/CacheOsStoreManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/store/CacheOsStoreManager.java
index f7f5f4e..ef7dc9a 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/store/CacheOsStoreManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/store/CacheOsStoreManager.java
@@ -23,7 +23,7 @@ import org.apache.ignite.configuration.CacheConfiguration;
 import org.apache.ignite.internal.GridKernalContext;
 import org.apache.ignite.internal.processors.platform.PlatformProcessor;
 import org.apache.ignite.internal.processors.platform.cache.store.PlatformCacheStore;
-import org.apache.ignite.marshaller.portable.PortableMarshaller;
+import org.apache.ignite.internal.portable.api.PortableMarshaller;
 
 /**
  * Default store manager implementation.
@@ -82,6 +82,6 @@ public class CacheOsStoreManager extends GridCacheStoreManagerAdapter {
 
     /** {@inheritDoc} */
     @Override public boolean configuredConvertPortable() {
-        return !(ctx.config().getMarshaller() instanceof PortableMarshaller && cfg.isKeepPortableInStore());
+        return true;
     }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/71379a80/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/dotnet/PlatformDotNetConfiguration.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/dotnet/PlatformDotNetConfiguration.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/dotnet/PlatformDotNetConfiguration.java
index fac69fb..423b5e7 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/dotnet/PlatformDotNetConfiguration.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/dotnet/PlatformDotNetConfiguration.java
@@ -19,12 +19,12 @@ package org.apache.ignite.internal.processors.platform.dotnet;
 
 import org.apache.ignite.internal.processors.platform.PlatformConfiguration;
 import org.apache.ignite.internal.util.typedef.internal.S;
-import org.apache.ignite.portable.PortableException;
-import org.apache.ignite.portable.PortableMarshalAware;
-import org.apache.ignite.portable.PortableRawReader;
-import org.apache.ignite.portable.PortableRawWriter;
-import org.apache.ignite.portable.PortableReader;
-import org.apache.ignite.portable.PortableWriter;
+import org.apache.ignite.internal.portable.api.PortableException;
+import org.apache.ignite.internal.portable.api.PortableMarshalAware;
+import org.apache.ignite.internal.portable.api.PortableRawReader;
+import org.apache.ignite.internal.portable.api.PortableRawWriter;
+import org.apache.ignite.internal.portable.api.PortableReader;
+import org.apache.ignite.internal.portable.api.PortableWriter;
 
 import java.util.ArrayList;
 import java.util.List;

http://git-wip-us.apache.org/repos/asf/ignite/blob/71379a80/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/dotnet/PlatformDotNetPortableConfiguration.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/dotnet/PlatformDotNetPortableConfiguration.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/dotnet/PlatformDotNetPortableConfiguration.java
index a9b6022..92028b3 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/dotnet/PlatformDotNetPortableConfiguration.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/dotnet/PlatformDotNetPortableConfiguration.java
@@ -18,12 +18,12 @@
 package org.apache.ignite.internal.processors.platform.dotnet;
 
 import org.apache.ignite.internal.util.typedef.internal.S;
-import org.apache.ignite.portable.PortableException;
-import org.apache.ignite.portable.PortableMarshalAware;
-import org.apache.ignite.portable.PortableRawReader;
-import org.apache.ignite.portable.PortableRawWriter;
-import org.apache.ignite.portable.PortableReader;
-import org.apache.ignite.portable.PortableWriter;
+import org.apache.ignite.internal.portable.api.PortableException;
+import org.apache.ignite.internal.portable.api.PortableMarshalAware;
+import org.apache.ignite.internal.portable.api.PortableRawReader;
+import org.apache.ignite.internal.portable.api.PortableRawWriter;
+import org.apache.ignite.internal.portable.api.PortableReader;
+import org.apache.ignite.internal.portable.api.PortableWriter;
 
 import java.util.ArrayList;
 import java.util.Collection;

http://git-wip-us.apache.org/repos/asf/ignite/blob/71379a80/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/dotnet/PlatformDotNetPortableTypeConfiguration.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/dotnet/PlatformDotNetPortableTypeConfiguration.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/dotnet/PlatformDotNetPortableTypeConfiguration.java
index d7f1ab1..a307860 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/dotnet/PlatformDotNetPortableTypeConfiguration.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/dotnet/PlatformDotNetPortableTypeConfiguration.java
@@ -18,12 +18,12 @@
 package org.apache.ignite.internal.processors.platform.dotnet;
 
 import org.apache.ignite.internal.util.typedef.internal.S;
-import org.apache.ignite.portable.PortableException;
-import org.apache.ignite.portable.PortableMarshalAware;
-import org.apache.ignite.portable.PortableRawReader;
-import org.apache.ignite.portable.PortableRawWriter;
-import org.apache.ignite.portable.PortableReader;
-import org.apache.ignite.portable.PortableWriter;
+import org.apache.ignite.internal.portable.api.PortableException;
+import org.apache.ignite.internal.portable.api.PortableMarshalAware;
+import org.apache.ignite.internal.portable.api.PortableRawReader;
+import org.apache.ignite.internal.portable.api.PortableRawWriter;
+import org.apache.ignite.internal.portable.api.PortableReader;
+import org.apache.ignite.internal.portable.api.PortableWriter;
 import org.jetbrains.annotations.Nullable;
 
 /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/71379a80/modules/core/src/main/java/org/apache/ignite/marshaller/portable/PortableMarshaller.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/marshaller/portable/PortableMarshaller.java b/modules/core/src/main/java/org/apache/ignite/marshaller/portable/PortableMarshaller.java
deleted file mode 100644
index bfc34cd..0000000
--- a/modules/core/src/main/java/org/apache/ignite/marshaller/portable/PortableMarshaller.java
+++ /dev/null
@@ -1,358 +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.marshaller.portable;
-
-import java.io.ByteArrayOutputStream;
-import java.io.IOException;
-import java.io.InputStream;
-import java.io.OutputStream;
-import java.sql.Timestamp;
-import java.util.ArrayList;
-import java.util.Collection;
-import org.apache.ignite.IgniteCheckedException;
-import org.apache.ignite.internal.portable.GridPortableMarshaller;
-import org.apache.ignite.internal.portable.PortableContext;
-import org.apache.ignite.marshaller.AbstractMarshaller;
-import org.apache.ignite.marshaller.MarshallerContext;
-import org.apache.ignite.portable.PortableException;
-import org.apache.ignite.portable.PortableIdMapper;
-import org.apache.ignite.portable.PortableObject;
-import org.apache.ignite.portable.PortableProtocolVersion;
-import org.apache.ignite.portable.PortableSerializer;
-import org.apache.ignite.portable.PortableTypeConfiguration;
-import org.jetbrains.annotations.Nullable;
-
-/**
- * Implementation of {@link org.apache.ignite.marshaller.Marshaller} that lets to serialize and deserialize all objects
- * in the portable format.
- * <p>
- * {@code PortableMarshaller} is tested only on Java HotSpot VM on other VMs it could yield unexpected results.
- * <p>
- * <h1 class="header">Configuration</h1>
- * <h2 class="header">Mandatory</h2>
- * This marshaller has no mandatory configuration parameters.
- * <h2 class="header">Java Example</h2>
- * <pre name="code" class="java">
- * PortableMarshaller marshaller = new PortableMarshaller();
- *
- * IgniteConfiguration cfg = new IgniteConfiguration();
- *
- * // Override marshaller.
- * cfg.setMarshaller(marshaller);
- *
- * // Starts grid.
- * G.start(cfg);
- * </pre>
- * <h2 class="header">Spring Example</h2>
- * PortableMarshaller can be configured from Spring XML configuration file:
- * <pre name="code" class="xml">
- * &lt;bean id="grid.custom.cfg" class="org.apache.ignite.configuration.IgniteConfiguration" singleton="true"&gt;
- *     ...
- *     &lt;property name="marshaller"&gt;
- *         &lt;bean class="org.apache.ignite.marshaller.portable.PortableMarshaller"&gt;
- *            ...
- *         &lt;/bean&gt;
- *     &lt;/property&gt;
- *     ...
- * &lt;/bean&gt;
- * </pre>
- * <p>
- * <img src="http://ignite.apache.org/images/spring-small.png">
- * <br>
- * For information about Spring framework visit <a href="http://www.springframework.org/">www.springframework.org</a>
- */
-public class PortableMarshaller extends AbstractMarshaller {
-    /** Default portable protocol version. */
-    public static final PortableProtocolVersion DFLT_PORTABLE_PROTO_VER = PortableProtocolVersion.VER_1_4_0;
-
-    /** Class names. */
-    private Collection<String> clsNames;
-
-    /** ID mapper. */
-    private PortableIdMapper idMapper;
-
-    /** Serializer. */
-    private PortableSerializer serializer;
-
-    /** Types. */
-    private Collection<PortableTypeConfiguration> typeCfgs;
-
-    /** Use timestamp flag. */
-    private boolean useTs = true;
-
-    /** Whether to convert string to bytes using UTF-8 encoding. */
-    private boolean convertString = true;
-
-    /** Meta data enabled flag. */
-    private boolean metaDataEnabled = true;
-
-    /** Keep deserialized flag. */
-    private boolean keepDeserialized = true;
-
-    /** Protocol version. */
-    private PortableProtocolVersion protoVer = DFLT_PORTABLE_PROTO_VER;
-
-    /** */
-    private GridPortableMarshaller impl;
-
-    /**
-     * Gets class names.
-     *
-     * @return Class names.
-     */
-    public Collection<String> getClassNames() {
-        return clsNames;
-    }
-
-    /**
-     * Sets class names of portable objects explicitly.
-     *
-     * @param clsNames Class names.
-     */
-    public void setClassNames(Collection<String> clsNames) {
-        this.clsNames = new ArrayList<>(clsNames.size());
-
-        for (String clsName : clsNames)
-            this.clsNames.add(clsName.trim());
-    }
-
-    /**
-     * Gets ID mapper.
-     *
-     * @return ID mapper.
-     */
-    public PortableIdMapper getIdMapper() {
-        return idMapper;
-    }
-
-    /**
-     * Sets ID mapper.
-     *
-     * @param idMapper ID mapper.
-     */
-    public void setIdMapper(PortableIdMapper idMapper) {
-        this.idMapper = idMapper;
-    }
-
-    /**
-     * Gets serializer.
-     *
-     * @return Serializer.
-     */
-    public PortableSerializer getSerializer() {
-        return serializer;
-    }
-
-    /**
-     * Sets serializer.
-     *
-     * @param serializer Serializer.
-     */
-    public void setSerializer(PortableSerializer serializer) {
-        this.serializer = serializer;
-    }
-
-    /**
-     * Gets types configuration.
-     *
-     * @return Types configuration.
-     */
-    public Collection<PortableTypeConfiguration> getTypeConfigurations() {
-        return typeCfgs;
-    }
-
-    /**
-     * Sets type configurations.
-     *
-     * @param typeCfgs Type configurations.
-     */
-    public void setTypeConfigurations(Collection<PortableTypeConfiguration> typeCfgs) {
-        this.typeCfgs = typeCfgs;
-    }
-
-    /**
-     * If {@code true} then date values converted to {@link Timestamp} on deserialization.
-     * <p>
-     * Default value is {@code true}.
-     *
-     * @return Flag indicating whether date values converted to {@link Timestamp} during unmarshalling.
-     */
-    public boolean isUseTimestamp() {
-        return useTs;
-    }
-
-    /**
-     * @param useTs Flag indicating whether date values converted to {@link Timestamp} during unmarshalling.
-     */
-    public void setUseTimestamp(boolean useTs) {
-        this.useTs = useTs;
-    }
-
-    /**
-     * Gets strings must be converted to or from bytes using UTF-8 encoding.
-     * <p>
-     * Default value is {@code true}.
-     *
-     * @return Flag indicating whether string must be converted to byte array using UTF-8 encoding.
-     */
-    public boolean isConvertStringToBytes() {
-        return convertString;
-    }
-
-    /**
-     * Sets strings must be converted to or from bytes using UTF-8 encoding.
-     * <p>
-     * Default value is {@code true}.
-     *
-     * @param convertString Flag indicating whether string must be converted to byte array using UTF-8 encoding.
-     */
-    public void setConvertStringToBytes(boolean convertString) {
-        this.convertString = convertString;
-    }
-
-    /**
-     * If {@code true}, meta data will be collected or all types. If you need to override this behaviour for
-     * some specific type, use {@link PortableTypeConfiguration#setMetaDataEnabled(Boolean)} method.
-     * <p>
-     * Default value if {@code true}.
-     *
-     * @return Whether meta data is collected.
-     */
-    public boolean isMetaDataEnabled() {
-        return metaDataEnabled;
-    }
-
-    /**
-     * @param metaDataEnabled Whether meta data is collected.
-     */
-    public void setMetaDataEnabled(boolean metaDataEnabled) {
-        this.metaDataEnabled = metaDataEnabled;
-    }
-
-    /**
-     * If {@code true}, {@link PortableObject} will cache deserialized instance after
-     * {@link PortableObject#deserialize()} is called. All consequent calls of this
-     * method on the same instance of {@link PortableObject} will return that cached
-     * value without actually deserializing portable object. If you need to override this
-     * behaviour for some specific type, use {@link PortableTypeConfiguration#setKeepDeserialized(Boolean)}
-     * method.
-     * <p>
-     * Default value if {@code true}.
-     *
-     * @return Whether deserialized value is kept.
-     */
-    public boolean isKeepDeserialized() {
-        return keepDeserialized;
-    }
-
-    /**
-     * @param keepDeserialized Whether deserialized value is kept.
-     */
-    public void setKeepDeserialized(boolean keepDeserialized) {
-        this.keepDeserialized = keepDeserialized;
-    }
-
-    /**
-     * Gets portable protocol version.
-     * <p>
-     * Defaults to {@link #DFLT_PORTABLE_PROTO_VER}.
-     *
-     * @return Portable protocol version.
-     */
-    public PortableProtocolVersion getProtocolVersion() {
-        return protoVer;
-    }
-
-    /**
-     * Sets portable protocol version.
-     * <p>
-     * Defaults to {@link #DFLT_PORTABLE_PROTO_VER}.
-     *
-     * @param protoVer Portable protocol version.
-     */
-    public void setProtocolVersion(PortableProtocolVersion protoVer) {
-        if (protoVer == null)
-            throw new IllegalArgumentException("Wrong portable protocol version: " + protoVer);
-
-        this.protoVer = protoVer;
-    }
-
-    /**
-     * Returns currently set {@link MarshallerContext}.
-     *
-     * @return Marshaller context.
-     */
-    public MarshallerContext getContext() {
-        return ctx;
-    }
-
-    /**
-     * Sets {@link PortableContext}.
-     * <p/>
-     * @param ctx Portable context.
-     */
-    private void setPortableContext(PortableContext ctx) {
-        ctx.configure(this);
-
-        impl = new GridPortableMarshaller(ctx);
-    }
-
-    /** {@inheritDoc} */
-    @Override public byte[] marshal(@Nullable Object obj) throws IgniteCheckedException {
-        return impl.marshal(obj, 0);
-    }
-
-    /** {@inheritDoc} */
-    @Override public void marshal(@Nullable Object obj, OutputStream out) throws IgniteCheckedException {
-        byte[] arr = marshal(obj);
-
-        try {
-            out.write(arr);
-        }
-        catch (IOException e) {
-            throw new PortableException("Failed to marshal the object: " + obj, e);
-        }
-    }
-
-    /** {@inheritDoc} */
-    @Override public <T> T unmarshal(byte[] bytes, @Nullable ClassLoader clsLdr) throws IgniteCheckedException {
-        return impl.deserialize(bytes, clsLdr);
-    }
-
-    /** {@inheritDoc} */
-    @Override public <T> T unmarshal(InputStream in, @Nullable ClassLoader clsLdr) throws IgniteCheckedException {
-        ByteArrayOutputStream buffer = new ByteArrayOutputStream();
-
-        byte[] arr = new byte[4096];
-        int cnt;
-
-        // we have to fully read the InputStream because GridPortableMarshaller requires support of a method that
-        // returns number of bytes remaining.
-        try {
-            while ((cnt = in.read(arr)) != -1)
-                buffer.write(arr, 0, cnt);
-
-            buffer.flush();
-
-            return impl.deserialize(buffer.toByteArray(), clsLdr);
-        }
-        catch (IOException e) {
-            throw new PortableException("Failed to unmarshal the object from InputStream", e);
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/ignite/blob/71379a80/modules/core/src/main/java/org/apache/ignite/marshaller/portable/package-info.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/marshaller/portable/package-info.java b/modules/core/src/main/java/org/apache/ignite/marshaller/portable/package-info.java
deleted file mode 100644
index 90cc5e6..0000000
--- a/modules/core/src/main/java/org/apache/ignite/marshaller/portable/package-info.java
+++ /dev/null
@@ -1,22 +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 description. -->
- * Contains portable marshaller API classes.
- */
-package org.apache.ignite.marshaller.portable;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/71379a80/modules/core/src/main/java/org/apache/ignite/portable/PortableBuilder.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/portable/PortableBuilder.java b/modules/core/src/main/java/org/apache/ignite/portable/PortableBuilder.java
deleted file mode 100644
index 377fcdc..0000000
--- a/modules/core/src/main/java/org/apache/ignite/portable/PortableBuilder.java
+++ /dev/null
@@ -1,137 +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.portable;
-
-import org.apache.ignite.IgnitePortables;
-import org.jetbrains.annotations.Nullable;
-
-/**
- * Portable object builder. Provides ability to build portable objects dynamically without having class definitions.
- * <p>
- * Here is an example of how a portable object can be built dynamically:
- * <pre name=code class=java>
- * PortableBuilder builder = Ignition.ignite().portables().builder("org.project.MyObject");
- *
- * builder.setField("fieldA", "A");
- * builder.setField("fieldB", "B");
- *
- * PortableObject portableObj = builder.build();
- * </pre>
- *
- * <p>
- * Also builder can be initialized by existing portable object. This allows changing some fields without affecting
- * other fields.
- * <pre name=code class=java>
- * PortableBuilder builder = Ignition.ignite().portables().builder(person);
- *
- * builder.setField("name", "John");
- *
- * person = builder.build();
- * </pre>
- * </p>
- *
- * If you need to modify nested portable object you can get builder for nested object using
- * {@link #getField(String)}, changes made on nested builder will affect parent object,
- * for example:
- *
- * <pre name=code class=java>
- * PortableBuilder personBuilder = grid.portables().createBuilder(personPortableObj);
- * PortableBuilder addressBuilder = personBuilder.setField("address");
- *
- * addressBuilder.setField("city", "New York");
- *
- * personPortableObj = personBuilder.build();
- *
- * // Should be "New York".
- * String city = personPortableObj.getField("address").getField("city");
- * </pre>
- *
- * @see IgnitePortables#builder(int)
- * @see IgnitePortables#builder(String)
- * @see IgnitePortables#builder(PortableObject)
- */
-public interface PortableBuilder {
-    /**
-     * Returns value assigned to the specified field.
-     * If the value is a portable object instance of {@code GridPortableBuilder} will be returned,
-     * which can be modified.
-     * <p>
-     * Collections and maps returned from this method are modifiable.
-     *
-     * @param name Field name.
-     * @return Filed value.
-     */
-    public <T> T getField(String name);
-
-    /**
-     * Sets field value.
-     *
-     * @param name Field name.
-     * @param val Field value (cannot be {@code null}).
-     * @see PortableObject#metaData()
-     */
-    public PortableBuilder setField(String name, Object val);
-
-    /**
-     * Sets field value with value type specification.
-     * <p>
-     * Field type is needed for proper metadata update.
-     *
-     * @param name Field name.
-     * @param val Field value.
-     * @param type Field type.
-     * @see PortableObject#metaData()
-     */
-    public <T> PortableBuilder setField(String name, @Nullable T val, Class<? super T> type);
-
-    /**
-     * Sets field value.
-     * <p>
-     * This method should be used if field is portable object.
-     *
-     * @param name Field name.
-     * @param builder Builder for object field.
-     */
-    public PortableBuilder setField(String name, @Nullable PortableBuilder builder);
-
-    /**
-     * Removes field from this builder.
-     *
-     * @param fieldName Field name.
-     * @return {@code this} instance for chaining.
-     */
-    public PortableBuilder removeField(String fieldName);
-
-    /**
-     * Sets hash code for resulting portable object returned by {@link #build()} method.
-     * <p>
-     * If not set {@code 0} is used.
-     *
-     * @param hashCode Hash code.
-     * @return {@code this} instance for chaining.
-     */
-    public PortableBuilder hashCode(int hashCode);
-
-    /**
-     * Builds portable object.
-     *
-     * @return Portable object.
-     * @throws PortableException In case of error.
-     */
-    public PortableObject build() throws PortableException;
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/71379a80/modules/core/src/main/java/org/apache/ignite/portable/PortableException.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/portable/PortableException.java b/modules/core/src/main/java/org/apache/ignite/portable/PortableException.java
deleted file mode 100644
index 0f8d78b..0000000
--- a/modules/core/src/main/java/org/apache/ignite/portable/PortableException.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.portable;
-
-import org.apache.ignite.IgniteException;
-import org.jetbrains.annotations.Nullable;
-
-/**
- * Exception indicating portable object serialization error.
- */
-public class PortableException extends IgniteException {
-    /** */
-    private static final long serialVersionUID = 0L;
-
-    /**
-     * Creates portable exception with error message.
-     *
-     * @param msg Error message.
-     */
-    public PortableException(String msg) {
-        super(msg);
-    }
-
-    /**
-     * Creates portable exception with {@link Throwable} as a cause.
-     *
-     * @param cause Cause.
-     */
-    public PortableException(Throwable cause) {
-        super(cause);
-    }
-
-    /**
-     * Creates portable exception with error message and {@link Throwable} as a cause.
-     *
-     * @param msg Error message.
-     * @param cause Cause.
-     */
-    public PortableException(String msg, @Nullable Throwable cause) {
-        super(msg, cause);
-    }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/71379a80/modules/core/src/main/java/org/apache/ignite/portable/PortableIdMapper.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/portable/PortableIdMapper.java b/modules/core/src/main/java/org/apache/ignite/portable/PortableIdMapper.java
deleted file mode 100644
index 368e415..0000000
--- a/modules/core/src/main/java/org/apache/ignite/portable/PortableIdMapper.java
+++ /dev/null
@@ -1,56 +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.portable;
-
-import org.apache.ignite.marshaller.portable.PortableMarshaller;
-
-/**
- * Type and field ID mapper for portable objects. Ignite never writes full
- * strings for field or type names. Instead, for performance reasons, Ignite
- * writes integer hash codes for type and field names. It has been tested that
- * hash code conflicts for the type names or the field names
- * within the same type are virtually non-existent and, to gain performance, it is safe
- * to work with hash codes. For the cases when hash codes for different types or fields
- * actually do collide {@code PortableIdMapper} allows to override the automatically
- * generated hash code IDs for the type and field names.
- * <p>
- * Portable ID mapper can be configured for all portable objects via {@link PortableMarshaller#getIdMapper()} method,
- * or for a specific portable type via {@link PortableTypeConfiguration#getIdMapper()} method.
- */
-public interface PortableIdMapper {
-    /**
-     * Gets type ID for provided class name.
-     * <p>
-     * If {@code 0} is returned, hash code of class simple name will be used.
-     *
-     * @param clsName Class name.
-     * @return Type ID.
-     */
-    public int typeId(String clsName);
-
-    /**
-     * Gets ID for provided field.
-     * <p>
-     * If {@code 0} is returned, hash code of field name will be used.
-     *
-     * @param typeId Type ID.
-     * @param fieldName Field name.
-     * @return Field ID.
-     */
-    public int fieldId(int typeId, String fieldName);
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/71379a80/modules/core/src/main/java/org/apache/ignite/portable/PortableInvalidClassException.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/portable/PortableInvalidClassException.java b/modules/core/src/main/java/org/apache/ignite/portable/PortableInvalidClassException.java
deleted file mode 100644
index 0098ec3..0000000
--- a/modules/core/src/main/java/org/apache/ignite/portable/PortableInvalidClassException.java
+++ /dev/null
@@ -1,58 +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.portable;
-
-import org.jetbrains.annotations.Nullable;
-
-/**
- * Exception indicating that class needed for deserialization of portable object does not exist.
- * <p>
- * Thrown from {@link PortableObject#deserialize()} method.
- */
-public class PortableInvalidClassException extends PortableException {
-    /** */
-    private static final long serialVersionUID = 0L;
-
-    /**
-     * Creates invalid class exception with error message.
-     *
-     * @param msg Error message.
-     */
-    public PortableInvalidClassException(String msg) {
-        super(msg);
-    }
-
-    /**
-     * Creates invalid class exception with {@link Throwable} as a cause.
-     *
-     * @param cause Cause.
-     */
-    public PortableInvalidClassException(Throwable cause) {
-        super(cause);
-    }
-
-    /**
-     * Creates invalid class exception with error message and {@link Throwable} as a cause.
-     *
-     * @param msg Error message.
-     * @param cause Cause.
-     */
-    public PortableInvalidClassException(String msg, @Nullable Throwable cause) {
-        super(msg, cause);
-    }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/71379a80/modules/core/src/main/java/org/apache/ignite/portable/PortableMarshalAware.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/portable/PortableMarshalAware.java b/modules/core/src/main/java/org/apache/ignite/portable/PortableMarshalAware.java
deleted file mode 100644
index 4270885..0000000
--- a/modules/core/src/main/java/org/apache/ignite/portable/PortableMarshalAware.java
+++ /dev/null
@@ -1,48 +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.portable;
-
-/**
- * Interface that allows to implement custom serialization
- * logic for portable objects. Portable objects are not required
- * to implement this interface, in which case Ignite will automatically
- * serialize portable objects using reflection.
- * <p>
- * This interface, in a way, is analogous to {@link java.io.Externalizable}
- * interface, which allows users to override default serialization logic,
- * usually for performance reasons. The only difference here is that portable
- * serialization is already very fast and implementing custom serialization
- * logic for portables does not provide significant performance gains.
- */
-public interface PortableMarshalAware {
-    /**
-     * Writes fields to provided writer.
-     *
-     * @param writer Portable object writer.
-     * @throws PortableException In case of error.
-     */
-    public void writePortable(PortableWriter writer) throws PortableException;
-
-    /**
-     * Reads fields from provided reader.
-     *
-     * @param reader Portable object reader.
-     * @throws PortableException In case of error.
-     */
-    public void readPortable(PortableReader reader) throws PortableException;
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/71379a80/modules/core/src/main/java/org/apache/ignite/portable/PortableMetadata.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/portable/PortableMetadata.java b/modules/core/src/main/java/org/apache/ignite/portable/PortableMetadata.java
deleted file mode 100644
index 4ea808b..0000000
--- a/modules/core/src/main/java/org/apache/ignite/portable/PortableMetadata.java
+++ /dev/null
@@ -1,61 +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.portable;
-
-import java.util.Collection;
-import org.apache.ignite.IgnitePortables;
-import org.jetbrains.annotations.Nullable;
-
-/**
- * Portable type meta data. Metadata for portable types can be accessed from any of the
- * {@link IgnitePortables#metadata(String)} methods.
- * Having metadata also allows for proper formatting of {@code PortableObject#toString()} method,
- * even when portable objects are kept in binary format only, which may be necessary for audit reasons.
- */
-public interface PortableMetadata {
-    /**
-     * Gets portable type name.
-     *
-     * @return Portable type name.
-     */
-    public String typeName();
-
-    /**
-     * Gets collection of all field names for this portable type.
-     *
-     * @return Collection of all field names for this portable type.
-     */
-    public Collection<String> fields();
-
-    /**
-     * Gets name of the field type for a given field.
-     *
-     * @param fieldName Field name.
-     * @return Field type name.
-     */
-    @Nullable public String fieldTypeName(String fieldName);
-
-    /**
-     * Portable objects can optionally specify custom key-affinity mapping in the
-     * configuration. This method returns the name of the field which should be
-     * used for the key-affinity mapping.
-     *
-     * @return Affinity key field name.
-     */
-    @Nullable public String affinityKeyFieldName();
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/71379a80/modules/core/src/main/java/org/apache/ignite/portable/PortableObject.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/portable/PortableObject.java b/modules/core/src/main/java/org/apache/ignite/portable/PortableObject.java
deleted file mode 100644
index 66b8f76..0000000
--- a/modules/core/src/main/java/org/apache/ignite/portable/PortableObject.java
+++ /dev/null
@@ -1,154 +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.portable;
-
-import java.io.Serializable;
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.LinkedList;
-import java.util.TreeMap;
-import org.apache.ignite.IgnitePortables;
-import org.apache.ignite.marshaller.portable.PortableMarshaller;
-import org.jetbrains.annotations.Nullable;
-
-/**
- * Wrapper for portable object in portable binary format. Once an object is defined as portable,
- * Ignite will always store it in memory in the portable (i.e. binary) format.
- * User can choose to work either with the portable format or with the deserialized form
- * (assuming that class definitions are present in the classpath).
- * <p>
- * <b>NOTE:</b> user does not need to (and should not) implement this interface directly.
- * <p>
- * To work with the portable format directly, user should create a cache projection
- * over {@code PortableObject} class and then retrieve individual fields as needed:
- * <pre name=code class=java>
- * IgniteCache&lt;PortableObject, PortableObject&gt; prj = cache.withKeepPortable();
- *
- * // Convert instance of MyKey to portable format.
- * // We could also use GridPortableBuilder to create the key in portable format directly.
- * PortableObject key = grid.portables().toPortable(new MyKey());
- *
- * PortableObject val = prj.get(key);
- *
- * String field = val.field("myFieldName");
- * </pre>
- * Alternatively, if we have class definitions in the classpath, we may choose to work with deserialized
- * typed objects at all times. In this case we do incur the deserialization cost. However, if
- * {@link PortableMarshaller#isKeepDeserialized()} is {@code true} then Ignite will only deserialize on the first access
- * and will cache the deserialized object, so it does not have to be deserialized again:
- * <pre name=code class=java>
- * IgniteCache&lt;MyKey.class, MyValue.class&gt; cache = grid.cache(null);
- *
- * MyValue val = cache.get(new MyKey());
- *
- * // Normal java getter.
- * String fieldVal = val.getMyFieldName();
- * </pre>
- * <h1 class="header">Working With Maps and Collections</h1>
- * All maps and collections in the portable objects are serialized automatically. When working
- * with different platforms, e.g. C++ or .NET, Ignite will automatically pick the most
- * adequate collection or map in either language. For example, {@link ArrayList} in Java will become
- * {@code List} in C#, {@link LinkedList} in Java is {@link LinkedList} in C#, {@link HashMap}
- * in Java is {@code Dictionary} in C#, and {@link TreeMap} in Java becomes {@code SortedDictionary}
- * in C#, etc.
- * <h1 class="header">Dynamic Structure Changes</h1>
- * Since objects are always cached in the portable binary format, server does not need to
- * be aware of the class definitions. Moreover, if class definitions are not present or not
- * used on the server, then clients can continuously change the structure of the portable
- * objects without having to restart the cluster. For example, if one client stores a
- * certain class with fields A and B, and another client stores the same class with
- * fields B and C, then the server-side portable object will have the fields A, B, and C.
- * As the structure of a portable object changes, the new fields become available for SQL queries
- * automatically.
- * <h1 class="header">Building Portable Objects</h1>
- * Ignite comes with {@link PortableBuilder} which allows to build portable objects dynamically:
- * <pre name=code class=java>
- * PortableBuilder builder = Ignition.ignite().portables().builder("org.project.MyObject");
- *
- * builder.setField("fieldA", "A");
- * builder.setField("fieldB", "B");
- *
- * PortableObject portableObj = builder.build();
- * </pre>
- * For the cases when class definition is present
- * in the class path, it is also possible to populate a standard POJO and then
- * convert it to portable format, like so:
- * <pre name=code class=java>
- * MyObject obj = new MyObject();
- *
- * obj.setFieldA("A");
- * obj.setFieldB(123);
- *
- * PortableObject portableObj = Ignition.ignite().portables().toPortable(obj);
- * </pre>
- * <h1 class="header">Portable Metadata</h1>
- * Even though Ignite portable protocol only works with hash codes for type and field names
- * to achieve better performance, Ignite provides metadata for all portable types which
- * can be queried ar runtime via any of the {@link IgnitePortables#metadata(Class)}
- * methods. Having metadata also allows for proper formatting of {@code PortableObject.toString()} method,
- * even when portable objects are kept in binary format only, which may be necessary for audit reasons.
- */
-public interface PortableObject extends Serializable, Cloneable {
-    /**
-     * Gets portable object type ID.
-     *
-     * @return Type ID.
-     */
-    public int typeId();
-
-    /**
-     * Gets meta data for this portable object.
-     *
-     * @return Meta data.
-     * @throws PortableException In case of error.
-     */
-    @Nullable public PortableMetadata metaData() throws PortableException;
-
-    /**
-     * Gets field value.
-     *
-     * @param fieldName Field name.
-     * @return Field value.
-     * @throws PortableException In case of any other error.
-     */
-    @Nullable public <F> F field(String fieldName) throws PortableException;
-
-    /**
-     * Checks whether field is set.
-     *
-     * @param fieldName Field name.
-     * @return {@code true} if field is set.
-     */
-    public boolean hasField(String fieldName);
-
-    /**
-     * Gets fully deserialized instance of portable object.
-     *
-     * @return Fully deserialized instance of portable object.
-     * @throws PortableInvalidClassException If class doesn't exist.
-     * @throws PortableException In case of any other error.
-     */
-    @Nullable public <T> T deserialize() throws PortableException;
-
-    /**
-     * Copies this portable object.
-     *
-     * @return Copy of this portable object.
-     */
-    public PortableObject clone() throws CloneNotSupportedException;
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/71379a80/modules/core/src/main/java/org/apache/ignite/portable/PortableProtocolVersion.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/portable/PortableProtocolVersion.java b/modules/core/src/main/java/org/apache/ignite/portable/PortableProtocolVersion.java
deleted file mode 100644
index 9189b28..0000000
--- a/modules/core/src/main/java/org/apache/ignite/portable/PortableProtocolVersion.java
+++ /dev/null
@@ -1,41 +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.portable;
-
-import org.jetbrains.annotations.Nullable;
-
-/**
- * Portable protocol version.
- */
-public enum PortableProtocolVersion {
-    /** Ignite 1.4.0 release. */
-    VER_1_4_0;
-
-    /** Enumerated values. */
-    private static final PortableProtocolVersion[] VALS = values();
-
-    /**
-     * Efficiently gets enumerated value from its ordinal.
-     *
-     * @param ord Ordinal value.
-     * @return Enumerated value or {@code null} if ordinal out of range.
-     */
-    @Nullable public static PortableProtocolVersion fromOrdinal(int ord) {
-        return ord >= 0 && ord < VALS.length ? VALS[ord] : null;
-    }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/71379a80/modules/core/src/main/java/org/apache/ignite/portable/PortableRawReader.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/portable/PortableRawReader.java b/modules/core/src/main/java/org/apache/ignite/portable/PortableRawReader.java
deleted file mode 100644
index 3bae2e1..0000000
--- a/modules/core/src/main/java/org/apache/ignite/portable/PortableRawReader.java
+++ /dev/null
@@ -1,234 +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.portable;
-
-import java.math.BigDecimal;
-import java.sql.Timestamp;
-import java.util.Collection;
-import java.util.Date;
-import java.util.Map;
-import java.util.UUID;
-import org.jetbrains.annotations.Nullable;
-
-/**
- * Raw reader for portable objects. Raw reader does not use field name hash codes, therefore,
- * making the format even more compact. However, if the raw reader is used,
- * dynamic structure changes to the portable objects are not supported.
- */
-public interface PortableRawReader {
-    /**
-     * @return Byte value.
-     * @throws PortableException In case of error.
-     */
-    public byte readByte() throws PortableException;
-
-    /**
-     * @return Short value.
-     * @throws PortableException In case of error.
-     */
-    public short readShort() throws PortableException;
-
-    /**
-     * @return Integer value.
-     * @throws PortableException In case of error.
-     */
-    public int readInt() throws PortableException;
-
-    /**
-     * @return Long value.
-     * @throws PortableException In case of error.
-     */
-    public long readLong() throws PortableException;
-
-    /**
-     * @return Float value.
-     * @throws PortableException In case of error.
-     */
-    public float readFloat() throws PortableException;
-
-    /**
-     * @return Double value.
-     * @throws PortableException In case of error.
-     */
-    public double readDouble() throws PortableException;
-
-    /**
-     * @return Char value.
-     * @throws PortableException In case of error.
-     */
-    public char readChar() throws PortableException;
-
-    /**
-     * @return Boolean value.
-     * @throws PortableException In case of error.
-     */
-    public boolean readBoolean() throws PortableException;
-
-    /**
-     * @return Decimal value.
-     * @throws PortableException In case of error.
-     */
-    @Nullable public BigDecimal readDecimal() throws PortableException;
-
-    /**
-     * @return String value.
-     * @throws PortableException In case of error.
-     */
-    @Nullable public String readString() throws PortableException;
-
-    /**
-     * @return UUID.
-     * @throws PortableException In case of error.
-     */
-    @Nullable public UUID readUuid() throws PortableException;
-
-    /**
-     * @return Date.
-     * @throws PortableException In case of error.
-     */
-    @Nullable public Date readDate() throws PortableException;
-
-    /**
-     * @return Timestamp.
-     * @throws PortableException In case of error.
-     */
-    @Nullable public Timestamp readTimestamp() throws PortableException;
-
-    /**
-     * @return Object.
-     * @throws PortableException In case of error.
-     */
-    @Nullable public <T> T readObject() throws PortableException;
-
-    /**
-     * @return Byte array.
-     * @throws PortableException In case of error.
-     */
-    @Nullable public byte[] readByteArray() throws PortableException;
-
-    /**
-     * @return Short array.
-     * @throws PortableException In case of error.
-     */
-    @Nullable public short[] readShortArray() throws PortableException;
-
-    /**
-     * @return Integer array.
-     * @throws PortableException In case of error.
-     */
-    @Nullable public int[] readIntArray() throws PortableException;
-
-    /**
-     * @return Long array.
-     * @throws PortableException In case of error.
-     */
-    @Nullable public long[] readLongArray() throws PortableException;
-
-    /**
-     * @return Float array.
-     * @throws PortableException In case of error.
-     */
-    @Nullable public float[] readFloatArray() throws PortableException;
-
-    /**
-     * @return Byte array.
-     * @throws PortableException In case of error.
-     */
-    @Nullable public double[] readDoubleArray() throws PortableException;
-
-    /**
-     * @return Char array.
-     * @throws PortableException In case of error.
-     */
-    @Nullable public char[] readCharArray() throws PortableException;
-
-    /**
-     * @return Boolean array.
-     * @throws PortableException In case of error.
-     */
-    @Nullable public boolean[] readBooleanArray() throws PortableException;
-
-    /**
-     * @return Decimal array.
-     * @throws PortableException In case of error.
-     */
-    @Nullable public BigDecimal[] readDecimalArray() throws PortableException;
-
-    /**
-     * @return String array.
-     * @throws PortableException In case of error.
-     */
-    @Nullable public String[] readStringArray() throws PortableException;
-
-    /**
-     * @return UUID array.
-     * @throws PortableException In case of error.
-     */
-    @Nullable public UUID[] readUuidArray() throws PortableException;
-
-    /**
-     * @return Date array.
-     * @throws PortableException In case of error.
-     */
-    @Nullable public Date[] readDateArray() throws PortableException;
-
-    /**
-     * @return Object array.
-     * @throws PortableException In case of error.
-     */
-    @Nullable public Object[] readObjectArray() throws PortableException;
-
-    /**
-     * @return Collection.
-     * @throws PortableException In case of error.
-     */
-    @Nullable public <T> Collection<T> readCollection() throws PortableException;
-
-    /**
-     * @param colCls Collection class.
-     * @return Collection.
-     * @throws PortableException In case of error.
-     */
-    @Nullable public <T> Collection<T> readCollection(Class<? extends Collection<T>> colCls)
-        throws PortableException;
-
-    /**
-     * @return Map.
-     * @throws PortableException In case of error.
-     */
-    @Nullable public <K, V> Map<K, V> readMap() throws PortableException;
-
-    /**
-     * @param mapCls Map class.
-     * @return Map.
-     * @throws PortableException In case of error.
-     */
-    @Nullable public <K, V> Map<K, V> readMap(Class<? extends Map<K, V>> mapCls) throws PortableException;
-
-    /**
-     * @return Value.
-     * @throws PortableException In case of error.
-     */
-    @Nullable public <T extends Enum<?>> T readEnum() throws PortableException;
-
-    /**
-     * @return Value.
-     * @throws PortableException In case of error.
-     */
-    @Nullable public <T extends Enum<?>> T[] readEnumArray() throws PortableException;
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/71379a80/modules/core/src/main/java/org/apache/ignite/portable/PortableRawWriter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/portable/PortableRawWriter.java b/modules/core/src/main/java/org/apache/ignite/portable/PortableRawWriter.java
deleted file mode 100644
index 53f4f92..0000000
--- a/modules/core/src/main/java/org/apache/ignite/portable/PortableRawWriter.java
+++ /dev/null
@@ -1,219 +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.portable;
-
-import java.math.BigDecimal;
-import java.sql.Timestamp;
-import java.util.Collection;
-import java.util.Date;
-import java.util.Map;
-import java.util.UUID;
-import org.jetbrains.annotations.Nullable;
-
-/**
- * Raw writer for portable object. Raw writer does not write field name hash codes, therefore,
- * making the format even more compact. However, if the raw writer is used,
- * dynamic structure changes to the portable objects are not supported.
- */
-public interface PortableRawWriter {
-    /**
-     * @param val Value to write.
-     * @throws PortableException In case of error.
-     */
-    public void writeByte(byte val) throws PortableException;
-
-    /**
-     * @param val Value to write.
-     * @throws PortableException In case of error.
-     */
-    public void writeShort(short val) throws PortableException;
-
-    /**
-     * @param val Value to write.
-     * @throws PortableException In case of error.
-     */
-    public void writeInt(int val) throws PortableException;
-
-    /**
-     * @param val Value to write.
-     * @throws PortableException In case of error.
-     */
-    public void writeLong(long val) throws PortableException;
-
-    /**
-     * @param val Value to write.
-     * @throws PortableException In case of error.
-     */
-    public void writeFloat(float val) throws PortableException;
-
-    /**
-     * @param val Value to write.
-     * @throws PortableException In case of error.
-     */
-    public void writeDouble(double val) throws PortableException;
-
-    /**
-     * @param val Value to write.
-     * @throws PortableException In case of error.
-     */
-    public void writeChar(char val) throws PortableException;
-
-    /**
-     * @param val Value to write.
-     * @throws PortableException In case of error.
-     */
-    public void writeBoolean(boolean val) throws PortableException;
-
-    /**
-     * @param val Value to write.
-     * @throws PortableException In case of error.
-     */
-    public void writeDecimal(@Nullable BigDecimal val) throws PortableException;
-
-    /**
-     * @param val Value to write.
-     * @throws PortableException In case of error.
-     */
-    public void writeString(@Nullable String val) throws PortableException;
-
-    /**
-     * @param val UUID to write.
-     * @throws PortableException In case of error.
-     */
-    public void writeUuid(@Nullable UUID val) throws PortableException;
-
-    /**
-     * @param val Date to write.
-     * @throws PortableException In case of error.
-     */
-    public void writeDate(@Nullable Date val) throws PortableException;
-
-    /**
-     * @param val Timestamp to write.
-     * @throws PortableException In case of error.
-     */
-    public void writeTimestamp(@Nullable Timestamp val) throws PortableException;
-
-    /**
-     * @param obj Value to write.
-     * @throws PortableException In case of error.
-     */
-    public void writeObject(@Nullable Object obj) throws PortableException;
-
-    /**
-     * @param val Value to write.
-     * @throws PortableException In case of error.
-     */
-    public void writeByteArray(@Nullable byte[] val) throws PortableException;
-
-    /**
-     * @param val Value to write.
-     * @throws PortableException In case of error.
-     */
-    public void writeShortArray(@Nullable short[] val) throws PortableException;
-
-    /**
-     * @param val Value to write.
-     * @throws PortableException In case of error.
-     */
-    public void writeIntArray(@Nullable int[] val) throws PortableException;
-
-    /**
-     * @param val Value to write.
-     * @throws PortableException In case of error.
-     */
-    public void writeLongArray(@Nullable long[] val) throws PortableException;
-
-    /**
-     * @param val Value to write.
-     * @throws PortableException In case of error.
-     */
-    public void writeFloatArray(@Nullable float[] val) throws PortableException;
-
-    /**
-     * @param val Value to write.
-     * @throws PortableException In case of error.
-     */
-    public void writeDoubleArray(@Nullable double[] val) throws PortableException;
-
-    /**
-     * @param val Value to write.
-     * @throws PortableException In case of error.
-     */
-    public void writeCharArray(@Nullable char[] val) throws PortableException;
-
-    /**
-     * @param val Value to write.
-     * @throws PortableException In case of error.
-     */
-    public void writeBooleanArray(@Nullable boolean[] val) throws PortableException;
-
-    /**
-     * @param val Value to write.
-     * @throws PortableException In case of error.
-     */
-    public void writeDecimalArray(@Nullable BigDecimal[] val) throws PortableException;
-
-    /**
-     * @param val Value to write.
-     * @throws PortableException In case of error.
-     */
-    public void writeStringArray(@Nullable String[] val) throws PortableException;
-
-    /**
-     * @param val Value to write.
-     * @throws PortableException In case of error.
-     */
-    public void writeUuidArray(@Nullable UUID[] val) throws PortableException;
-
-    /**
-     * @param val Value to write.
-     * @throws PortableException In case of error.
-     */
-    public void writeDateArray(@Nullable Date[] val) throws PortableException;
-
-    /**
-     * @param val Value to write.
-     * @throws PortableException In case of error.
-     */
-    public void writeObjectArray(@Nullable Object[] val) throws PortableException;
-
-    /**
-     * @param col Collection to write.
-     * @throws PortableException In case of error.
-     */
-    public <T> void writeCollection(@Nullable Collection<T> col) throws PortableException;
-
-    /**
-     * @param map Map to write.
-     * @throws PortableException In case of error.
-     */
-    public <K, V> void writeMap(@Nullable Map<K, V> map) throws PortableException;
-
-    /**
-     * @param val Value to write.
-     * @throws PortableException In case of error.
-     */
-    public <T extends Enum<?>> void writeEnum(T val) throws PortableException;
-
-    /**
-     * @param val Value to write.
-     * @throws PortableException In case of error.
-     */
-    public <T extends Enum<?>> void writeEnumArray(T[] val) throws PortableException;
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/71379a80/modules/core/src/main/java/org/apache/ignite/portable/PortableReader.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/portable/PortableReader.java b/modules/core/src/main/java/org/apache/ignite/portable/PortableReader.java
deleted file mode 100644
index 58f078d..0000000
--- a/modules/core/src/main/java/org/apache/ignite/portable/PortableReader.java
+++ /dev/null
@@ -1,284 +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.portable;
-
-import java.math.BigDecimal;
-import java.sql.Timestamp;
-import java.util.Collection;
-import java.util.Date;
-import java.util.Map;
-import java.util.UUID;
-import org.jetbrains.annotations.Nullable;
-
-/**
- * Reader for portable objects used in {@link PortableMarshalAware} implementations.
- * Useful for the cases when user wants a fine-grained control over serialization.
- * <p>
- * Note that Ignite never writes full strings for field or type names. Instead,
- * for performance reasons, Ignite writes integer hash codes for type and field names.
- * It has been tested that hash code conflicts for the type names or the field names
- * within the same type are virtually non-existent and, to gain performance, it is safe
- * to work with hash codes. For the cases when hash codes for different types or fields
- * actually do collide, Ignite provides {@link PortableIdMapper} which
- * allows to override the automatically generated hash code IDs for the type and field names.
- */
-public interface PortableReader {
-    /**
-     * @param fieldName Field name.
-     * @return Byte value.
-     * @throws PortableException In case of error.
-     */
-    public byte readByte(String fieldName) throws PortableException;
-
-    /**
-     * @param fieldName Field name.
-     * @return Short value.
-     * @throws PortableException In case of error.
-     */
-    public short readShort(String fieldName) throws PortableException;
-
-    /**
-     * @param fieldName Field name.
-     * @return Integer value.
-     * @throws PortableException In case of error.
-     */
-    public int readInt(String fieldName) throws PortableException;
-
-    /**
-     * @param fieldName Field name.
-     * @return Long value.
-     * @throws PortableException In case of error.
-     */
-    public long readLong(String fieldName) throws PortableException;
-
-    /**
-     * @param fieldName Field name.
-     * @throws PortableException In case of error.
-     * @return Float value.
-     */
-    public float readFloat(String fieldName) throws PortableException;
-
-    /**
-     * @param fieldName Field name.
-     * @return Double value.
-     * @throws PortableException In case of error.
-     */
-    public double readDouble(String fieldName) throws PortableException;
-
-    /**
-     * @param fieldName Field name.
-     * @return Char value.
-     * @throws PortableException In case of error.
-     */
-    public char readChar(String fieldName) throws PortableException;
-
-    /**
-     * @param fieldName Field name.
-     * @return Boolean value.
-     * @throws PortableException In case of error.
-     */
-    public boolean readBoolean(String fieldName) throws PortableException;
-
-    /**
-     * @param fieldName Field name.
-     * @return Decimal value.
-     * @throws PortableException In case of error.
-     */
-    @Nullable public BigDecimal readDecimal(String fieldName) throws PortableException;
-
-    /**
-     * @param fieldName Field name.
-     * @return String value.
-     * @throws PortableException In case of error.
-     */
-    @Nullable public String readString(String fieldName) throws PortableException;
-
-    /**
-     * @param fieldName Field name.
-     * @return UUID.
-     * @throws PortableException In case of error.
-     */
-    @Nullable public UUID readUuid(String fieldName) throws PortableException;
-
-    /**
-     * @param fieldName Field name.
-     * @return Date.
-     * @throws PortableException In case of error.
-     */
-    @Nullable public Date readDate(String fieldName) throws PortableException;
-
-    /**
-     * @param fieldName Field name.
-     * @return Timestamp.
-     * @throws PortableException In case of error.
-     */
-    @Nullable public Timestamp readTimestamp(String fieldName) throws PortableException;
-
-    /**
-     * @param fieldName Field name.
-     * @return Object.
-     * @throws PortableException In case of error.
-     */
-    @Nullable public <T> T readObject(String fieldName) throws PortableException;
-
-    /**
-     * @param fieldName Field name.
-     * @return Byte array.
-     * @throws PortableException In case of error.
-     */
-    @Nullable public byte[] readByteArray(String fieldName) throws PortableException;
-
-    /**
-     * @param fieldName Field name.
-     * @return Short array.
-     * @throws PortableException In case of error.
-     */
-    @Nullable public short[] readShortArray(String fieldName) throws PortableException;
-
-    /**
-     * @param fieldName Field name.
-     * @return Integer array.
-     * @throws PortableException In case of error.
-     */
-    @Nullable public int[] readIntArray(String fieldName) throws PortableException;
-
-    /**
-     * @param fieldName Field name.
-     * @return Long array.
-     * @throws PortableException In case of error.
-     */
-    @Nullable public long[] readLongArray(String fieldName) throws PortableException;
-
-    /**
-     * @param fieldName Field name.
-     * @return Float array.
-     * @throws PortableException In case of error.
-     */
-    @Nullable public float[] readFloatArray(String fieldName) throws PortableException;
-
-    /**
-     * @param fieldName Field name.
-     * @return Byte array.
-     * @throws PortableException In case of error.
-     */
-    @Nullable public double[] readDoubleArray(String fieldName) throws PortableException;
-
-    /**
-     * @param fieldName Field name.
-     * @return Char array.
-     * @throws PortableException In case of error.
-     */
-    @Nullable public char[] readCharArray(String fieldName) throws PortableException;
-
-    /**
-     * @param fieldName Field name.
-     * @return Boolean array.
-     * @throws PortableException In case of error.
-     */
-    @Nullable public boolean[] readBooleanArray(String fieldName) throws PortableException;
-
-    /**
-     * @param fieldName Field name.
-     * @return Decimal array.
-     * @throws PortableException In case of error.
-     */
-    @Nullable public BigDecimal[] readDecimalArray(String fieldName) throws PortableException;
-
-    /**
-     * @param fieldName Field name.
-     * @return String array.
-     * @throws PortableException In case of error.
-     */
-    @Nullable public String[] readStringArray(String fieldName) throws PortableException;
-
-    /**
-     * @param fieldName Field name.
-     * @return UUID array.
-     * @throws PortableException In case of error.
-     */
-    @Nullable public UUID[] readUuidArray(String fieldName) throws PortableException;
-
-    /**
-     * @param fieldName Field name.
-     * @return Date array.
-     * @throws PortableException In case of error.
-     */
-    @Nullable public Date[] readDateArray(String fieldName) throws PortableException;
-
-    /**
-     * @param fieldName Field name.
-     * @return Object array.
-     * @throws PortableException In case of error.
-     */
-    @Nullable public Object[] readObjectArray(String fieldName) throws PortableException;
-
-    /**
-     * @param fieldName Field name.
-     * @return Collection.
-     * @throws PortableException In case of error.
-     */
-    @Nullable public <T> Collection<T> readCollection(String fieldName) throws PortableException;
-
-    /**
-     * @param fieldName Field name.
-     * @param colCls Collection class.
-     * @return Collection.
-     * @throws PortableException In case of error.
-     */
-    @Nullable public <T> Collection<T> readCollection(String fieldName, Class<? extends Collection<T>> colCls)
-        throws PortableException;
-
-    /**
-     * @param fieldName Field name.
-     * @return Map.
-     * @throws PortableException In case of error.
-     */
-    @Nullable public <K, V> Map<K, V> readMap(String fieldName) throws PortableException;
-
-    /**
-     * @param fieldName Field name.
-     * @param mapCls Map class.
-     * @return Map.
-     * @throws PortableException In case of error.
-     */
-    @Nullable public <K, V> Map<K, V> readMap(String fieldName, Class<? extends Map<K, V>> mapCls)
-        throws PortableException;
-
-    /**
-     * @param fieldName Field name.
-     * @return Value.
-     * @throws PortableException In case of error.
-     */
-    @Nullable public <T extends Enum<?>> T readEnum(String fieldName) throws PortableException;
-
-    /**
-     * @param fieldName Field name.
-     * @return Value.
-     * @throws PortableException In case of error.
-     */
-    @Nullable public <T extends Enum<?>> T[] readEnumArray(String fieldName) throws PortableException;
-
-    /**
-     * Gets raw reader. Raw reader does not use field name hash codes, therefore,
-     * making the format even more compact. However, if the raw reader is used,
-     * dynamic structure changes to the portable objects are not supported.
-     *
-     * @return Raw reader.
-     */
-    public PortableRawReader rawReader();
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/71379a80/modules/core/src/main/java/org/apache/ignite/portable/PortableSerializer.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/portable/PortableSerializer.java b/modules/core/src/main/java/org/apache/ignite/portable/PortableSerializer.java
deleted file mode 100644
index 90ee562..0000000
--- a/modules/core/src/main/java/org/apache/ignite/portable/PortableSerializer.java
+++ /dev/null
@@ -1,49 +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.portable;
-
-import org.apache.ignite.marshaller.portable.PortableMarshaller;
-
-/**
- * Interface that allows to implement custom serialization logic for portable objects.
- * Can be used instead of {@link PortableMarshalAware} in case if the class
- * cannot be changed directly.
- * <p>
- * Portable serializer can be configured for all portable objects via
- * {@link PortableMarshaller#getSerializer()} method, or for a specific
- * portable type via {@link PortableTypeConfiguration#getSerializer()} method.
- */
-public interface PortableSerializer {
-    /**
-     * Writes fields to provided writer.
-     *
-     * @param obj Empty object.
-     * @param writer Portable object writer.
-     * @throws PortableException In case of error.
-     */
-    public void writePortable(Object obj, PortableWriter writer) throws PortableException;
-
-    /**
-     * Reads fields from provided reader.
-     *
-     * @param obj Empty object
-     * @param reader Portable object reader.
-     * @throws PortableException In case of error.
-     */
-    public void readPortable(Object obj, PortableReader reader) throws PortableException;
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/71379a80/modules/core/src/main/java/org/apache/ignite/portable/PortableTypeConfiguration.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/portable/PortableTypeConfiguration.java b/modules/core/src/main/java/org/apache/ignite/portable/PortableTypeConfiguration.java
deleted file mode 100644
index 5e6e09d..0000000
--- a/modules/core/src/main/java/org/apache/ignite/portable/PortableTypeConfiguration.java
+++ /dev/null
@@ -1,196 +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.portable;
-
-import java.sql.Timestamp;
-import java.util.Collection;
-import org.apache.ignite.internal.util.typedef.internal.S;
-import org.apache.ignite.marshaller.portable.PortableMarshaller;
-
-/**
- * Defines configuration properties for a specific portable type. Providing per-type
- * configuration is optional, as it is generally enough, and also optional, to provide global portable
- * configuration using {@link PortableMarshaller#setClassNames(Collection)}.
- * However, this class allows you to change configuration properties for a specific
- * portable type without affecting configuration for other portable types.
- * <p>
- * Per-type portable configuration can be specified in {@link PortableMarshaller#getTypeConfigurations()} method.
- */
-public class PortableTypeConfiguration {
-    /** Class name. */
-    private String clsName;
-
-    /** ID mapper. */
-    private PortableIdMapper idMapper;
-
-    /** Serializer. */
-    private PortableSerializer serializer;
-
-    /** Use timestamp flag. */
-    private Boolean useTs;
-
-    /** Meta data enabled flag. */
-    private Boolean metaDataEnabled;
-
-    /** Keep deserialized flag. */
-    private Boolean keepDeserialized;
-
-    /** Affinity key field name. */
-    private String affKeyFieldName;
-
-    /**
-     */
-    public PortableTypeConfiguration() {
-        // No-op.
-    }
-
-    /**
-     * @param clsName Class name.
-     */
-    public PortableTypeConfiguration(String clsName) {
-        this.clsName = clsName;
-    }
-
-    /**
-     * Gets type name.
-     *
-     * @return Type name.
-     */
-    public String getClassName() {
-        return clsName;
-    }
-
-    /**
-     * Sets type name.
-     *
-     * @param clsName Type name.
-     */
-    public void setClassName(String clsName) {
-        this.clsName = clsName;
-    }
-
-    /**
-     * Gets ID mapper.
-     *
-     * @return ID mapper.
-     */
-    public PortableIdMapper getIdMapper() {
-        return idMapper;
-    }
-
-    /**
-     * Sets ID mapper.
-     *
-     * @param idMapper ID mapper.
-     */
-    public void setIdMapper(PortableIdMapper idMapper) {
-        this.idMapper = idMapper;
-    }
-
-    /**
-     * Gets serializer.
-     *
-     * @return Serializer.
-     */
-    public PortableSerializer getSerializer() {
-        return serializer;
-    }
-
-    /**
-     * Sets serializer.
-     *
-     * @param serializer Serializer.
-     */
-    public void setSerializer(PortableSerializer serializer) {
-        this.serializer = serializer;
-    }
-
-    /**
-     * If {@code true} then date values converted to {@link Timestamp} during unmarshalling.
-     *
-     * @return Flag indicating whether date values converted to {@link Timestamp} during unmarshalling.
-     */
-    public Boolean isUseTimestamp() {
-        return useTs;
-    }
-
-    /**
-     * @param useTs Flag indicating whether date values converted to {@link Timestamp} during unmarshalling.
-     */
-    public void setUseTimestamp(Boolean useTs) {
-        this.useTs = useTs;
-    }
-
-    /**
-     * Defines whether meta data is collected for this type. If provided, this value will override
-     * {@link PortableMarshaller#isMetaDataEnabled()} property.
-     *
-     * @return Whether meta data is collected.
-     */
-    public Boolean isMetaDataEnabled() {
-        return metaDataEnabled;
-    }
-
-    /**
-     * @param metaDataEnabled Whether meta data is collected.
-     */
-    public void setMetaDataEnabled(Boolean metaDataEnabled) {
-        this.metaDataEnabled = metaDataEnabled;
-    }
-
-    /**
-     * Defines whether {@link PortableObject} should cache deserialized instance. If provided,
-     * this value will override {@link PortableMarshaller#isKeepDeserialized()}
-     * property.
-     *
-     * @return Whether deserialized value is kept.
-     */
-    public Boolean isKeepDeserialized() {
-        return keepDeserialized;
-    }
-
-    /**
-     * @param keepDeserialized Whether deserialized value is kept.
-     */
-    public void setKeepDeserialized(Boolean keepDeserialized) {
-        this.keepDeserialized = keepDeserialized;
-    }
-
-    /**
-     * Gets affinity key field name.
-     *
-     * @return Affinity key field name.
-     */
-    public String getAffinityKeyFieldName() {
-        return affKeyFieldName;
-    }
-
-    /**
-     * Sets affinity key field name.
-     *
-     * @param affKeyFieldName Affinity key field name.
-     */
-    public void setAffinityKeyFieldName(String affKeyFieldName) {
-        this.affKeyFieldName = affKeyFieldName;
-    }
-
-    /** {@inheritDoc} */
-    @Override public String toString() {
-        return S.toString(PortableTypeConfiguration.class, this, super.toString());
-    }
-}
\ No newline at end of file


[25/55] [abbrv] ignite git commit: IGNITE-1477: Fixed.

Posted by ag...@apache.org.
IGNITE-1477: Fixed.


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

Branch: refs/heads/ignite-1171
Commit: c0e1ac1842df19a4de83dbbcc99090b43371c913
Parents: c065512
Author: vozerov-gridgain <vo...@gridgain.com>
Authored: Tue Sep 15 10:25:09 2015 +0300
Committer: vozerov-gridgain <vo...@gridgain.com>
Committed: Tue Sep 15 10:25:09 2015 +0300

----------------------------------------------------------------------
 .../hadoop/igfs/HadoopIgfsWrapper.java          | 94 +++++++++++---------
 1 file changed, 53 insertions(+), 41 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/c0e1ac18/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/igfs/HadoopIgfsWrapper.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/igfs/HadoopIgfsWrapper.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/igfs/HadoopIgfsWrapper.java
index abbb142..01189f7 100644
--- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/igfs/HadoopIgfsWrapper.java
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/igfs/HadoopIgfsWrapper.java
@@ -35,6 +35,7 @@ import org.apache.ignite.internal.processors.igfs.IgfsHandshakeResponse;
 import org.apache.ignite.internal.processors.igfs.IgfsStatus;
 import org.apache.ignite.internal.util.typedef.F;
 import org.apache.ignite.internal.util.typedef.G;
+import org.apache.ignite.internal.util.typedef.internal.SB;
 import org.apache.ignite.internal.util.typedef.internal.U;
 import org.jetbrains.annotations.Nullable;
 
@@ -339,7 +340,9 @@ public class HadoopIgfsWrapper implements HadoopIgfs {
      * @return Delegate.
      */
     private Delegate delegate() throws HadoopIgfsCommunicationException {
-        Exception err = null;
+        // These fields will contain possible exceptions from shmem and TCP endpoints.
+        Exception errShmem = null;
+        Exception errTcp = null;
 
         // 1. If delegate is set, return it immediately.
         Delegate curDelegate = delegateRef.get();
@@ -357,8 +360,8 @@ public class HadoopIgfsWrapper implements HadoopIgfs {
 
                     igfs = (IgfsEx)ignite.fileSystem(endpoint.igfs());
                 }
-                catch (Exception e) {
-                    err = e;
+                catch (Exception ignore) {
+                    // No-op.
                 }
             }
             else {
@@ -368,8 +371,8 @@ public class HadoopIgfsWrapper implements HadoopIgfs {
 
                         break;
                     }
-                    catch (Exception e) {
-                        err = e;
+                    catch (Exception ignore) {
+                        // No-op.
                     }
                 }
             }
@@ -388,57 +391,54 @@ public class HadoopIgfsWrapper implements HadoopIgfs {
                             hadoop.close(true);
 
                     if (log.isDebugEnabled())
-                        log.debug("Failed to connect to in-proc IGFS, fallback to IPC mode.", e);
-
-                    err = e;
+                        log.debug("Failed to connect to in-process IGFS, fallback to IPC mode.", e);
                 }
             }
         }
 
         // 3. Try connecting using shmem.
-        if (!parameter(conf, PARAM_IGFS_ENDPOINT_NO_LOCAL_SHMEM, authority, false)) {
-            if (curDelegate == null && !U.isWindows()) {
-                HadoopIgfsEx hadoop = null;
+        boolean skipLocShmem = parameter(conf, PARAM_IGFS_ENDPOINT_NO_LOCAL_SHMEM, authority, false);
 
-                try {
-                    hadoop = new HadoopIgfsOutProc(endpoint.port(), endpoint.grid(), endpoint.igfs(), log, userName);
+        if (curDelegate == null && !skipLocShmem && !U.isWindows()) {
+            HadoopIgfsEx hadoop = null;
 
-                    curDelegate = new Delegate(hadoop, hadoop.handshake(logDir));
-                }
-                catch (IOException | IgniteCheckedException e) {
-                    if (e instanceof HadoopIgfsCommunicationException)
-                        hadoop.close(true);
+            try {
+                hadoop = new HadoopIgfsOutProc(endpoint.port(), endpoint.grid(), endpoint.igfs(), log, userName);
 
-                    if (log.isDebugEnabled())
-                        log.debug("Failed to connect to out-proc local IGFS using shmem.", e);
+                curDelegate = new Delegate(hadoop, hadoop.handshake(logDir));
+            }
+            catch (IOException | IgniteCheckedException e) {
+                if (e instanceof HadoopIgfsCommunicationException)
+                    hadoop.close(true);
 
-                    err = e;
-                }
+                if (log.isDebugEnabled())
+                    log.debug("Failed to connect to IGFS using shared memory [port=" + endpoint.port() + ']', e);
+
+                errShmem = e;
             }
         }
 
         // 4. Try local TCP connection.
         boolean skipLocTcp = parameter(conf, PARAM_IGFS_ENDPOINT_NO_LOCAL_TCP, authority, false);
 
-        if (!skipLocTcp) {
-            if (curDelegate == null) {
-                HadoopIgfsEx hadoop = null;
+        if (curDelegate == null && !skipLocTcp) {
+            HadoopIgfsEx hadoop = null;
 
-                try {
-                    hadoop = new HadoopIgfsOutProc(LOCALHOST, endpoint.port(), endpoint.grid(), endpoint.igfs(),
-                        log, userName);
+            try {
+                hadoop = new HadoopIgfsOutProc(LOCALHOST, endpoint.port(), endpoint.grid(), endpoint.igfs(),
+                    log, userName);
 
-                    curDelegate = new Delegate(hadoop, hadoop.handshake(logDir));
-                }
-                catch (IOException | IgniteCheckedException e) {
-                    if (e instanceof HadoopIgfsCommunicationException)
-                        hadoop.close(true);
+                curDelegate = new Delegate(hadoop, hadoop.handshake(logDir));
+            }
+            catch (IOException | IgniteCheckedException e) {
+                if (e instanceof HadoopIgfsCommunicationException)
+                    hadoop.close(true);
 
-                    if (log.isDebugEnabled())
-                        log.debug("Failed to connect to out-proc local IGFS using TCP.", e);
+                if (log.isDebugEnabled())
+                    log.debug("Failed to connect to IGFS using TCP [host=" + endpoint.host() +
+                        ", port=" + endpoint.port() + ']', e);
 
-                    err = e;
-                }
+                errTcp = e;
             }
         }
 
@@ -457,9 +457,10 @@ public class HadoopIgfsWrapper implements HadoopIgfs {
                     hadoop.close(true);
 
                 if (log.isDebugEnabled())
-                    log.debug("Failed to connect to out-proc remote IGFS using TCP.", e);
+                    log.debug("Failed to connect to IGFS using TCP [host=" + endpoint.host() +
+                        ", port=" + endpoint.port() + ']', e);
 
-                err = e;
+                errTcp = e;
             }
         }
 
@@ -469,8 +470,19 @@ public class HadoopIgfsWrapper implements HadoopIgfs {
 
             return curDelegate;
         }
-        else
-            throw new HadoopIgfsCommunicationException("Failed to connect to IGFS: " + endpoint, err);
+        else {
+            SB errMsg = new SB("Failed to connect to IGFS [endpoint=" + authority + ", attempts=[");
+
+            if (errShmem != null)
+                errMsg.a("[type=SHMEM, port=" + endpoint.port() + ", err=" + errShmem + "], ");
+
+            errMsg.a("[type=TCP, host=" + endpoint.host() + ", port=" + endpoint.port() + ", err=" + errTcp + "]] ");
+
+            errMsg.a("(ensure that IGFS is running and have IPC endpoint enabled; ensure that " +
+                "ignite-shmem-1.0.0.jar is in Hadoop classpath if you use shared memory endpoint).");
+
+            throw new HadoopIgfsCommunicationException(errMsg.toString());
+        }
     }
 
     /**


[05/55] [abbrv] ignite git commit: ignite-1462: hid portable API in 1.4 release

Posted by ag...@apache.org.
http://git-wip-us.apache.org/repos/asf/ignite/blob/71379a80/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
deleted file mode 100644
index 21fc81c..0000000
--- a/modules/core/src/test/java/org/apache/ignite/internal/portable/GridPortableMarshallerSelfTest.java
+++ /dev/null
@@ -1,3807 +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.lang.reflect.Field;
-import java.lang.reflect.InvocationTargetException;
-import java.lang.reflect.Method;
-import java.math.BigDecimal;
-import java.math.BigInteger;
-import java.net.InetSocketAddress;
-import java.sql.Timestamp;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.Date;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.LinkedHashMap;
-import java.util.LinkedHashSet;
-import java.util.Map;
-import java.util.TreeMap;
-import java.util.TreeSet;
-import java.util.UUID;
-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;
-import org.apache.ignite.internal.util.typedef.F;
-import org.apache.ignite.internal.util.typedef.internal.S;
-import org.apache.ignite.internal.util.typedef.internal.U;
-import org.apache.ignite.marshaller.MarshallerContextTestImpl;
-import org.apache.ignite.marshaller.portable.PortableMarshaller;
-import org.apache.ignite.portable.PortableBuilder;
-import org.apache.ignite.portable.PortableException;
-import org.apache.ignite.portable.PortableIdMapper;
-import org.apache.ignite.portable.PortableInvalidClassException;
-import org.apache.ignite.portable.PortableMarshalAware;
-import org.apache.ignite.portable.PortableMetadata;
-import org.apache.ignite.portable.PortableObject;
-import org.apache.ignite.portable.PortableRawReader;
-import org.apache.ignite.portable.PortableRawWriter;
-import org.apache.ignite.portable.PortableReader;
-import org.apache.ignite.portable.PortableSerializer;
-import org.apache.ignite.portable.PortableTypeConfiguration;
-import org.apache.ignite.portable.PortableWriter;
-import org.apache.ignite.testframework.GridTestUtils;
-import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
-import org.jsr166.ConcurrentHashMap8;
-import sun.misc.Unsafe;
-
-import static org.apache.ignite.internal.portable.PortableThreadLocalMemoryAllocator.THREAD_LOCAL_ALLOC;
-import static org.junit.Assert.assertArrayEquals;
-
-/**
- * Portable marshaller tests.
- */
-@SuppressWarnings({"OverlyStrongTypeCast", "ArrayHashCode", "ConstantConditions"})
-public class GridPortableMarshallerSelfTest extends GridCommonAbstractTest {
-    /** */
-    private static final Unsafe UNSAFE = GridUnsafe.unsafe();
-
-    /** */
-    protected static final long BYTE_ARR_OFF = UNSAFE.arrayBaseOffset(byte[].class);
-
-    /** */
-    protected static final PortableMetaDataHandler META_HND = new PortableMetaDataHandler() {
-        @Override public void addMeta(int typeId, PortableMetadata meta) {
-            // No-op.
-        }
-
-        @Override public PortableMetadata metadata(int typeId) {
-            return null;
-        }
-    };
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testNull() throws Exception {
-        assertNull(marshalUnmarshal(null));
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testByte() throws Exception {
-        assertEquals((byte)100, marshalUnmarshal((byte)100).byteValue());
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testShort() throws Exception {
-        assertEquals((short)100, marshalUnmarshal((short)100).shortValue());
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testInt() throws Exception {
-        assertEquals(100, marshalUnmarshal(100).intValue());
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testLong() throws Exception {
-        assertEquals(100L, marshalUnmarshal(100L).longValue());
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testFloat() throws Exception {
-        assertEquals(100.001f, marshalUnmarshal(100.001f).floatValue(), 0);
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testDouble() throws Exception {
-        assertEquals(100.001d, marshalUnmarshal(100.001d).doubleValue(), 0);
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testChar() throws Exception {
-        assertEquals((char)100, marshalUnmarshal((char)100).charValue());
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testBoolean() throws Exception {
-        assertEquals(true, marshalUnmarshal(true).booleanValue());
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testDecimal() throws Exception {
-        BigDecimal val;
-
-        assertEquals((val = BigDecimal.ZERO), marshalUnmarshal(val));
-        assertEquals((val = BigDecimal.valueOf(Long.MAX_VALUE, 0)), marshalUnmarshal(val));
-        assertEquals((val = BigDecimal.valueOf(Long.MIN_VALUE, 0)), marshalUnmarshal(val));
-        assertEquals((val = BigDecimal.valueOf(Long.MAX_VALUE, 8)), marshalUnmarshal(val));
-        assertEquals((val = BigDecimal.valueOf(Long.MIN_VALUE, 8)), marshalUnmarshal(val));
-
-        assertEquals((val = new BigDecimal(new BigInteger("-79228162514264337593543950336"))), marshalUnmarshal(val));
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testString() throws Exception {
-        assertEquals("str", marshalUnmarshal("str"));
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testUuid() throws Exception {
-        UUID uuid = UUID.randomUUID();
-
-        assertEquals(uuid, marshalUnmarshal(uuid));
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testDate() throws Exception {
-        Date date = new Date();
-
-        Date val = marshalUnmarshal(date);
-
-        assertEquals(date, val);
-        assertEquals(Timestamp.class, val.getClass()); // With default configuration should unmarshal as Timestamp.
-
-        PortableMarshaller marsh = new PortableMarshaller();
-
-        marsh.setUseTimestamp(false);
-
-        val = marshalUnmarshal(date, marsh);
-
-        assertEquals(date, val);
-        assertEquals(Date.class, val.getClass());
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testTimestamp() throws Exception {
-        Timestamp ts = new Timestamp(System.currentTimeMillis());
-
-        ts.setNanos(999999999);
-
-        assertEquals(ts, marshalUnmarshal(ts));
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testByteArray() throws Exception {
-        byte[] arr = new byte[] {10, 20, 30};
-
-        assertArrayEquals(arr, marshalUnmarshal(arr));
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testShortArray() throws Exception {
-        short[] arr = new short[] {10, 20, 30};
-
-        assertArrayEquals(arr, marshalUnmarshal(arr));
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testIntArray() throws Exception {
-        int[] arr = new int[] {10, 20, 30};
-
-        assertArrayEquals(arr, marshalUnmarshal(arr));
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testLongArray() throws Exception {
-        long[] arr = new long[] {10, 20, 30};
-
-        assertArrayEquals(arr, marshalUnmarshal(arr));
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testFloatArray() throws Exception {
-        float[] arr = new float[] {10.1f, 20.1f, 30.1f};
-
-        assertArrayEquals(arr, marshalUnmarshal(arr), 0);
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testDoubleArray() throws Exception {
-        double[] arr = new double[] {10.1d, 20.1d, 30.1d};
-
-        assertArrayEquals(arr, marshalUnmarshal(arr), 0);
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testCharArray() throws Exception {
-        char[] arr = new char[] {10, 20, 30};
-
-        assertArrayEquals(arr, marshalUnmarshal(arr));
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testBooleanArray() throws Exception {
-        boolean[] arr = new boolean[] {true, false, true};
-
-        assertBooleanArrayEquals(arr, marshalUnmarshal(arr));
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testDecimalArray() throws Exception {
-        BigDecimal[] arr = new BigDecimal[] { BigDecimal.ZERO, BigDecimal.ONE, BigDecimal.TEN } ;
-
-        assertArrayEquals(arr, marshalUnmarshal(arr));
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testStringArray() throws Exception {
-        String[] arr = new String[] {"str1", "str2", "str3"};
-
-        assertArrayEquals(arr, marshalUnmarshal(arr));
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testUuidArray() throws Exception {
-        UUID[] arr = new UUID[] {UUID.randomUUID(), UUID.randomUUID(), UUID.randomUUID()};
-
-        assertArrayEquals(arr, marshalUnmarshal(arr));
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testDateArray() throws Exception {
-        Date[] arr = new Date[] {new Date(11111), new Date(22222), new Date(33333)};
-
-        assertArrayEquals(arr, marshalUnmarshal(arr));
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testObjectArray() throws Exception {
-        Object[] arr = new Object[] {1, 2, 3};
-
-        assertArrayEquals(arr, marshalUnmarshal(arr));
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testCollection() throws Exception {
-        testCollection(new ArrayList<Integer>(3));
-        testCollection(new LinkedHashSet<Integer>());
-        testCollection(new HashSet<Integer>());
-        testCollection(new TreeSet<Integer>());
-        testCollection(new ConcurrentSkipListSet<Integer>());
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    private void testCollection(Collection<Integer> col) throws Exception {
-        col.add(1);
-        col.add(2);
-        col.add(3);
-
-        assertEquals(col, marshalUnmarshal(col));
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testMap() throws Exception {
-        testMap(new HashMap<Integer, String>());
-        testMap(new LinkedHashMap());
-        testMap(new TreeMap<Integer, String>());
-        testMap(new ConcurrentHashMap8<Integer, String>());
-        testMap(new ConcurrentHashMap<Integer, String>());
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    private void testMap(Map<Integer, String> map) throws Exception {
-        map.put(1, "str1");
-        map.put(2, "str2");
-        map.put(3, "str3");
-
-        assertEquals(map, marshalUnmarshal(map));
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testMapEntry() throws Exception {
-        Map.Entry<Integer, String> e = new GridMapEntry<>(1, "str1");
-
-        assertEquals(e, marshalUnmarshal(e));
-
-        Map<Integer, String> map = new HashMap<>(1);
-
-        map.put(2, "str2");
-
-        e = F.firstEntry(map);
-
-        Map.Entry<Integer, String> e0 = marshalUnmarshal(e);
-
-        assertEquals(2, e0.getKey().intValue());
-        assertEquals("str2", e0.getValue());
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testPortableObject() throws Exception {
-        PortableMarshaller marsh = new PortableMarshaller();
-
-        marsh.setTypeConfigurations(Arrays.asList(new PortableTypeConfiguration(SimpleObject.class.getName())));
-
-        SimpleObject obj = simpleObject();
-
-        PortableObject po = marshal(obj, marsh);
-
-        PortableObject po0 = marshalUnmarshal(po, marsh);
-
-        assertTrue(po.hasField("b"));
-        assertTrue(po.hasField("s"));
-        assertTrue(po.hasField("i"));
-        assertTrue(po.hasField("l"));
-        assertTrue(po.hasField("f"));
-        assertTrue(po.hasField("d"));
-        assertTrue(po.hasField("c"));
-        assertTrue(po.hasField("bool"));
-
-        assertFalse(po.hasField("no_such_field"));
-
-        assertEquals(obj, po.deserialize());
-        assertEquals(obj, po0.deserialize());
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testEnum() throws Exception {
-        PortableMarshaller marsh = new PortableMarshaller();
-
-        marsh.setClassNames(Arrays.asList(TestEnum.class.getName()));
-
-        assertEquals(TestEnum.B, marshalUnmarshal(TestEnum.B, marsh));
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testUseTimestampFlag() throws Exception {
-        PortableTypeConfiguration cfg1 = new PortableTypeConfiguration(DateClass1.class.getName());
-
-        PortableTypeConfiguration cfg2 = new PortableTypeConfiguration(DateClass2.class.getName());
-
-        cfg2.setUseTimestamp(false);
-
-        PortableMarshaller marsh = new PortableMarshaller();
-
-        marsh.setTypeConfigurations(Arrays.asList(cfg1, cfg2));
-
-        Date date = new Date();
-        Timestamp ts = new Timestamp(System.currentTimeMillis());
-
-        DateClass1 obj1 = new DateClass1();
-        obj1.date = date;
-        obj1.ts = ts;
-
-        DateClass2 obj2 = new DateClass2();
-        obj2.date = date;
-        obj2.ts = ts;
-
-        PortableObject po1 = marshal(obj1, marsh);
-
-        assertEquals(date, po1.field("date"));
-        assertEquals(Timestamp.class, po1.field("date").getClass());
-        assertEquals(ts, po1.field("ts"));
-
-        PortableObject po2 = marshal(obj2, marsh);
-
-        assertEquals(date, po2.field("date"));
-        assertEquals(Date.class, po2.field("date").getClass());
-        assertEquals(new Date(ts.getTime()), po2.field("ts"));
-        assertEquals(Date.class, po2.field("ts").getClass());
-
-        obj1 = po1.deserialize();
-        assertEquals(date, obj1.date);
-        assertEquals(Date.class, obj1.date.getClass());
-        assertEquals(ts, obj1.ts);
-
-        obj2 = po2.deserialize();
-        assertEquals(date, obj2.date);
-        assertEquals(Date.class, obj2.date.getClass());
-        assertEquals(ts, obj2.ts);
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testSimpleObject() throws Exception {
-        PortableMarshaller marsh = new PortableMarshaller();
-
-        marsh.setTypeConfigurations(Arrays.asList(
-            new PortableTypeConfiguration(SimpleObject.class.getName())
-        ));
-
-        SimpleObject obj = simpleObject();
-
-        PortableObject po = marshal(obj, marsh);
-
-        assertEquals(obj.hashCode(), po.hashCode());
-
-        assertEquals(obj, po.deserialize());
-
-        assertEquals(obj.b, (byte)po.field("b"));
-        assertEquals(obj.s, (short)po.field("s"));
-        assertEquals(obj.i, (int)po.field("i"));
-        assertEquals(obj.l, (long)po.field("l"));
-        assertEquals(obj.f, (float)po.field("f"), 0);
-        assertEquals(obj.d, (double)po.field("d"), 0);
-        assertEquals(obj.c, (char)po.field("c"));
-        assertEquals(obj.bool, (boolean)po.field("bool"));
-        assertEquals(obj.str, po.field("str"));
-        assertEquals(obj.uuid, po.field("uuid"));
-        assertEquals(obj.date, po.field("date"));
-        assertEquals(Date.class, obj.date.getClass());
-        assertEquals(obj.ts, po.field("ts"));
-        assertArrayEquals(obj.bArr, (byte[])po.field("bArr"));
-        assertArrayEquals(obj.sArr, (short[])po.field("sArr"));
-        assertArrayEquals(obj.iArr, (int[])po.field("iArr"));
-        assertArrayEquals(obj.lArr, (long[])po.field("lArr"));
-        assertArrayEquals(obj.fArr, (float[])po.field("fArr"), 0);
-        assertArrayEquals(obj.dArr, (double[])po.field("dArr"), 0);
-        assertArrayEquals(obj.cArr, (char[])po.field("cArr"));
-        assertBooleanArrayEquals(obj.boolArr, (boolean[])po.field("boolArr"));
-        assertArrayEquals(obj.strArr, (String[])po.field("strArr"));
-        assertArrayEquals(obj.uuidArr, (UUID[])po.field("uuidArr"));
-        assertArrayEquals(obj.dateArr, (Date[])po.field("dateArr"));
-        assertArrayEquals(obj.objArr, (Object[])po.field("objArr"));
-        assertEquals(obj.col, po.field("col"));
-        assertEquals(obj.map, po.field("map"));
-        assertEquals(new Integer(obj.enumVal.ordinal()), new Integer(((Enum<?>)po.field("enumVal")).ordinal()));
-        assertArrayEquals(ordinals(obj.enumArr), ordinals((Enum<?>[])po.field("enumArr")));
-        assertNull(po.field("unknown"));
-
-        PortableObject innerPo = po.field("inner");
-
-        assertEquals(obj.inner, innerPo.deserialize());
-
-        assertEquals(obj.inner.b, (byte)innerPo.field("b"));
-        assertEquals(obj.inner.s, (short)innerPo.field("s"));
-        assertEquals(obj.inner.i, (int)innerPo.field("i"));
-        assertEquals(obj.inner.l, (long)innerPo.field("l"));
-        assertEquals(obj.inner.f, (float)innerPo.field("f"), 0);
-        assertEquals(obj.inner.d, (double)innerPo.field("d"), 0);
-        assertEquals(obj.inner.c, (char)innerPo.field("c"));
-        assertEquals(obj.inner.bool, (boolean)innerPo.field("bool"));
-        assertEquals(obj.inner.str, innerPo.field("str"));
-        assertEquals(obj.inner.uuid, innerPo.field("uuid"));
-        assertEquals(obj.inner.date, innerPo.field("date"));
-        assertEquals(Date.class, obj.inner.date.getClass());
-        assertEquals(obj.inner.ts, innerPo.field("ts"));
-        assertArrayEquals(obj.inner.bArr, (byte[])innerPo.field("bArr"));
-        assertArrayEquals(obj.inner.sArr, (short[])innerPo.field("sArr"));
-        assertArrayEquals(obj.inner.iArr, (int[])innerPo.field("iArr"));
-        assertArrayEquals(obj.inner.lArr, (long[])innerPo.field("lArr"));
-        assertArrayEquals(obj.inner.fArr, (float[])innerPo.field("fArr"), 0);
-        assertArrayEquals(obj.inner.dArr, (double[])innerPo.field("dArr"), 0);
-        assertArrayEquals(obj.inner.cArr, (char[])innerPo.field("cArr"));
-        assertBooleanArrayEquals(obj.inner.boolArr, (boolean[])innerPo.field("boolArr"));
-        assertArrayEquals(obj.inner.strArr, (String[])innerPo.field("strArr"));
-        assertArrayEquals(obj.inner.uuidArr, (UUID[])innerPo.field("uuidArr"));
-        assertArrayEquals(obj.inner.dateArr, (Date[])innerPo.field("dateArr"));
-        assertArrayEquals(obj.inner.objArr, (Object[])innerPo.field("objArr"));
-        assertEquals(obj.inner.col, innerPo.field("col"));
-        assertEquals(obj.inner.map, innerPo.field("map"));
-        assertEquals(new Integer(obj.inner.enumVal.ordinal()),
-            new Integer(((Enum<?>)innerPo.field("enumVal")).ordinal()));
-        assertArrayEquals(ordinals(obj.inner.enumArr), ordinals((Enum<?>[])innerPo.field("enumArr")));
-        assertNull(innerPo.field("inner"));
-        assertNull(innerPo.field("unknown"));
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testPortable() throws Exception {
-        PortableMarshaller marsh = new PortableMarshaller();
-
-        marsh.setTypeConfigurations(Arrays.asList(
-            new PortableTypeConfiguration(SimpleObject.class.getName()),
-            new PortableTypeConfiguration(TestPortableObject.class.getName())
-        ));
-
-        TestPortableObject obj = portableObject();
-
-        PortableObject po = marshal(obj, marsh);
-
-        assertEquals(obj.hashCode(), po.hashCode());
-
-        assertEquals(obj, po.deserialize());
-
-        assertEquals(obj.b, (byte)po.field("_b"));
-        assertEquals(obj.s, (short)po.field("_s"));
-        assertEquals(obj.i, (int)po.field("_i"));
-        assertEquals(obj.l, (long)po.field("_l"));
-        assertEquals(obj.f, (float)po.field("_f"), 0);
-        assertEquals(obj.d, (double)po.field("_d"), 0);
-        assertEquals(obj.c, (char)po.field("_c"));
-        assertEquals(obj.bool, (boolean)po.field("_bool"));
-        assertEquals(obj.str, po.field("_str"));
-        assertEquals(obj.uuid, po.field("_uuid"));
-        assertEquals(obj.date, po.field("_date"));
-        assertEquals(obj.ts, po.field("_ts"));
-        assertArrayEquals(obj.bArr, (byte[])po.field("_bArr"));
-        assertArrayEquals(obj.sArr, (short[])po.field("_sArr"));
-        assertArrayEquals(obj.iArr, (int[])po.field("_iArr"));
-        assertArrayEquals(obj.lArr, (long[])po.field("_lArr"));
-        assertArrayEquals(obj.fArr, (float[])po.field("_fArr"), 0);
-        assertArrayEquals(obj.dArr, (double[])po.field("_dArr"), 0);
-        assertArrayEquals(obj.cArr, (char[])po.field("_cArr"));
-        assertBooleanArrayEquals(obj.boolArr, (boolean[])po.field("_boolArr"));
-        assertArrayEquals(obj.strArr, (String[])po.field("_strArr"));
-        assertArrayEquals(obj.uuidArr, (UUID[])po.field("_uuidArr"));
-        assertArrayEquals(obj.dateArr, (Date[])po.field("_dateArr"));
-        assertArrayEquals(obj.objArr, (Object[])po.field("_objArr"));
-        assertEquals(obj.col, po.field("_col"));
-        assertEquals(obj.map, po.field("_map"));
-        assertEquals(new Integer(obj.enumVal.ordinal()), new Integer(((Enum<?>)po.field("_enumVal")).ordinal()));
-        assertArrayEquals(ordinals(obj.enumArr), ordinals((Enum<?>[])po.field("_enumArr")));
-        assertNull(po.field("unknown"));
-
-        PortableObject simplePo = po.field("_simple");
-
-        assertEquals(obj.simple, simplePo.deserialize());
-
-        assertEquals(obj.simple.b, (byte)simplePo.field("b"));
-        assertEquals(obj.simple.s, (short)simplePo.field("s"));
-        assertEquals(obj.simple.i, (int)simplePo.field("i"));
-        assertEquals(obj.simple.l, (long)simplePo.field("l"));
-        assertEquals(obj.simple.f, (float)simplePo.field("f"), 0);
-        assertEquals(obj.simple.d, (double)simplePo.field("d"), 0);
-        assertEquals(obj.simple.c, (char)simplePo.field("c"));
-        assertEquals(obj.simple.bool, (boolean)simplePo.field("bool"));
-        assertEquals(obj.simple.str, simplePo.field("str"));
-        assertEquals(obj.simple.uuid, simplePo.field("uuid"));
-        assertEquals(obj.simple.date, simplePo.field("date"));
-        assertEquals(Date.class, obj.simple.date.getClass());
-        assertEquals(obj.simple.ts, simplePo.field("ts"));
-        assertArrayEquals(obj.simple.bArr, (byte[])simplePo.field("bArr"));
-        assertArrayEquals(obj.simple.sArr, (short[])simplePo.field("sArr"));
-        assertArrayEquals(obj.simple.iArr, (int[])simplePo.field("iArr"));
-        assertArrayEquals(obj.simple.lArr, (long[])simplePo.field("lArr"));
-        assertArrayEquals(obj.simple.fArr, (float[])simplePo.field("fArr"), 0);
-        assertArrayEquals(obj.simple.dArr, (double[])simplePo.field("dArr"), 0);
-        assertArrayEquals(obj.simple.cArr, (char[])simplePo.field("cArr"));
-        assertBooleanArrayEquals(obj.simple.boolArr, (boolean[])simplePo.field("boolArr"));
-        assertArrayEquals(obj.simple.strArr, (String[])simplePo.field("strArr"));
-        assertArrayEquals(obj.simple.uuidArr, (UUID[])simplePo.field("uuidArr"));
-        assertArrayEquals(obj.simple.dateArr, (Date[])simplePo.field("dateArr"));
-        assertArrayEquals(obj.simple.objArr, (Object[])simplePo.field("objArr"));
-        assertEquals(obj.simple.col, simplePo.field("col"));
-        assertEquals(obj.simple.map, simplePo.field("map"));
-        assertEquals(new Integer(obj.simple.enumVal.ordinal()),
-            new Integer(((Enum<?>)simplePo.field("enumVal")).ordinal()));
-        assertArrayEquals(ordinals(obj.simple.enumArr), ordinals((Enum<?>[])simplePo.field("enumArr")));
-        assertNull(simplePo.field("simple"));
-        assertNull(simplePo.field("portable"));
-        assertNull(simplePo.field("unknown"));
-
-        PortableObject portablePo = po.field("_portable");
-
-        assertEquals(obj.portable, portablePo.deserialize());
-
-        assertEquals(obj.portable.b, (byte)portablePo.field("_b"));
-        assertEquals(obj.portable.s, (short)portablePo.field("_s"));
-        assertEquals(obj.portable.i, (int)portablePo.field("_i"));
-        assertEquals(obj.portable.l, (long)portablePo.field("_l"));
-        assertEquals(obj.portable.f, (float)portablePo.field("_f"), 0);
-        assertEquals(obj.portable.d, (double)portablePo.field("_d"), 0);
-        assertEquals(obj.portable.c, (char)portablePo.field("_c"));
-        assertEquals(obj.portable.bool, (boolean)portablePo.field("_bool"));
-        assertEquals(obj.portable.str, portablePo.field("_str"));
-        assertEquals(obj.portable.uuid, portablePo.field("_uuid"));
-        assertEquals(obj.portable.date, portablePo.field("_date"));
-        assertEquals(obj.portable.ts, portablePo.field("_ts"));
-        assertArrayEquals(obj.portable.bArr, (byte[])portablePo.field("_bArr"));
-        assertArrayEquals(obj.portable.sArr, (short[])portablePo.field("_sArr"));
-        assertArrayEquals(obj.portable.iArr, (int[])portablePo.field("_iArr"));
-        assertArrayEquals(obj.portable.lArr, (long[])portablePo.field("_lArr"));
-        assertArrayEquals(obj.portable.fArr, (float[])portablePo.field("_fArr"), 0);
-        assertArrayEquals(obj.portable.dArr, (double[])portablePo.field("_dArr"), 0);
-        assertArrayEquals(obj.portable.cArr, (char[])portablePo.field("_cArr"));
-        assertBooleanArrayEquals(obj.portable.boolArr, (boolean[])portablePo.field("_boolArr"));
-        assertArrayEquals(obj.portable.strArr, (String[])portablePo.field("_strArr"));
-        assertArrayEquals(obj.portable.uuidArr, (UUID[])portablePo.field("_uuidArr"));
-        assertArrayEquals(obj.portable.dateArr, (Date[])portablePo.field("_dateArr"));
-        assertArrayEquals(obj.portable.objArr, (Object[])portablePo.field("_objArr"));
-        assertEquals(obj.portable.col, portablePo.field("_col"));
-        assertEquals(obj.portable.map, portablePo.field("_map"));
-        assertEquals(new Integer(obj.portable.enumVal.ordinal()),
-            new Integer(((Enum<?>)portablePo.field("_enumVal")).ordinal()));
-        assertArrayEquals(ordinals(obj.portable.enumArr), ordinals((Enum<?>[])portablePo.field("_enumArr")));
-        assertNull(portablePo.field("_simple"));
-        assertNull(portablePo.field("_portable"));
-        assertNull(portablePo.field("unknown"));
-    }
-
-    /**
-     * @param obj Simple object.
-     * @param po Portable object.
-     */
-    private void checkSimpleObjectData(SimpleObject obj, PortableObject po) {
-        assertEquals(obj.b, (byte)po.field("b"));
-        assertEquals(obj.s, (short)po.field("s"));
-        assertEquals(obj.i, (int)po.field("i"));
-        assertEquals(obj.l, (long)po.field("l"));
-        assertEquals(obj.f, (float)po.field("f"), 0);
-        assertEquals(obj.d, (double)po.field("d"), 0);
-        assertEquals(obj.c, (char)po.field("c"));
-        assertEquals(obj.bool, (boolean)po.field("bool"));
-        assertEquals(obj.str, po.field("str"));
-        assertEquals(obj.uuid, po.field("uuid"));
-        assertEquals(obj.date, po.field("date"));
-        assertEquals(Date.class, obj.date.getClass());
-        assertEquals(obj.ts, po.field("ts"));
-        assertArrayEquals(obj.bArr, (byte[])po.field("bArr"));
-        assertArrayEquals(obj.sArr, (short[])po.field("sArr"));
-        assertArrayEquals(obj.iArr, (int[])po.field("iArr"));
-        assertArrayEquals(obj.lArr, (long[])po.field("lArr"));
-        assertArrayEquals(obj.fArr, (float[])po.field("fArr"), 0);
-        assertArrayEquals(obj.dArr, (double[])po.field("dArr"), 0);
-        assertArrayEquals(obj.cArr, (char[])po.field("cArr"));
-        assertBooleanArrayEquals(obj.boolArr, (boolean[])po.field("boolArr"));
-        assertArrayEquals(obj.strArr, (String[])po.field("strArr"));
-        assertArrayEquals(obj.uuidArr, (UUID[])po.field("uuidArr"));
-        assertArrayEquals(obj.dateArr, (Date[])po.field("dateArr"));
-        assertArrayEquals(obj.objArr, (Object[])po.field("objArr"));
-        assertEquals(obj.col, po.field("col"));
-        assertEquals(obj.map, po.field("map"));
-        assertEquals(new Integer(obj.enumVal.ordinal()), new Integer(((Enum<?>)po.field("enumVal")).ordinal()));
-        assertArrayEquals(ordinals(obj.enumArr), ordinals((Enum<?>[])po.field("enumArr")));
-        assertNull(po.field("unknown"));
-
-        assertEquals(obj, po.deserialize());
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testInvalidClass() throws Exception {
-        byte[] arr = new byte[20];
-
-        arr[0] = 103;
-
-        U.intToBytes(Integer.reverseBytes(11111), arr, 2);
-
-        final PortableObject po = new PortableObjectImpl(initPortableContext(new PortableMarshaller()), arr, 0);
-
-        GridTestUtils.assertThrows(log, new Callable<Object>() {
-                                       @Override public Object call() throws Exception {
-                                           po.deserialize();
-
-                                           return null;
-                                       }
-                                   }, PortableInvalidClassException.class, "Unknown type ID: 11111"
-        );
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testClassWithoutPublicConstructor() throws Exception {
-        PortableMarshaller marsh = new PortableMarshaller();
-
-        marsh.setTypeConfigurations(Arrays.asList(
-                                        new PortableTypeConfiguration(NoPublicConstructor.class.getName()),
-                                        new PortableTypeConfiguration(NoPublicDefaultConstructor.class.getName()),
-                                        new PortableTypeConfiguration(ProtectedConstructor.class.getName()))
-        );
-
-        initPortableContext(marsh);
-
-        NoPublicConstructor npc = new NoPublicConstructor();
-        PortableObject npc2 = marshal(npc, marsh);
-
-        assertEquals("test", npc2.<NoPublicConstructor>deserialize().val);
-
-        NoPublicDefaultConstructor npdc = new NoPublicDefaultConstructor(239);
-        PortableObject npdc2 = marshal(npdc, marsh);
-
-        assertEquals(239, npdc2.<NoPublicDefaultConstructor>deserialize().val);
-
-        ProtectedConstructor pc = new ProtectedConstructor();
-        PortableObject pc2 = marshal(pc, marsh);
-
-        assertEquals(ProtectedConstructor.class, pc2.<ProtectedConstructor>deserialize().getClass());
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testCustomSerializer() throws Exception {
-        PortableMarshaller marsh = new PortableMarshaller();
-
-        PortableTypeConfiguration type =
-            new PortableTypeConfiguration(CustomSerializedObject1.class.getName());
-
-        type.setSerializer(new CustomSerializer1());
-
-        marsh.setTypeConfigurations(Arrays.asList(type));
-
-        CustomSerializedObject1 obj1 = new CustomSerializedObject1(10);
-
-        PortableObject po1 = marshal(obj1, marsh);
-
-        assertEquals(20, po1.<CustomSerializedObject1>deserialize().val);
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testCustomSerializerWithGlobal() throws Exception {
-        PortableMarshaller marsh = new PortableMarshaller();
-
-        marsh.setSerializer(new CustomSerializer1());
-
-        PortableTypeConfiguration type1 =
-            new PortableTypeConfiguration(CustomSerializedObject1.class.getName());
-        PortableTypeConfiguration type2 =
-            new PortableTypeConfiguration(CustomSerializedObject2.class.getName());
-
-        type2.setSerializer(new CustomSerializer2());
-
-        marsh.setTypeConfigurations(Arrays.asList(type1, type2));
-
-        CustomSerializedObject1 obj1 = new CustomSerializedObject1(10);
-
-        PortableObject po1 = marshal(obj1, marsh);
-
-        assertEquals(20, po1.<CustomSerializedObject1>deserialize().val);
-
-        CustomSerializedObject2 obj2 = new CustomSerializedObject2(10);
-
-        PortableObject po2 = marshal(obj2, marsh);
-
-        assertEquals(30, po2.<CustomSerializedObject2>deserialize().val);
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testCustomIdMapper() throws Exception {
-        PortableMarshaller marsh = new PortableMarshaller();
-
-        PortableTypeConfiguration type =
-            new PortableTypeConfiguration(CustomMappedObject1.class.getName());
-
-        type.setIdMapper(new PortableIdMapper() {
-            @Override public int typeId(String clsName) {
-                return 11111;
-            }
-
-            @Override public int fieldId(int typeId, String fieldName) {
-                assert typeId == 11111;
-
-                if ("val1".equals(fieldName))
-                    return 22222;
-                else if ("val2".equals(fieldName))
-                    return 33333;
-
-                assert false : "Unknown field: " + fieldName;
-
-                return 0;
-            }
-        });
-
-        marsh.setTypeConfigurations(Arrays.asList(type));
-
-        CustomMappedObject1 obj1 = new CustomMappedObject1(10, "str");
-
-        PortableObject po1 = marshal(obj1, marsh);
-
-        assertEquals(11111, po1.typeId());
-        assertEquals(22222, intFromPortable(po1, 18));
-        assertEquals(33333, intFromPortable(po1, 31));
-
-        assertEquals(10, po1.<CustomMappedObject1>deserialize().val1);
-        assertEquals("str", po1.<CustomMappedObject1>deserialize().val2);
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testCustomIdMapperWithGlobal() throws Exception {
-        PortableMarshaller marsh = new PortableMarshaller();
-
-        marsh.setIdMapper(new PortableIdMapper() {
-            @Override public int typeId(String clsName) {
-                return 11111;
-            }
-
-            @Override public int fieldId(int typeId, String fieldName) {
-                assert typeId == 11111;
-
-                if ("val1".equals(fieldName)) return 22222;
-                else if ("val2".equals(fieldName)) return 33333;
-
-                assert false : "Unknown field: " + fieldName;
-
-                return 0;
-            }
-        });
-
-        PortableTypeConfiguration type1 =
-            new PortableTypeConfiguration(CustomMappedObject1.class.getName());
-        PortableTypeConfiguration type2 =
-            new PortableTypeConfiguration(CustomMappedObject2.class.getName());
-
-        type2.setIdMapper(new PortableIdMapper() {
-            @Override public int typeId(String clsName) {
-                return 44444;
-            }
-
-            @Override public int fieldId(int typeId, String fieldName) {
-                assert typeId == 44444;
-
-                if ("val1".equals(fieldName)) return 55555;
-                else if ("val2".equals(fieldName)) return 66666;
-
-                assert false : "Unknown field: " + fieldName;
-
-                return 0;
-            }
-        });
-
-        marsh.setTypeConfigurations(Arrays.asList(type1, type2));
-
-        CustomMappedObject1 obj1 = new CustomMappedObject1(10, "str1");
-
-        PortableObject po1 = marshal(obj1, marsh);
-
-        assertEquals(11111, po1.typeId());
-        assertEquals(22222, intFromPortable(po1, 18));
-        assertEquals(33333, intFromPortable(po1, 31));
-
-        assertEquals(10, po1.<CustomMappedObject1>deserialize().val1);
-        assertEquals("str1", po1.<CustomMappedObject1>deserialize().val2);
-
-        CustomMappedObject2 obj2 = new CustomMappedObject2(20, "str2");
-
-        PortableObject po2 = marshal(obj2, marsh);
-
-        assertEquals(44444, po2.typeId());
-        assertEquals(55555, intFromPortable(po2, 18));
-        assertEquals(66666, intFromPortable(po2, 31));
-
-        assertEquals(20, po2.<CustomMappedObject2>deserialize().val1);
-        assertEquals("str2", po2.<CustomMappedObject2>deserialize().val2);
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testDynamicObject() throws Exception {
-        PortableMarshaller marsh = new PortableMarshaller();
-
-        marsh.setTypeConfigurations(Arrays.asList(
-            new PortableTypeConfiguration(DynamicObject.class.getName())
-        ));
-
-        PortableObject po1 = marshal(new DynamicObject(0, 10, 20, 30), marsh);
-
-        assertEquals(new Integer(10), po1.field("val1"));
-        assertEquals(null, po1.field("val2"));
-        assertEquals(null, po1.field("val3"));
-
-        DynamicObject do1 = po1.deserialize();
-
-        assertEquals(10, do1.val1);
-        assertEquals(0, do1.val2);
-        assertEquals(0, do1.val3);
-
-        PortableObject po2 = marshal(new DynamicObject(1, 10, 20, 30), marsh);
-
-        assertEquals(new Integer(10), po2.field("val1"));
-        assertEquals(new Integer(20), po2.field("val2"));
-        assertEquals(null, po2.field("val3"));
-
-        DynamicObject do2 = po2.deserialize();
-
-        assertEquals(10, do2.val1);
-        assertEquals(20, do2.val2);
-        assertEquals(0, do2.val3);
-
-        PortableObject po3 = marshal(new DynamicObject(2, 10, 20, 30), marsh);
-
-        assertEquals(new Integer(10), po3.field("val1"));
-        assertEquals(new Integer(20), po3.field("val2"));
-        assertEquals(new Integer(30), po3.field("val3"));
-
-        DynamicObject do3 = po3.deserialize();
-
-        assertEquals(10, do3.val1);
-        assertEquals(20, do3.val2);
-        assertEquals(30, do3.val3);
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testCycleLink() throws Exception {
-        PortableMarshaller marsh = new PortableMarshaller();
-
-        marsh.setTypeConfigurations(Arrays.asList(
-            new PortableTypeConfiguration(CycleLinkObject.class.getName())
-        ));
-
-        CycleLinkObject obj = new CycleLinkObject();
-
-        obj.self = obj;
-
-        PortableObject po = marshal(obj, marsh);
-
-        CycleLinkObject obj0 = po.deserialize();
-
-        assert obj0.self == obj0;
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testDetached() throws Exception {
-        PortableMarshaller marsh = new PortableMarshaller();
-
-        marsh.setTypeConfigurations(Arrays.asList(
-            new PortableTypeConfiguration(DetachedTestObject.class.getName()),
-            new PortableTypeConfiguration(DetachedInnerTestObject.class.getName())
-        ));
-
-        UUID id = UUID.randomUUID();
-
-        DetachedTestObject obj = marshal(new DetachedTestObject(
-            new DetachedInnerTestObject(null, id)), marsh).deserialize();
-
-        assertEquals(id, obj.inner1.id);
-        assertEquals(id, obj.inner4.id);
-
-        assert obj.inner1 == obj.inner4;
-
-        PortableObjectImpl innerPo = (PortableObjectImpl)obj.inner2;
-
-        assert innerPo.detached();
-
-        DetachedInnerTestObject inner = innerPo.deserialize();
-
-        assertEquals(id, inner.id);
-
-        PortableObjectImpl detachedPo = (PortableObjectImpl)innerPo.detach();
-
-        assert detachedPo.detached();
-
-        inner = detachedPo.deserialize();
-
-        assertEquals(id, inner.id);
-
-        innerPo = (PortableObjectImpl)obj.inner3;
-
-        assert innerPo.detached();
-
-        inner = innerPo.deserialize();
-
-        assertEquals(id, inner.id);
-        assertNotNull(inner.inner);
-
-        detachedPo = (PortableObjectImpl)innerPo.detach();
-
-        assert detachedPo.detached();
-
-        inner = innerPo.deserialize();
-
-        assertEquals(id, inner.id);
-        assertNotNull(inner.inner);
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testCollectionFields() throws Exception {
-        PortableMarshaller marsh = new PortableMarshaller();
-
-        marsh.setTypeConfigurations(Arrays.asList(
-            new PortableTypeConfiguration(CollectionFieldsObject.class.getName()),
-            new PortableTypeConfiguration(Key.class.getName()),
-            new PortableTypeConfiguration(Value.class.getName())
-        ));
-
-        Object[] arr = new Object[] {new Value(1), new Value(2), new Value(3)};
-        Collection<Value> col = Arrays.asList(new Value(4), new Value(5), new Value(6));
-        Map<Key, Value> map = F.asMap(new Key(10), new Value(10), new Key(20), new Value(20), new Key(30), new Value(30));
-
-        CollectionFieldsObject obj = new CollectionFieldsObject(arr, col, map);
-
-        PortableObject po = marshal(obj, marsh);
-
-        Object[] arr0 = po.field("arr");
-
-        assertEquals(3, arr0.length);
-
-        int i = 1;
-
-        for (Object valPo : arr0)
-            assertEquals(i++, ((PortableObject)valPo).<Value>deserialize().val);
-
-        Collection<PortableObject> col0 = po.field("col");
-
-        i = 4;
-
-        for (PortableObject valPo : col0)
-            assertEquals(i++, valPo.<Value>deserialize().val);
-
-        Map<PortableObject, PortableObject> map0 = po.field("map");
-
-        for (Map.Entry<PortableObject, PortableObject> e : map0.entrySet())
-            assertEquals(e.getKey().<Key>deserialize().key, e.getValue().<Value>deserialize().val);
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testDefaultMapping() throws Exception {
-        PortableMarshaller marsh1 = new PortableMarshaller();
-
-        PortableTypeConfiguration customMappingType =
-            new PortableTypeConfiguration(TestPortableObject.class.getName());
-
-        customMappingType.setIdMapper(new PortableIdMapper() {
-            @Override public int typeId(String clsName) {
-                String typeName;
-
-                try {
-                    Method mtd = PortableContext.class.getDeclaredMethod("typeName", String.class);
-
-                    mtd.setAccessible(true);
-
-                    typeName = (String)mtd.invoke(null, clsName);
-                }
-                catch (NoSuchMethodException | InvocationTargetException | IllegalAccessException e) {
-                    throw new RuntimeException(e);
-                }
-
-                return typeName.toLowerCase().hashCode();
-            }
-
-            @Override public int fieldId(int typeId, String fieldName) {
-                return fieldName.toLowerCase().hashCode();
-            }
-        });
-
-        marsh1.setTypeConfigurations(Arrays.asList(
-            new PortableTypeConfiguration(SimpleObject.class.getName()),
-            customMappingType
-        ));
-
-        TestPortableObject obj = portableObject();
-
-        PortableObjectImpl po = marshal(obj, marsh1);
-
-        PortableMarshaller marsh2 = new PortableMarshaller();
-
-        marsh2.setTypeConfigurations(Arrays.asList(
-            new PortableTypeConfiguration(SimpleObject.class.getName()),
-            new PortableTypeConfiguration(TestPortableObject.class.getName())
-        ));
-
-        PortableContext ctx = initPortableContext(marsh2);
-
-        po.context(ctx);
-
-        assertEquals(obj, po.deserialize());
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testTypeNames() throws Exception {
-        PortableMarshaller marsh = new PortableMarshaller();
-
-        PortableTypeConfiguration customType1 = new PortableTypeConfiguration(Value.class.getName());
-
-        customType1.setIdMapper(new PortableIdMapper() {
-            @Override public int typeId(String clsName) {
-                return 300;
-            }
-
-            @Override public int fieldId(int typeId, String fieldName) {
-                return 0;
-            }
-        });
-
-        PortableTypeConfiguration customType2 = new PortableTypeConfiguration("org.gridgain.NonExistentClass1");
-
-        customType2.setIdMapper(new PortableIdMapper() {
-            @Override public int typeId(String clsName) {
-                return 400;
-            }
-
-            @Override public int fieldId(int typeId, String fieldName) {
-                return 0;
-            }
-        });
-
-        PortableTypeConfiguration customType3 = new PortableTypeConfiguration("NonExistentClass2");
-
-        customType3.setIdMapper(new PortableIdMapper() {
-            @Override public int typeId(String clsName) {
-                return 500;
-            }
-
-            @Override public int fieldId(int typeId, String fieldName) {
-                return 0;
-            }
-        });
-
-        PortableTypeConfiguration customType4 = new PortableTypeConfiguration("NonExistentClass5");
-
-        customType4.setIdMapper(new PortableIdMapper() {
-            @Override public int typeId(String clsName) {
-                return 0;
-            }
-
-            @Override public int fieldId(int typeId, String fieldName) {
-                return 0;
-            }
-        });
-
-        marsh.setTypeConfigurations(Arrays.asList(
-            new PortableTypeConfiguration(Key.class.getName()),
-            new PortableTypeConfiguration("org.gridgain.NonExistentClass3"),
-            new PortableTypeConfiguration("NonExistentClass4"),
-            customType1,
-            customType2,
-            customType3,
-            customType4
-        ));
-
-        PortableContext ctx = initPortableContext(marsh);
-
-        assertEquals("notconfiguredclass".hashCode(), ctx.typeId("NotConfiguredClass"));
-        assertEquals("key".hashCode(), ctx.typeId("Key"));
-        assertEquals("nonexistentclass3".hashCode(), ctx.typeId("NonExistentClass3"));
-        assertEquals("nonexistentclass4".hashCode(), ctx.typeId("NonExistentClass4"));
-        assertEquals(300, ctx.typeId(getClass().getSimpleName() + "$Value"));
-        assertEquals(400, ctx.typeId("NonExistentClass1"));
-        assertEquals(500, ctx.typeId("NonExistentClass2"));
-        assertEquals("nonexistentclass5".hashCode(), ctx.typeId("NonExistentClass5"));
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testFieldIdMapping() throws Exception {
-        PortableMarshaller marsh = new PortableMarshaller();
-
-        PortableTypeConfiguration customType1 = new PortableTypeConfiguration(Value.class.getName());
-
-        customType1.setIdMapper(new PortableIdMapper() {
-            @Override public int typeId(String clsName) {
-                return 300;
-            }
-
-            @Override public int fieldId(int typeId, String fieldName) {
-                switch (fieldName) {
-                    case "val1":
-                        return 301;
-
-                    case "val2":
-                        return 302;
-
-                    default:
-                        return 0;
-                }
-            }
-        });
-
-        PortableTypeConfiguration customType2 = new PortableTypeConfiguration("NonExistentClass1");
-
-        customType2.setIdMapper(new PortableIdMapper() {
-            @Override public int typeId(String clsName) {
-                return 400;
-            }
-
-            @Override public int fieldId(int typeId, String fieldName) {
-                switch (fieldName) {
-                    case "val1":
-                        return 401;
-
-                    case "val2":
-                        return 402;
-
-                    default:
-                        return 0;
-                }
-            }
-        });
-
-        marsh.setTypeConfigurations(Arrays.asList(new PortableTypeConfiguration(Key.class.getName()),
-                                                  new PortableTypeConfiguration("NonExistentClass2"),
-                                                  customType1,
-                                                  customType2));
-
-        PortableContext ctx = initPortableContext(marsh);
-
-        assertEquals("val".hashCode(), ctx.fieldId("key".hashCode(), "val"));
-        assertEquals("val".hashCode(), ctx.fieldId("nonexistentclass2".hashCode(), "val"));
-        assertEquals("val".hashCode(), ctx.fieldId("notconfiguredclass".hashCode(), "val"));
-        assertEquals(301, ctx.fieldId(300, "val1"));
-        assertEquals(302, ctx.fieldId(300, "val2"));
-        assertEquals("val3".hashCode(), ctx.fieldId(300, "val3"));
-        assertEquals(401, ctx.fieldId(400, "val1"));
-        assertEquals(402, ctx.fieldId(400, "val2"));
-        assertEquals("val3".hashCode(), ctx.fieldId(400, "val3"));
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testDuplicateTypeId() throws Exception {
-        final PortableMarshaller marsh = new PortableMarshaller();
-
-        PortableTypeConfiguration customType1 = new PortableTypeConfiguration("org.gridgain.Class1");
-
-        customType1.setIdMapper(new PortableIdMapper() {
-            @Override public int typeId(String clsName) {
-                return 100;
-            }
-
-            @Override public int fieldId(int typeId, String fieldName) {
-                return 0;
-            }
-        });
-
-        PortableTypeConfiguration customType2 = new PortableTypeConfiguration("org.gridgain.Class2");
-
-        customType2.setIdMapper(new PortableIdMapper() {
-            @Override public int typeId(String clsName) {
-                return 100;
-            }
-
-            @Override public int fieldId(int typeId, String fieldName) {
-                return 0;
-            }
-        });
-
-        marsh.setTypeConfigurations(Arrays.asList(customType1, customType2));
-
-        try {
-            initPortableContext(marsh);
-        }
-        catch (IgniteCheckedException e) {
-            assertEquals("Duplicate type ID [clsName=org.gridgain.Class1, id=100]",
-                e.getCause().getCause().getMessage());
-
-            return;
-        }
-
-        assert false;
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testPortableCopy() throws Exception {
-        PortableMarshaller marsh = new PortableMarshaller();
-
-        marsh.setTypeConfigurations(Arrays.asList(
-            new PortableTypeConfiguration(SimpleObject.class.getName())
-        ));
-
-        SimpleObject obj = simpleObject();
-
-        final PortableObject po = marshal(obj, marsh);
-
-        PortableObject copy = copy(po, null);
-
-        assertEquals(obj, copy.deserialize());
-
-        copy = copy(po, new HashMap<String, Object>());
-
-        assertEquals(obj, copy.deserialize());
-
-        Map<String, Object> map = new HashMap<>(1, 1.0f);
-
-        map.put("i", 3);
-
-        copy = copy(po, map);
-
-        assertEquals((byte)2, copy.<Byte>field("b").byteValue());
-        assertEquals((short)2, copy.<Short>field("s").shortValue());
-        assertEquals(3, copy.<Integer>field("i").intValue());
-        assertEquals(2L, copy.<Long>field("l").longValue());
-        assertEquals(2.2f, copy.<Float>field("f").floatValue(), 0);
-        assertEquals(2.2d, copy.<Double>field("d").doubleValue(), 0);
-        assertEquals((char)2, copy.<Character>field("c").charValue());
-        assertEquals(false, copy.<Boolean>field("bool").booleanValue());
-
-        SimpleObject obj0 = copy.deserialize();
-
-        assertEquals((byte)2, obj0.b);
-        assertEquals((short)2, obj0.s);
-        assertEquals(3, obj0.i);
-        assertEquals(2L, obj0.l);
-        assertEquals(2.2f, obj0.f, 0);
-        assertEquals(2.2d, obj0.d, 0);
-        assertEquals((char)2, obj0.c);
-        assertEquals(false, obj0.bool);
-
-        map = new HashMap<>(3, 1.0f);
-
-        map.put("b", (byte)3);
-        map.put("l", 3L);
-        map.put("bool", true);
-
-        copy = copy(po, map);
-
-        assertEquals((byte)3, copy.<Byte>field("b").byteValue());
-        assertEquals((short)2, copy.<Short>field("s").shortValue());
-        assertEquals(2, copy.<Integer>field("i").intValue());
-        assertEquals(3L, copy.<Long>field("l").longValue());
-        assertEquals(2.2f, copy.<Float>field("f").floatValue(), 0);
-        assertEquals(2.2d, copy.<Double>field("d").doubleValue(), 0);
-        assertEquals((char)2, copy.<Character>field("c").charValue());
-        assertEquals(true, copy.<Boolean>field("bool").booleanValue());
-
-        obj0 = copy.deserialize();
-
-        assertEquals((byte)3, obj0.b);
-        assertEquals((short)2, obj0.s);
-        assertEquals(2, obj0.i);
-        assertEquals(3L, obj0.l);
-        assertEquals(2.2f, obj0.f, 0);
-        assertEquals(2.2d, obj0.d, 0);
-        assertEquals((char)2, obj0.c);
-        assertEquals(true, obj0.bool);
-
-        map = new HashMap<>(8, 1.0f);
-
-        map.put("b", (byte)3);
-        map.put("s", (short)3);
-        map.put("i", 3);
-        map.put("l", 3L);
-        map.put("f", 3.3f);
-        map.put("d", 3.3d);
-        map.put("c", (char)3);
-        map.put("bool", true);
-
-        copy = copy(po, map);
-
-        assertEquals((byte)3, copy.<Byte>field("b").byteValue());
-        assertEquals((short)3, copy.<Short>field("s").shortValue());
-        assertEquals(3, copy.<Integer>field("i").intValue());
-        assertEquals(3L, copy.<Long>field("l").longValue());
-        assertEquals(3.3f, copy.<Float>field("f").floatValue(), 0);
-        assertEquals(3.3d, copy.<Double>field("d").doubleValue(), 0);
-        assertEquals((char)3, copy.<Character>field("c").charValue());
-        assertEquals(true, copy.<Boolean>field("bool").booleanValue());
-
-        obj0 = copy.deserialize();
-
-        assertEquals((byte)3, obj0.b);
-        assertEquals((short)3, obj0.s);
-        assertEquals(3, obj0.i);
-        assertEquals(3L, obj0.l);
-        assertEquals(3.3f, obj0.f, 0);
-        assertEquals(3.3d, obj0.d, 0);
-        assertEquals((char)3, obj0.c);
-        assertEquals(true, obj0.bool);
-
-//        GridTestUtils.assertThrows(
-//            log,
-//            new Callable<Object>() {
-//                @Override public Object call() throws Exception {
-//                    po.copy(F.<String, Object>asMap("i", false));
-//
-//                    return null;
-//                }
-//            },
-//            PortableException.class,
-//            "Invalid value type for field: i"
-//        );
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testPortableCopyString() throws Exception {
-        PortableMarshaller marsh = new PortableMarshaller();
-
-        marsh.setTypeConfigurations(Arrays.asList(
-            new PortableTypeConfiguration(SimpleObject.class.getName())
-        ));
-
-        SimpleObject obj = simpleObject();
-
-        PortableObject po = marshal(obj, marsh);
-
-        PortableObject copy = copy(po, F.<String, Object>asMap("str", "str3"));
-
-        assertEquals("str3", copy.<String>field("str"));
-
-        SimpleObject obj0 = copy.deserialize();
-
-        assertEquals("str3", obj0.str);
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testPortableCopyUuid() throws Exception {
-        PortableMarshaller marsh = new PortableMarshaller();
-
-        marsh.setTypeConfigurations(Arrays.asList(
-            new PortableTypeConfiguration(SimpleObject.class.getName())
-        ));
-
-        SimpleObject obj = simpleObject();
-
-        PortableObject po = marshal(obj, marsh);
-
-        UUID uuid = UUID.randomUUID();
-
-        PortableObject copy = copy(po, F.<String, Object>asMap("uuid", uuid));
-
-        assertEquals(uuid, copy.<UUID>field("uuid"));
-
-        SimpleObject obj0 = copy.deserialize();
-
-        assertEquals(uuid, obj0.uuid);
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testPortableCopyByteArray() throws Exception {
-        PortableMarshaller marsh = new PortableMarshaller();
-
-        marsh.setTypeConfigurations(Arrays.asList(
-            new PortableTypeConfiguration(SimpleObject.class.getName())
-        ));
-
-        SimpleObject obj = simpleObject();
-
-        PortableObject po = marshal(obj, marsh);
-
-        PortableObject copy = copy(po, F.<String, Object>asMap("bArr", new byte[]{1, 2, 3}));
-
-        assertArrayEquals(new byte[] {1, 2, 3}, copy.<byte[]>field("bArr"));
-
-        SimpleObject obj0 = copy.deserialize();
-
-        assertArrayEquals(new byte[] {1, 2, 3}, obj0.bArr);
-    }
-
-    /**
-     * @param po Portable object.
-     * @param fields Fields.
-     * @return Copy.
-     */
-    private PortableObject copy(PortableObject po, Map<String, Object> fields) {
-        PortableBuilder builder = PortableBuilderImpl.wrap(po);
-
-        if (fields != null) {
-            for (Map.Entry<String, Object> e : fields.entrySet())
-                builder.setField(e.getKey(), e.getValue());
-        }
-
-        return builder.build();
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testPortableCopyShortArray() throws Exception {
-        PortableMarshaller marsh = new PortableMarshaller();
-
-        marsh.setTypeConfigurations(Arrays.asList(
-            new PortableTypeConfiguration(SimpleObject.class.getName())
-        ));
-
-        SimpleObject obj = simpleObject();
-
-        PortableObject po = marshal(obj, marsh);
-
-        PortableObject copy = copy(po, F.<String, Object>asMap("sArr", new short[]{1, 2, 3}));
-
-        assertArrayEquals(new short[] {1, 2, 3}, copy.<short[]>field("sArr"));
-
-        SimpleObject obj0 = copy.deserialize();
-
-        assertArrayEquals(new short[] {1, 2, 3}, obj0.sArr);
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testPortableCopyIntArray() throws Exception {
-        PortableMarshaller marsh = new PortableMarshaller();
-
-        marsh.setTypeConfigurations(Arrays.asList(
-            new PortableTypeConfiguration(SimpleObject.class.getName())
-        ));
-
-        SimpleObject obj = simpleObject();
-
-        PortableObject po = marshal(obj, marsh);
-
-        PortableObject copy = copy(po, F.<String, Object>asMap("iArr", new int[]{1, 2, 3}));
-
-        assertArrayEquals(new int[] {1, 2, 3}, copy.<int[]>field("iArr"));
-
-        SimpleObject obj0 = copy.deserialize();
-
-        assertArrayEquals(new int[] {1, 2, 3}, obj0.iArr);
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testPortableCopyLongArray() throws Exception {
-        PortableMarshaller marsh = new PortableMarshaller();
-
-        marsh.setTypeConfigurations(Arrays.asList(
-            new PortableTypeConfiguration(SimpleObject.class.getName())
-        ));
-
-        SimpleObject obj = simpleObject();
-
-        PortableObject po = marshal(obj, marsh);
-
-        PortableObject copy = copy(po, F.<String, Object>asMap("lArr", new long[]{1, 2, 3}));
-
-        assertArrayEquals(new long[] {1, 2, 3}, copy.<long[]>field("lArr"));
-
-        SimpleObject obj0 = copy.deserialize();
-
-        assertArrayEquals(new long[] {1, 2, 3}, obj0.lArr);
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testPortableCopyFloatArray() throws Exception {
-        PortableMarshaller marsh = new PortableMarshaller();
-
-        marsh.setTypeConfigurations(Arrays.asList(
-            new PortableTypeConfiguration(SimpleObject.class.getName())
-        ));
-
-        SimpleObject obj = simpleObject();
-
-        PortableObject po = marshal(obj, marsh);
-
-        PortableObject copy = copy(po, F.<String, Object>asMap("fArr", new float[]{1, 2, 3}));
-
-        assertArrayEquals(new float[] {1, 2, 3}, copy.<float[]>field("fArr"), 0);
-
-        SimpleObject obj0 = copy.deserialize();
-
-        assertArrayEquals(new float[] {1, 2, 3}, obj0.fArr, 0);
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testPortableCopyDoubleArray() throws Exception {
-        PortableMarshaller marsh = new PortableMarshaller();
-
-        marsh.setTypeConfigurations(Arrays.asList(
-            new PortableTypeConfiguration(SimpleObject.class.getName())
-        ));
-
-        SimpleObject obj = simpleObject();
-
-        PortableObject po = marshal(obj, marsh);
-
-        PortableObject copy = copy(po, F.<String, Object>asMap("dArr", new double[]{1, 2, 3}));
-
-        assertArrayEquals(new double[] {1, 2, 3}, copy.<double[]>field("dArr"), 0);
-
-        SimpleObject obj0 = copy.deserialize();
-
-        assertArrayEquals(new double[] {1, 2, 3}, obj0.dArr, 0);
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testPortableCopyCharArray() throws Exception {
-        PortableMarshaller marsh = new PortableMarshaller();
-
-        marsh.setTypeConfigurations(Arrays.asList(
-            new PortableTypeConfiguration(SimpleObject.class.getName())
-        ));
-
-        SimpleObject obj = simpleObject();
-
-        PortableObject po = marshal(obj, marsh);
-
-        PortableObject copy = copy(po, F.<String, Object>asMap("cArr", new char[]{1, 2, 3}));
-
-        assertArrayEquals(new char[]{1, 2, 3}, copy.<char[]>field("cArr"));
-
-        SimpleObject obj0 = copy.deserialize();
-
-        assertArrayEquals(new char[]{1, 2, 3}, obj0.cArr);
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testPortableCopyStringArray() throws Exception {
-        PortableMarshaller marsh = new PortableMarshaller();
-
-        marsh.setTypeConfigurations(Arrays.asList(
-            new PortableTypeConfiguration(SimpleObject.class.getName())
-        ));
-
-        SimpleObject obj = simpleObject();
-
-        PortableObject po = marshal(obj, marsh);
-
-        PortableObject copy = copy(po, F.<String, Object>asMap("strArr", new String[]{"str1", "str2"}));
-
-        assertArrayEquals(new String[]{"str1", "str2"}, copy.<String[]>field("strArr"));
-
-        SimpleObject obj0 = copy.deserialize();
-
-        assertArrayEquals(new String[]{"str1", "str2"}, obj0.strArr);
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testPortableCopyObject() throws Exception {
-        PortableMarshaller marsh = new PortableMarshaller();
-
-        marsh.setTypeConfigurations(Arrays.asList(
-            new PortableTypeConfiguration(SimpleObject.class.getName())
-        ));
-
-        SimpleObject obj = simpleObject();
-
-        PortableObject po = marshal(obj, marsh);
-
-        SimpleObject newObj = new SimpleObject();
-
-        newObj.i = 12345;
-        newObj.fArr = new float[] {5, 8, 0};
-        newObj.str = "newStr";
-
-        PortableObject copy = copy(po, F.<String, Object>asMap("inner", newObj));
-
-        assertEquals(newObj, copy.<PortableObject>field("inner").deserialize());
-
-        SimpleObject obj0 = copy.deserialize();
-
-        assertEquals(newObj, obj0.inner);
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testPortableCopyNonPrimitives() throws Exception {
-        PortableMarshaller marsh = new PortableMarshaller();
-
-        marsh.setTypeConfigurations(Arrays.asList(
-            new PortableTypeConfiguration(SimpleObject.class.getName())
-        ));
-
-        SimpleObject obj = simpleObject();
-
-        PortableObject po = marshal(obj, marsh);
-
-        Map<String, Object> map = new HashMap<>(3, 1.0f);
-
-        SimpleObject newObj = new SimpleObject();
-
-        newObj.i = 12345;
-        newObj.fArr = new float[] {5, 8, 0};
-        newObj.str = "newStr";
-
-        map.put("str", "str555");
-        map.put("inner", newObj);
-        map.put("bArr", new byte[]{6, 7, 9});
-
-        PortableObject copy = copy(po, map);
-
-        assertEquals("str555", copy.<String>field("str"));
-        assertEquals(newObj, copy.<PortableObject>field("inner").deserialize());
-        assertArrayEquals(new byte[]{6, 7, 9}, copy.<byte[]>field("bArr"));
-
-        SimpleObject obj0 = copy.deserialize();
-
-        assertEquals("str555", obj0.str);
-        assertEquals(newObj, obj0.inner);
-        assertArrayEquals(new byte[] {6, 7, 9}, obj0.bArr);
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testPortableCopyMixed() throws Exception {
-        PortableMarshaller marsh = new PortableMarshaller();
-
-        marsh.setTypeConfigurations(Arrays.asList(new PortableTypeConfiguration(SimpleObject.class.getName())));
-
-        SimpleObject obj = simpleObject();
-
-        PortableObject po = marshal(obj, marsh);
-
-        Map<String, Object> map = new HashMap<>(3, 1.0f);
-
-        SimpleObject newObj = new SimpleObject();
-
-        newObj.i = 12345;
-        newObj.fArr = new float[] {5, 8, 0};
-        newObj.str = "newStr";
-
-        map.put("i", 1234);
-        map.put("str", "str555");
-        map.put("inner", newObj);
-        map.put("s", (short)2323);
-        map.put("bArr", new byte[]{6, 7, 9});
-        map.put("b", (byte)111);
-
-        PortableObject copy = copy(po, map);
-
-        assertEquals(1234, copy.<Integer>field("i").intValue());
-        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"));
-        assertEquals((byte)111, copy.<Byte>field("b").byteValue());
-
-        SimpleObject obj0 = copy.deserialize();
-
-        assertEquals(1234, obj0.i);
-        assertEquals("str555", obj0.str);
-        assertEquals(newObj, obj0.inner);
-        assertEquals((short)2323, obj0.s);
-        assertArrayEquals(new byte[] {6, 7, 9}, obj0.bArr);
-        assertEquals((byte)111, obj0.b);
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testKeepDeserialized() throws Exception {
-        PortableMarshaller marsh = new PortableMarshaller();
-
-        marsh.setClassNames(Arrays.asList(SimpleObject.class.getName()));
-        marsh.setKeepDeserialized(true);
-
-        PortableObject po = marshal(simpleObject(), marsh);
-
-        assert po.deserialize() == po.deserialize();
-
-        marsh = new PortableMarshaller();
-
-        marsh.setClassNames(Arrays.asList(SimpleObject.class.getName()));
-        marsh.setKeepDeserialized(false);
-
-        po = marshal(simpleObject(), marsh);
-
-        assert po.deserialize() != po.deserialize();
-
-        marsh = new PortableMarshaller();
-
-        marsh.setKeepDeserialized(true);
-        marsh.setTypeConfigurations(Arrays.asList(
-            new PortableTypeConfiguration(SimpleObject.class.getName())));
-
-        po = marshal(simpleObject(), marsh);
-
-        assert po.deserialize() == po.deserialize();
-
-        marsh = new PortableMarshaller();
-
-        marsh.setKeepDeserialized(false);
-        marsh.setTypeConfigurations(Arrays.asList(
-            new PortableTypeConfiguration(SimpleObject.class.getName())));
-
-        po = marshal(simpleObject(), marsh);
-
-        assert po.deserialize() != po.deserialize();
-
-        marsh = new PortableMarshaller();
-
-        marsh.setKeepDeserialized(true);
-
-        PortableTypeConfiguration typeCfg = new PortableTypeConfiguration(SimpleObject.class.getName());
-
-        typeCfg.setKeepDeserialized(false);
-
-        marsh.setTypeConfigurations(Arrays.asList(typeCfg));
-
-        po = marshal(simpleObject(), marsh);
-
-        assert po.deserialize() != po.deserialize();
-
-        marsh = new PortableMarshaller();
-
-        marsh.setKeepDeserialized(false);
-
-        typeCfg = new PortableTypeConfiguration(SimpleObject.class.getName());
-
-        typeCfg.setKeepDeserialized(true);
-
-        marsh.setTypeConfigurations(Arrays.asList(typeCfg));
-
-        po = marshal(simpleObject(), marsh);
-
-        assert po.deserialize() == po.deserialize();
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testOffheapPortable() throws Exception {
-        PortableMarshaller marsh = new PortableMarshaller();
-
-        marsh.setTypeConfigurations(Arrays.asList(new PortableTypeConfiguration(SimpleObject.class.getName())));
-
-        PortableContext ctx = initPortableContext(marsh);
-
-        SimpleObject simpleObj = simpleObject();
-
-        PortableObjectImpl obj = marshal(simpleObj, marsh);
-
-        long ptr = 0;
-
-        long ptr1 = 0;
-
-        long ptr2 = 0;
-
-        try {
-            ptr = copyOffheap(obj);
-
-            PortableObjectOffheapImpl offheapObj = new PortableObjectOffheapImpl(ctx,
-                ptr,
-                0,
-                obj.array().length);
-
-            assertTrue(offheapObj.equals(offheapObj));
-            assertFalse(offheapObj.equals(null));
-            assertFalse(offheapObj.equals("str"));
-            assertTrue(offheapObj.equals(obj));
-            assertTrue(obj.equals(offheapObj));
-
-            ptr1 = copyOffheap(obj);
-
-            PortableObjectOffheapImpl offheapObj1 = new PortableObjectOffheapImpl(ctx,
-                ptr1,
-                0,
-                obj.array().length);
-
-            assertTrue(offheapObj.equals(offheapObj1));
-            assertTrue(offheapObj1.equals(offheapObj));
-
-            assertEquals(obj.typeId(), offheapObj.typeId());
-            assertEquals(obj.hashCode(), offheapObj.hashCode());
-
-            checkSimpleObjectData(simpleObj, offheapObj);
-
-            PortableObjectOffheapImpl innerOffheapObj = offheapObj.field("inner");
-
-            assertNotNull(innerOffheapObj);
-
-            checkSimpleObjectData(simpleObj.inner, innerOffheapObj);
-
-            obj = (PortableObjectImpl)offheapObj.heapCopy();
-
-            assertEquals(obj.typeId(), offheapObj.typeId());
-            assertEquals(obj.hashCode(), offheapObj.hashCode());
-
-            checkSimpleObjectData(simpleObj, obj);
-
-            PortableObjectImpl innerObj = obj.field("inner");
-
-            assertNotNull(innerObj);
-
-            checkSimpleObjectData(simpleObj.inner, innerObj);
-
-            simpleObj.d = 0;
-
-            obj = marshal(simpleObj, marsh);
-
-            assertFalse(offheapObj.equals(obj));
-            assertFalse(obj.equals(offheapObj));
-
-            ptr2 = copyOffheap(obj);
-
-            PortableObjectOffheapImpl offheapObj2 = new PortableObjectOffheapImpl(ctx,
-                ptr2,
-                0,
-                obj.array().length);
-
-            assertFalse(offheapObj.equals(offheapObj2));
-            assertFalse(offheapObj2.equals(offheapObj));
-        }
-        finally {
-            UNSAFE.freeMemory(ptr);
-
-            if (ptr1 > 0)
-                UNSAFE.freeMemory(ptr1);
-
-            if (ptr2 > 0)
-                UNSAFE.freeMemory(ptr2);
-        }
-    }
-
-    /**
-     *
-     */
-    public void testReadResolve() throws Exception {
-        PortableMarshaller marsh = new PortableMarshaller();
-
-        marsh.setClassNames(
-            Arrays.asList(MySingleton.class.getName(), SingletonMarker.class.getName()));
-
-        PortableObjectImpl portableObj = marshal(MySingleton.INSTANCE, marsh);
-
-        assertTrue(portableObj.array().length <= 1024); // Check that big string was not serialized.
-
-        MySingleton singleton = portableObj.deserialize();
-
-        assertSame(MySingleton.INSTANCE, singleton);
-    }
-
-    /**
-     *
-     */
-    public void testReadResolveOnPortableAware() throws Exception {
-        PortableMarshaller marsh = new PortableMarshaller();
-
-        marsh.setClassNames(Collections.singletonList(MyTestClass.class.getName()));
-
-        PortableObjectImpl portableObj = marshal(new MyTestClass(), marsh);
-
-        MyTestClass obj = portableObj.deserialize();
-
-        assertEquals("readResolve", obj.s);
-    }
-
-    /**
-     * @throws Exception If ecxeption thrown.
-     */
-    public void testDeclareReadResolveInParent() throws Exception {
-        PortableMarshaller marsh = new PortableMarshaller();
-
-        marsh.setClassNames(Arrays.asList(ChildPortable.class.getName()));
-
-        PortableObjectImpl portableObj = marshal(new ChildPortable(), marsh);
-
-        ChildPortable singleton = portableObj.deserialize();
-
-        assertNotNull(singleton.s);
-    }
-
-    /**
-     *
-     */
-    public void testDecimalFields() throws Exception {
-        PortableMarshaller marsh = new PortableMarshaller();
-
-        Collection<String> clsNames = new ArrayList<>();
-
-        clsNames.add(DecimalReflective.class.getName());
-        clsNames.add(DecimalMarshalAware.class.getName());
-
-        marsh.setClassNames(clsNames);
-
-        // 1. Test reflective stuff.
-        DecimalReflective obj1 = new DecimalReflective();
-
-        obj1.val = BigDecimal.ZERO;
-        obj1.valArr = new BigDecimal[] { BigDecimal.ONE, BigDecimal.TEN };
-
-        PortableObjectImpl portObj = marshal(obj1, marsh);
-
-        assertEquals(obj1.val, portObj.field("val"));
-        assertArrayEquals(obj1.valArr, portObj.<BigDecimal[]>field("valArr"));
-
-        assertEquals(obj1.val, portObj.<DecimalReflective>deserialize().val);
-        assertArrayEquals(obj1.valArr, portObj.<DecimalReflective>deserialize().valArr);
-
-        // 2. Test marshal aware stuff.
-        DecimalMarshalAware obj2 = new DecimalMarshalAware();
-
-        obj2.val = BigDecimal.ZERO;
-        obj2.valArr = new BigDecimal[] { BigDecimal.ONE, BigDecimal.TEN.negate() };
-        obj2.rawVal = BigDecimal.TEN;
-        obj2.rawValArr = new BigDecimal[] { BigDecimal.ZERO, BigDecimal.ONE };
-
-        portObj = marshal(obj2, marsh);
-
-        assertEquals(obj2.val, portObj.field("val"));
-        assertArrayEquals(obj2.valArr, portObj.<BigDecimal[]>field("valArr"));
-
-        assertEquals(obj2.val, portObj.<DecimalMarshalAware>deserialize().val);
-        assertArrayEquals(obj2.valArr, portObj.<DecimalMarshalAware>deserialize().valArr);
-        assertEquals(obj2.rawVal, portObj.<DecimalMarshalAware>deserialize().rawVal);
-        assertArrayEquals(obj2.rawValArr, portObj.<DecimalMarshalAware>deserialize().rawValArr);
-    }
-
-    /**
-     * @throws IgniteCheckedException If failed.
-     */
-    public void testFinalField() throws IgniteCheckedException {
-        PortableMarshaller marsh = new PortableMarshaller();
-
-        SimpleObjectWithFinal obj = new SimpleObjectWithFinal();
-
-        SimpleObjectWithFinal po0 = marshalUnmarshal(obj, marsh);
-
-        assertEquals(obj.time, po0.time);
-    }
-
-    /**
-     * @throws IgniteCheckedException If failed.
-     */
-    public void testThreadLocalArrayReleased() throws IgniteCheckedException {
-        // Checking the writer directly.
-        assertEquals(false, THREAD_LOCAL_ALLOC.isThreadLocalArrayAcquired());
-
-        try (PortableWriterExImpl writer = new PortableWriterExImpl(initPortableContext(new PortableMarshaller()), 0)) {
-            assertEquals(true, THREAD_LOCAL_ALLOC.isThreadLocalArrayAcquired());
-
-            writer.writeString("Thread local test");
-
-            writer.array();
-
-            assertEquals(true, THREAD_LOCAL_ALLOC.isThreadLocalArrayAcquired());
-        }
-
-        // Checking the portable marshaller.
-        assertEquals(false, THREAD_LOCAL_ALLOC.isThreadLocalArrayAcquired());
-
-        PortableMarshaller marsh = new PortableMarshaller();
-
-        initPortableContext(marsh);
-
-        marsh.marshal(new SimpleObject());
-
-        assertEquals(false, THREAD_LOCAL_ALLOC.isThreadLocalArrayAcquired());
-
-        // Checking the builder.
-        PortableBuilder builder = new PortableBuilderImpl(initPortableContext(new PortableMarshaller()),
-            "org.gridgain.foo.bar.TestClass");
-
-        builder.setField("a", "1");
-
-        PortableObject portableObj = builder.build();
-
-        assertEquals(false, THREAD_LOCAL_ALLOC.isThreadLocalArrayAcquired());
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testDuplicateName() throws Exception {
-        PortableMarshaller marsh = new PortableMarshaller();
-
-        initPortableContext(marsh);
-
-        Test1.Job job1 = new Test1().new Job();
-        Test2.Job job2 = new Test2().new Job();
-
-        marsh.marshal(job1);
-
-        try {
-            marsh.marshal(job2);
-        } catch (PortableException e) {
-            assertEquals(true, e.getMessage().contains("Failed to register class"));
-            return;
-        }
-
-        assert false;
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testClassFieldsMarshalling() throws Exception {
-        PortableMarshaller marsh = new PortableMarshaller();
-
-        initPortableContext(marsh);
-
-        ObjectWithClassFields obj = new ObjectWithClassFields();
-        obj.cls1 = GridPortableMarshallerSelfTest.class;
-
-        byte[] marshal = marsh.marshal(obj);
-
-        ObjectWithClassFields obj2 = marsh.unmarshal(marshal, null);
-
-        assertEquals(obj.cls1, obj2.cls1);
-        assertNull(obj2.cls2);
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testMarshallingThroughJdk() throws Exception {
-        PortableMarshaller marsh = new PortableMarshaller();
-
-        initPortableContext(marsh);
-
-        InetSocketAddress addr = new InetSocketAddress("192.168.0.2", 4545);
-
-        byte[] arr = marsh.marshal(addr);
-
-        InetSocketAddress addr2 = marsh.unmarshal(arr, null);
-
-        assertEquals(addr.getHostString(), addr2.getHostString());
-        assertEquals(addr.getPort(), addr2.getPort());
-
-        TestAddress testAddr = new TestAddress();
-        testAddr.addr = addr;
-        testAddr.str1 = "Hello World";
-
-        SimpleObject simpleObj = new SimpleObject();
-        simpleObj.c = 'g';
-        simpleObj.date = new Date();
-
-        testAddr.obj = simpleObj;
-
-        arr = marsh.marshal(testAddr);
-
-        TestAddress testAddr2 = marsh.unmarshal(arr, null);
-
-        assertEquals(testAddr.addr.getHostString(), testAddr2.addr.getHostString());
-        assertEquals(testAddr.addr.getPort(), testAddr2.addr.getPort());
-        assertEquals(testAddr.str1, testAddr2.str1);
-        assertEquals(testAddr.obj.c, testAddr2.obj.c);
-        assertEquals(testAddr.obj.date, testAddr2.obj.date);
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testPredefinedTypeIds() throws Exception {
-        PortableMarshaller marsh = new PortableMarshaller();
-
-        PortableContext pCtx = initPortableContext(marsh);
-
-        Field field = pCtx.getClass().getDeclaredField("predefinedTypeNames");
-
-        field.setAccessible(true);
-
-        Map<String, Integer> map = (Map<String, Integer>)field.get(pCtx);
-
-        assertTrue(map.size() > 0);
-
-        for (Map.Entry<String, Integer> entry : map.entrySet()) {
-            int id = entry.getValue();
-
-            if (id == GridPortableMarshaller.UNREGISTERED_TYPE_ID)
-                continue;
-
-            PortableClassDescriptor desc = pCtx.descriptorForTypeId(false, entry.getValue(), null);
-
-            assertEquals(desc.typeId(), pCtx.typeId(desc.describedClass().getName()));
-            assertEquals(desc.typeId(), pCtx.typeId(pCtx.typeName(desc.describedClass().getName())));
-        }
-    }
-
-    /**
-     * @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 {
-        private Class<?> cls1;
-
-        private Class<?> cls2;
-    }
-
-    /**
-     *
-     */
-    private static class TestAddress {
-        /** */
-        private SimpleObject obj;
-
-        /** */
-        private InetSocketAddress addr;
-
-        /** */
-        private String str1;
-    }
-
-    /**
-     *
-     */
-    private static class Test1 {
-        /**
-         *
-         */
-        private class Job {
-
-        }
-    }
-
-    /**
-     *
-     */
-    private static class Test2 {
-        /**
-         *
-         */
-        private class Job {
-
-        }
-    }
-
-    /**
-     * @param obj Object.
-     * @return Offheap address.
-     */
-    private long copyOffheap(PortableObjectImpl obj) {
-        byte[] arr = obj.array();
-
-        long ptr = UNSAFE.allocateMemory(arr.length);
-
-        UNSAFE.copyMemory(arr, BYTE_ARR_OFF, null, ptr, arr.length);
-
-        return ptr;
-    }
-
-    /**
-     * @param enumArr Enum array.
-     * @return Ordinals.
-     */
-    private <T extends Enum<?>> Integer[] ordinals(T[] enumArr) {
-        Integer[] ords = new Integer[enumArr.length];
-
-        for (int i = 0; i < enumArr.length; i++)
-            ords[i] = enumArr[i].ordinal();
-
-        return ords;
-    }
-
-    /**
-     * @param po Portable object.
-     * @param off Offset.
-     * @return Value.
-     */
-    private int intFromPortable(PortableObject po, int off) {
-        byte[] arr = U.field(po, "arr");
-
-        return Integer.reverseBytes(U.bytesToInt(arr, off));
-    }
-
-    /**
-     * @param obj Original object.
-     * @return Result object.
-     */
-    private <T> T marshalUnmarshal(T obj) throws IgniteCheckedException {
-        return marshalUnmarshal(obj, new PortableMarshaller());
-    }
-
-    /**
-     * @param obj Original object.
-     * @param marsh Marshaller.
-     * @return Result object.
-     */
-    private <T> T marshalUnmarshal(Object obj, PortableMarshaller marsh) throws IgniteCheckedException {
-        initPortableContext(marsh);
-
-        byte[] bytes = marsh.marshal(obj);
-
-        return marsh.unmarshal(bytes, null);
-    }
-
-    /**
-     * @param obj Object.
-     * @param marsh Marshaller.
-     * @return Portable object.
-     */
-    private <T> PortableObjectImpl marshal(T obj, PortableMarshaller marsh) throws IgniteCheckedException {
-        initPortableContext(marsh);
-
-        byte[] bytes = marsh.marshal(obj);
-
-        return new PortableObjectImpl(U.<GridPortableMarshaller>field(marsh, "impl").context(),
-            bytes, 0);
-    }
-
-    /**
-     * @return Portable context.
-     */
-    protected PortableContext initPortableContext(PortableMarshaller marsh) throws IgniteCheckedException {
-        PortableContext ctx = new PortableContext(META_HND, null);
-
-        marsh.setContext(new MarshallerContextTestImpl(null));
-
-        IgniteUtils.invoke(PortableMarshaller.class, marsh, "setPortableContext", ctx);
-
-        return ctx;
-    }
-
-    /**
-     * @param exp Expected.
-     * @param act Actual.
-     */
-    private void assertBooleanArrayEquals(boolean[] exp, boolean[] act) {
-        assertEquals(exp.length, act.length);
-
-        for (int i = 0; i < act.length; i++)
-            assertEquals(exp[i], act[i]);
-    }
-
-    /**
-     *
-     */
-    private static class SimpleObjectWithFinal {
-        /** */
-        private final long time = System.currentTimeMillis();
-    }
-
-    /**
-     * @return Simple object.
-     */
-    private SimpleObject simpleObject() {
-        SimpleObject inner = new SimpleObject();
-
-        inner.b = 1;
-        inner.s = 1;
-        inner.i = 1;
-        inner.l = 1;
-        inner.f = 1.1f;
-        inner.d = 1.1d;
-        inner.c = 1;
-        inner.bool = true;
-        inner.str = "str1";
-        inner.uuid = UUID.randomUUID();
-        inner.date = new Date();
-        inner.ts = new Timestamp(System.currentTimeMillis());
-        inner.bArr = new byte[] {1, 2, 3};
-        inner.sArr = new short[] {1, 2, 3};
-        inner.iArr = new int[] {1, 2, 3};
-        inner.lArr = new long[] {1, 2, 3};
-        inner.fArr = new float[] {1.1f, 2.2f, 3.3f};
-        inner.dArr = new double[] {1.1d, 2.2d, 3.3d};
-        inner.cArr = new char[] {1, 2, 3};
-        inner.boolArr = new boolean[] {true, false, true};
-        inner.strArr = new String[] {"str1", "str2", "str3"};
-        inner.uuidArr = new UUID[] {UUID.randomUUID(), UUID.randomUUID(), UUID.randomUUID()};
-        inner.dateArr = new Date[] {new Date(11111), new Date(22222), new Date(33333)};
-        inner.objArr = new Object[] {UUID.randomUUID(), UUID.randomUUID(), UUID.randomUUID()};
-        inner.col = new ArrayList<>();
-        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");
-        inner.col.add("str3");
-
-        inner.map.put(1, "str1");
-        inner.map.put(2, "str2");
-        inner.map.put(3, "str3");
-
-        inner.mEntry = inner.map.entrySet().iterator().next();
-
-        SimpleObject outer = new SimpleObject();
-
-        outer.b = 2;
-        outer.s = 2;
-        outer.i = 2;
-        outer.l = 2;
-        outer.f = 2.2f;
-        outer.d = 2.2d;
-        outer.c = 2;
-        outer.bool = false;
-        outer.str = "str2";
-        outer.uuid = UUID.randomUUID();
-        outer.date = new Date();
-        outer.ts = new Timestamp(System.currentTimeMillis());
-        outer.bArr = new byte[] {10, 20, 30};
-        outer.sArr = new short[] {10, 20, 30};
-        outer.iArr = new int[] {10, 20, 30};
-        outer.lArr = new long[] {10, 20, 30};
-        outer.fArr = new float[] {10.01f, 20.02f, 30.03f};
-        outer.dArr = new double[] {10.01d, 20.02d, 30.03d};
-        outer.cArr = new char[] {10, 20, 30};
-        outer.boolArr = new boolean[] {false, true, false};
-        outer.strArr = new String[] {"str10", "str20", "str30"};
-        outer.uuidArr = new UUID[] {UUID.randomUUID(), UUID.randomUUID(), UUID.randomUUID()};
-        outer.dateArr = new Date[] {new Date(44444), new Date(55555), new Date(66666)};
-        outer.objArr = new Object[] {UUID.randomUUID(), UUID.randomUUID(), UUID.randomUUID()};
-        outer.col = new ArrayList<>();
-        outer.map = new HashMap<>();
-        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");
-        outer.col.add("str6");
-
-        outer.map.put(4, "str4");
-        o

<TRUNCATED>

[46/55] [abbrv] 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/cb7d972e
Tree: http://git-wip-us.apache.org/repos/asf/ignite/tree/cb7d972e
Diff: http://git-wip-us.apache.org/repos/asf/ignite/diff/cb7d972e

Branch: refs/heads/ignite-1171
Commit: cb7d972e6dfe7db70ac42a9a1889a661a2c26819
Parents: 126e68b 6bd0f8f
Author: vozerov-gridgain <vo...@gridgain.com>
Authored: Tue Sep 15 17:03:19 2015 +0300
Committer: vozerov-gridgain <vo...@gridgain.com>
Committed: Tue Sep 15 17:03:19 2015 +0300

----------------------------------------------------------------------
 .../configuration/NearCacheConfiguration.java   | 18 ++++++-
 ...CacheScanPartitionQueryFallbackSelfTest.java | 54 ++++++++------------
 2 files changed, 37 insertions(+), 35 deletions(-)
----------------------------------------------------------------------



[19/55] [abbrv] 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/d39345b8
Tree: http://git-wip-us.apache.org/repos/asf/ignite/tree/d39345b8
Diff: http://git-wip-us.apache.org/repos/asf/ignite/diff/d39345b8

Branch: refs/heads/ignite-1171
Commit: d39345b874097d8f6852b35c9d47c8037c13d19b
Parents: c70680a cc0d1f5
Author: Alexey Goncharuk <al...@gmail.com>
Authored: Mon Sep 14 18:55:54 2015 -0700
Committer: Alexey Goncharuk <al...@gmail.com>
Committed: Mon Sep 14 18:55:54 2015 -0700

----------------------------------------------------------------------
 .../org/apache/ignite/internal/GridKernalContext.java     |  7 ++++---
 .../org/apache/ignite/internal/GridKernalContextImpl.java | 10 +++++-----
 .../java/org/apache/ignite/internal/GridLoggerProxy.java  |  6 ++++--
 .../java/org/apache/ignite/internal/IgniteKernal.java     |  6 +++---
 .../ignite/internal/executor/GridExecutorService.java     |  4 ++--
 .../managers/deployment/GridDeploymentStoreAdapter.java   |  4 ++--
 .../internal/processors/cache/GridCacheAdapter.java       |  4 ++--
 .../processors/cache/GridCacheClearAllRunnable.java       |  4 ++--
 .../ignite/internal/processors/cache/GridCacheLogger.java |  4 ++--
 .../internal/processors/cache/GridCacheSharedContext.java |  4 ++--
 .../datastructures/GridCacheAtomicLongImpl.java           |  4 ++--
 .../datastructures/GridCacheAtomicReferenceImpl.java      |  4 ++--
 .../datastructures/GridCacheAtomicSequenceImpl.java       |  4 ++--
 .../datastructures/GridCacheAtomicStampedImpl.java        |  4 ++--
 .../datastructures/GridCacheCountDownLatchImpl.java       |  4 ++--
 .../internal/processors/igfs/IgfsFragmentizerManager.java |  8 +++++---
 .../internal/processors/igfs/IgfsServerManager.java       |  5 +++--
 .../ignite/internal/processors/job/GridJobWorker.java     |  4 ++--
 .../ignite/internal/processors/task/GridTaskWorker.java   |  4 ++--
 19 files changed, 50 insertions(+), 44 deletions(-)
----------------------------------------------------------------------



[06/55] [abbrv] ignite git commit: ignite-1462: hid portable API in 1.4 release

Posted by ag...@apache.org.
http://git-wip-us.apache.org/repos/asf/ignite/blob/71379a80/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
deleted file mode 100644
index 7f23c1f..0000000
--- a/modules/core/src/test/java/org/apache/ignite/internal/portable/GridPortableBuilderSelfTest.java
+++ /dev/null
@@ -1,1021 +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.math.BigDecimal;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.List;
-import java.util.Map;
-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;
-import org.apache.ignite.internal.portable.mutabletest.GridPortableTestClasses.TestObjectOuter;
-import org.apache.ignite.internal.portable.mutabletest.GridPortableTestClasses.TestObjectPlainPortable;
-import org.apache.ignite.internal.processors.cache.portable.CacheObjectPortableProcessorImpl;
-import org.apache.ignite.internal.util.GridUnsafe;
-import org.apache.ignite.internal.util.typedef.F;
-import org.apache.ignite.marshaller.portable.PortableMarshaller;
-import org.apache.ignite.portable.PortableBuilder;
-import org.apache.ignite.portable.PortableIdMapper;
-import org.apache.ignite.portable.PortableMetadata;
-import org.apache.ignite.portable.PortableObject;
-import org.apache.ignite.portable.PortableTypeConfiguration;
-import org.apache.ignite.testframework.GridTestUtils;
-import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
-import sun.misc.Unsafe;
-
-/**
- * Portable builder test.
- */
-public class GridPortableBuilderSelfTest extends GridCommonAbstractTest {
-    /** */
-    private static final Unsafe UNSAFE = GridUnsafe.unsafe();
-
-    /** */
-    protected static final long BYTE_ARR_OFF = UNSAFE.arrayBaseOffset(byte[].class);
-
-    /** {@inheritDoc} */
-    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
-        IgniteConfiguration cfg = super.getConfiguration(gridName);
-
-        PortableMarshaller marsh = new PortableMarshaller();
-
-        marsh.setClassNames(Arrays.asList(Key.class.getName(), Value.class.getName(),
-            "org.gridgain.grid.internal.util.portable.mutabletest.*"));
-
-        PortableTypeConfiguration customIdMapper = new PortableTypeConfiguration();
-
-        customIdMapper.setClassName(CustomIdMapper.class.getName());
-        customIdMapper.setIdMapper(new PortableIdMapper() {
-            @Override public int typeId(String clsName) {
-                return ~PortableContext.DFLT_ID_MAPPER.typeId(clsName);
-            }
-
-            @Override public int fieldId(int typeId, String fieldName) {
-                return typeId + ~PortableContext.DFLT_ID_MAPPER.fieldId(typeId, fieldName);
-            }
-        });
-
-        marsh.setTypeConfigurations(Collections.singleton(customIdMapper));
-
-        marsh.setConvertStringToBytes(useUtf8());
-
-        cfg.setMarshaller(marsh);
-
-        return cfg;
-    }
-
-    /** {@inheritDoc} */
-    @Override protected void beforeTestsStarted() throws Exception {
-        startGrids(1);
-    }
-
-    /** {@inheritDoc} */
-    @Override protected void afterTestsStopped() throws Exception {
-        stopAllGrids();
-    }
-
-    /**
-     * @return Whether to use UTF8 strings.
-     */
-    protected boolean useUtf8() {
-        return true;
-    }
-
-    /**
-     *
-     */
-    public void testAllFieldsSerialization() {
-        TestObjectAllTypes obj = new TestObjectAllTypes();
-        obj.setDefaultData();
-        obj.enumArr = null;
-
-        TestObjectAllTypes deserialized = builder(toPortable(obj)).build().deserialize();
-
-        GridTestUtils.deepEquals(obj, deserialized);
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testByteField() throws Exception {
-        PortableBuilder builder = builder("Class");
-
-        builder.hashCode(100);
-
-        builder.setField("byteField", (byte)1);
-
-        PortableObject po = builder.build();
-
-        assertEquals("class".hashCode(), po.typeId());
-        assertEquals(100, po.hashCode());
-
-        assertEquals((byte)1, po.<Byte>field("byteField").byteValue());
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testShortField() throws Exception {
-        PortableBuilder builder = builder("Class");
-
-        builder.hashCode(100);
-
-        builder.setField("shortField", (short)1);
-
-        PortableObject po = builder.build();
-
-        assertEquals("class".hashCode(), po.typeId());
-        assertEquals(100, po.hashCode());
-
-        assertEquals((short)1, po.<Short>field("shortField").shortValue());
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testIntField() throws Exception {
-        PortableBuilder builder = builder("Class");
-
-        builder.hashCode(100);
-
-        builder.setField("intField", 1);
-
-        PortableObject po = builder.build();
-
-        assertEquals("class".hashCode(), po.typeId());
-        assertEquals(100, po.hashCode());
-
-        assertEquals(1, po.<Integer>field("intField").intValue());
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testLongField() throws Exception {
-        PortableBuilder builder = builder("Class");
-
-        builder.hashCode(100);
-
-        builder.setField("longField", 1L);
-
-        PortableObject po = builder.build();
-
-        assertEquals("class".hashCode(), po.typeId());
-        assertEquals(100, po.hashCode());
-
-        assertEquals(1L, po.<Long>field("longField").longValue());
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testFloatField() throws Exception {
-        PortableBuilder builder = builder("Class");
-
-        builder.hashCode(100);
-
-        builder.setField("floatField", 1.0f);
-
-        PortableObject po = builder.build();
-
-        assertEquals("class".hashCode(), po.typeId());
-        assertEquals(100, po.hashCode());
-
-        assertEquals(1.0f, po.<Float>field("floatField").floatValue(), 0);
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testDoubleField() throws Exception {
-        PortableBuilder builder = builder("Class");
-
-        builder.hashCode(100);
-
-        builder.setField("doubleField", 1.0d);
-
-        PortableObject po = builder.build();
-
-        assertEquals("class".hashCode(), po.typeId());
-        assertEquals(100, po.hashCode());
-
-        assertEquals(1.0d, po.<Double>field("doubleField").doubleValue(), 0);
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testCharField() throws Exception {
-        PortableBuilder builder = builder("Class");
-
-        builder.hashCode(100);
-
-        builder.setField("charField", (char)1);
-
-        PortableObject po = builder.build();
-
-        assertEquals("class".hashCode(), po.typeId());
-        assertEquals(100, po.hashCode());
-
-        assertEquals((char)1, po.<Character>field("charField").charValue());
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testBooleanField() throws Exception {
-        PortableBuilder builder = builder("Class");
-
-        builder.hashCode(100);
-
-        builder.setField("booleanField", true);
-
-        PortableObject po = builder.build();
-
-        assertEquals("class".hashCode(), po.typeId());
-        assertEquals(100, po.hashCode());
-
-        assertTrue(po.<Boolean>field("booleanField"));
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testDecimalField() throws Exception {
-        PortableBuilder builder = builder("Class");
-
-        builder.hashCode(100);
-
-        builder.setField("decimalField", BigDecimal.TEN);
-
-        PortableObject po = builder.build();
-
-        assertEquals("class".hashCode(), po.typeId());
-        assertEquals(100, po.hashCode());
-
-        assertEquals(BigDecimal.TEN, po.<String>field("decimalField"));
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testStringField() throws Exception {
-        PortableBuilder builder = builder("Class");
-
-        builder.hashCode(100);
-
-        builder.setField("stringField", "str");
-
-        PortableObject po = builder.build();
-
-        assertEquals("class".hashCode(), po.typeId());
-        assertEquals(100, po.hashCode());
-
-        assertEquals("str", po.<String>field("stringField"));
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testUuidField() throws Exception {
-        PortableBuilder builder = builder("Class");
-
-        builder.hashCode(100);
-
-        UUID uuid = UUID.randomUUID();
-
-        builder.setField("uuidField", uuid);
-
-        PortableObject po = builder.build();
-
-        assertEquals("class".hashCode(), po.typeId());
-        assertEquals(100, po.hashCode());
-
-        assertEquals(uuid, po.<UUID>field("uuidField"));
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testByteArrayField() throws Exception {
-        PortableBuilder builder = builder("Class");
-
-        builder.hashCode(100);
-
-        builder.setField("byteArrayField", new byte[] {1, 2, 3});
-
-        PortableObject po = builder.build();
-
-        assertEquals("class".hashCode(), po.typeId());
-        assertEquals(100, po.hashCode());
-
-        assertTrue(Arrays.equals(new byte[] {1, 2, 3}, po.<byte[]>field("byteArrayField")));
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testShortArrayField() throws Exception {
-        PortableBuilder builder = builder("Class");
-
-        builder.hashCode(100);
-
-        builder.setField("shortArrayField", new short[] {1, 2, 3});
-
-        PortableObject po = builder.build();
-
-        assertEquals("class".hashCode(), po.typeId());
-        assertEquals(100, po.hashCode());
-
-        assertTrue(Arrays.equals(new short[] {1, 2, 3}, po.<short[]>field("shortArrayField")));
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testIntArrayField() throws Exception {
-        PortableBuilder builder = builder("Class");
-
-        builder.hashCode(100);
-
-        builder.setField("intArrayField", new int[] {1, 2, 3});
-
-        PortableObject po = builder.build();
-
-        assertEquals("class".hashCode(), po.typeId());
-        assertEquals(100, po.hashCode());
-
-        assertTrue(Arrays.equals(new int[] {1, 2, 3}, po.<int[]>field("intArrayField")));
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testLongArrayField() throws Exception {
-        PortableBuilder builder = builder("Class");
-
-        builder.hashCode(100);
-
-        builder.setField("longArrayField", new long[] {1, 2, 3});
-
-        PortableObject po = builder.build();
-
-        assertEquals("class".hashCode(), po.typeId());
-        assertEquals(100, po.hashCode());
-
-        assertTrue(Arrays.equals(new long[] {1, 2, 3}, po.<long[]>field("longArrayField")));
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testFloatArrayField() throws Exception {
-        PortableBuilder builder = builder("Class");
-
-        builder.hashCode(100);
-
-        builder.setField("floatArrayField", new float[] {1, 2, 3});
-
-        PortableObject po = builder.build();
-
-        assertEquals("class".hashCode(), po.typeId());
-        assertEquals(100, po.hashCode());
-
-        assertTrue(Arrays.equals(new float[] {1, 2, 3}, po.<float[]>field("floatArrayField")));
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testDoubleArrayField() throws Exception {
-        PortableBuilder builder = builder("Class");
-
-        builder.hashCode(100);
-
-        builder.setField("doubleArrayField", new double[] {1, 2, 3});
-
-        PortableObject po = builder.build();
-
-        assertEquals("class".hashCode(), po.typeId());
-        assertEquals(100, po.hashCode());
-
-        assertTrue(Arrays.equals(new double[] {1, 2, 3}, po.<double[]>field("doubleArrayField")));
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testCharArrayField() throws Exception {
-        PortableBuilder builder = builder("Class");
-
-        builder.hashCode(100);
-
-        builder.setField("charArrayField", new char[] {1, 2, 3});
-
-        PortableObject po = builder.build();
-
-        assertEquals("class".hashCode(), po.typeId());
-        assertEquals(100, po.hashCode());
-
-        assertTrue(Arrays.equals(new char[] {1, 2, 3}, po.<char[]>field("charArrayField")));
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testBooleanArrayField() throws Exception {
-        PortableBuilder builder = builder("Class");
-
-        builder.hashCode(100);
-
-        builder.setField("booleanArrayField", new boolean[] {true, false});
-
-        PortableObject po = builder.build();
-
-        assertEquals("class".hashCode(), po.typeId());
-        assertEquals(100, po.hashCode());
-
-        boolean[] arr = po.field("booleanArrayField");
-
-        assertEquals(2, arr.length);
-
-        assertTrue(arr[0]);
-        assertFalse(arr[1]);
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testDecimalArrayField() throws Exception {
-        PortableBuilder builder = builder("Class");
-
-        builder.hashCode(100);
-
-        builder.setField("decimalArrayField", new BigDecimal[] {BigDecimal.ONE, BigDecimal.TEN});
-
-        PortableObject po = builder.build();
-
-        assertEquals("class".hashCode(), po.typeId());
-        assertEquals(100, po.hashCode());
-
-        assertTrue(Arrays.equals(new BigDecimal[] {BigDecimal.ONE, BigDecimal.TEN}, po.<String[]>field("decimalArrayField")));
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testStringArrayField() throws Exception {
-        PortableBuilder builder = builder("Class");
-
-        builder.hashCode(100);
-
-        builder.setField("stringArrayField", new String[] {"str1", "str2", "str3"});
-
-        PortableObject po = builder.build();
-
-        assertEquals("class".hashCode(), po.typeId());
-        assertEquals(100, po.hashCode());
-
-        assertTrue(Arrays.equals(new String[] {"str1", "str2", "str3"}, po.<String[]>field("stringArrayField")));
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testUuidArrayField() throws Exception {
-        PortableBuilder builder = builder("Class");
-
-        builder.hashCode(100);
-
-        UUID[] arr = new UUID[] {UUID.randomUUID(), UUID.randomUUID()};
-
-        builder.setField("uuidArrayField", arr);
-
-        PortableObject po = builder.build();
-
-        assertEquals("class".hashCode(), po.typeId());
-        assertEquals(100, po.hashCode());
-
-        assertTrue(Arrays.equals(arr, po.<UUID[]>field("uuidArrayField")));
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testObjectField() throws Exception {
-        PortableBuilder builder = builder("Class");
-
-        builder.hashCode(100);
-
-        builder.setField("objectField", new Value(1));
-
-        PortableObject po = builder.build();
-
-        assertEquals("class".hashCode(), po.typeId());
-        assertEquals(100, po.hashCode());
-
-        assertEquals(1, po.<PortableObject>field("objectField").<Value>deserialize().i);
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testObjectArrayField() throws Exception {
-        PortableBuilder builder = builder("Class");
-
-        builder.hashCode(100);
-
-        builder.setField("objectArrayField", new Value[] {new Value(1), new Value(2)});
-
-        PortableObject po = builder.build();
-
-        assertEquals("class".hashCode(), po.typeId());
-        assertEquals(100, po.hashCode());
-
-        Object[] arr = po.field("objectArrayField");
-
-        assertEquals(2, arr.length);
-
-        assertEquals(1, ((PortableObject)arr[0]).<Value>deserialize().i);
-        assertEquals(2, ((PortableObject)arr[1]).<Value>deserialize().i);
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testCollectionField() throws Exception {
-        PortableBuilder builder = builder("Class");
-
-        builder.hashCode(100);
-
-        builder.setField("collectionField", Arrays.asList(new Value(1), new Value(2)));
-
-        PortableObject po = builder.build();
-
-        assertEquals("class".hashCode(), po.typeId());
-        assertEquals(100, po.hashCode());
-
-        List<PortableObject> list = po.field("collectionField");
-
-        assertEquals(2, list.size());
-
-        assertEquals(1, list.get(0).<Value>deserialize().i);
-        assertEquals(2, list.get(1).<Value>deserialize().i);
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testMapField() throws Exception {
-        PortableBuilder builder = builder("Class");
-
-        builder.hashCode(100);
-
-        builder.setField("mapField", F.asMap(new Key(1), new Value(1), new Key(2), new Value(2)));
-
-        PortableObject po = builder.build();
-
-        assertEquals("class".hashCode(), po.typeId());
-        assertEquals(100, po.hashCode());
-
-        Map<PortableObject, PortableObject> map = po.field("mapField");
-
-        assertEquals(2, map.size());
-
-        for (Map.Entry<PortableObject, PortableObject> e : map.entrySet())
-            assertEquals(e.getKey().<Key>deserialize().i, e.getValue().<Value>deserialize().i);
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testSeveralFields() throws Exception {
-        PortableBuilder builder = builder("Class");
-
-        builder.hashCode(100);
-
-        builder.setField("i", 111);
-        builder.setField("f", 111.111f);
-        builder.setField("iArr", new int[] {1, 2, 3});
-        builder.setField("obj", new Key(1));
-        builder.setField("col", Arrays.asList(new Value(1), new Value(2)));
-
-        PortableObject po = builder.build();
-
-        assertEquals("class".hashCode(), po.typeId());
-        assertEquals(100, po.hashCode());
-
-        assertEquals(111, po.<Integer>field("i").intValue());
-        assertEquals(111.111f, po.<Float>field("f").floatValue(), 0);
-        assertTrue(Arrays.equals(new int[] {1, 2, 3}, po.<int[]>field("iArr")));
-        assertEquals(1, po.<PortableObject>field("obj").<Key>deserialize().i);
-
-        List<PortableObject> list = po.field("col");
-
-        assertEquals(2, list.size());
-
-        assertEquals(1, list.get(0).<Value>deserialize().i);
-        assertEquals(2, list.get(1).<Value>deserialize().i);
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testOffheapPortable() throws Exception {
-        PortableBuilder builder = builder("Class");
-
-        builder.hashCode(100);
-
-        builder.setField("i", 111);
-        builder.setField("f", 111.111f);
-        builder.setField("iArr", new int[] {1, 2, 3});
-        builder.setField("obj", new Key(1));
-        builder.setField("col", Arrays.asList(new Value(1), new Value(2)));
-
-        PortableObject po = builder.build();
-
-        byte[] arr = ((CacheObjectPortableProcessorImpl)(grid(0)).context().cacheObjects()).marshal(po);
-
-        long ptr = UNSAFE.allocateMemory(arr.length + 5);
-
-        try {
-            long ptr0 = ptr;
-
-            UNSAFE.putBoolean(null, ptr0++, false);
-
-            UNSAFE.putInt(ptr0, arr.length);
-
-            UNSAFE.copyMemory(arr, BYTE_ARR_OFF, null, ptr0 + 4, arr.length);
-
-            PortableObject offheapObj = (PortableObject)
-                ((CacheObjectPortableProcessorImpl)(grid(0)).context().cacheObjects()).unmarshal(ptr, false);
-
-            assertEquals(PortableObjectOffheapImpl.class, offheapObj.getClass());
-
-            assertEquals("class".hashCode(), offheapObj.typeId());
-            assertEquals(100, offheapObj.hashCode());
-
-            assertEquals(111, offheapObj.<Integer>field("i").intValue());
-            assertEquals(111.111f, offheapObj.<Float>field("f").floatValue(), 0);
-            assertTrue(Arrays.equals(new int[] {1, 2, 3}, offheapObj.<int[]>field("iArr")));
-            assertEquals(1, offheapObj.<PortableObject>field("obj").<Key>deserialize().i);
-
-            List<PortableObject> list = offheapObj.field("col");
-
-            assertEquals(2, list.size());
-
-            assertEquals(1, list.get(0).<Value>deserialize().i);
-            assertEquals(2, list.get(1).<Value>deserialize().i);
-
-            assertEquals(po, offheapObj);
-            assertEquals(offheapObj, po);
-        }
-        finally {
-            UNSAFE.freeMemory(ptr);
-        }
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testBuildAndDeserialize() throws Exception {
-        PortableBuilder builder = builder(Value.class.getName());
-
-        builder.hashCode(100);
-
-        builder.setField("i", 1);
-
-        PortableObject po = builder.build();
-
-        assertEquals("value".hashCode(), po.typeId());
-        assertEquals(100, po.hashCode());
-
-        assertEquals(1, po.<Value>deserialize().i);
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testMetaData2() throws Exception {
-        PortableBuilder builder = builder("org.test.MetaTest2");
-
-        builder.setField("objectField", "a", Object.class);
-
-        PortableObject po = builder.build();
-
-        PortableMetadata meta = po.metaData();
-
-        assertEquals("MetaTest2", meta.typeName());
-        assertEquals("Object", meta.fieldTypeName("objectField"));
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testMetaData() throws Exception {
-        PortableBuilder builder = builder("org.test.MetaTest");
-
-        builder.hashCode(100);
-
-        builder.setField("intField", 1);
-        builder.setField("byteArrayField", new byte[] {1, 2, 3});
-
-        PortableObject po = builder.build();
-
-        PortableMetadata meta = po.metaData();
-
-        assertEquals("MetaTest", meta.typeName());
-
-        Collection<String> fields = meta.fields();
-
-        assertEquals(2, fields.size());
-
-        assertTrue(fields.contains("intField"));
-        assertTrue(fields.contains("byteArrayField"));
-
-        assertEquals("int", meta.fieldTypeName("intField"));
-        assertEquals("byte[]", meta.fieldTypeName("byteArrayField"));
-
-        builder = builder("org.test.MetaTest");
-
-        builder.hashCode(100);
-
-        builder.setField("intField", 2);
-        builder.setField("uuidField", UUID.randomUUID());
-
-        po = builder.build();
-
-        meta = po.metaData();
-
-        assertEquals("MetaTest", meta.typeName());
-
-        fields = meta.fields();
-
-        assertEquals(3, fields.size());
-
-        assertTrue(fields.contains("intField"));
-        assertTrue(fields.contains("byteArrayField"));
-        assertTrue(fields.contains("uuidField"));
-
-        assertEquals("int", meta.fieldTypeName("intField"));
-        assertEquals("byte[]", meta.fieldTypeName("byteArrayField"));
-        assertEquals("UUID", meta.fieldTypeName("uuidField"));
-    }
-
-    /**
-     *
-     */
-    public void testGetFromCopiedObj() {
-        PortableObject objStr = builder(TestObjectAllTypes.class.getName()).setField("str", "aaa").build();
-
-        PortableBuilderImpl builder = builder(objStr);
-        assertEquals("aaa", builder.getField("str"));
-
-        builder.setField("str", "bbb");
-        assertEquals("bbb", builder.getField("str"));
-
-        assertNull(builder.getField("i_"));
-        assertEquals("bbb", builder.build().<TestObjectAllTypes>deserialize().str);
-    }
-
-    /**
-     *
-     */
-    public void testCopyFromInnerObjects() {
-        ArrayList<Object> list = new ArrayList<>();
-        list.add(new TestObjectAllTypes());
-        list.add(list.get(0));
-
-        TestObjectContainer c = new TestObjectContainer(list);
-
-        PortableBuilderImpl builder = builder(toPortable(c));
-        builder.<List>getField("foo").add("!!!");
-
-        PortableObject res = builder.build();
-
-        TestObjectContainer deserialized = res.deserialize();
-
-        List deserializedList = (List)deserialized.foo;
-
-        assertSame(deserializedList.get(0), deserializedList.get(1));
-        assertEquals("!!!", deserializedList.get(2));
-        assertTrue(deserializedList.get(0) instanceof TestObjectAllTypes);
-    }
-
-    /**
-     *
-     */
-    public void testSetPortableObject() {
-        PortableObject portableObj = builder(TestObjectContainer.class.getName())
-            .setField("foo", toPortable(new TestObjectAllTypes()))
-            .build();
-
-        assertTrue(portableObj.<TestObjectContainer>deserialize().foo instanceof TestObjectAllTypes);
-    }
-
-    /**
-     *
-     */
-    public void testPlainPortableObjectCopyFrom() {
-        TestObjectPlainPortable obj = new TestObjectPlainPortable(toPortable(new TestObjectAllTypes()));
-
-        PortableBuilderImpl builder = builder(toPortable(obj));
-        assertTrue(builder.getField("plainPortable") instanceof PortableObject);
-
-        TestObjectPlainPortable deserialized = builder.build().deserialize();
-        assertTrue(deserialized.plainPortable instanceof PortableObject);
-    }
-
-    /**
-     *
-     */
-    public void testRemoveFromNewObject() {
-        PortableBuilder builder = builder(TestObjectAllTypes.class.getName());
-
-        builder.setField("str", "a");
-
-        builder.removeField("str");
-
-        assertNull(builder.build().<TestObjectAllTypes>deserialize().str);
-    }
-
-    /**
-     *
-     */
-    public void testRemoveFromExistingObject() {
-        TestObjectAllTypes obj = new TestObjectAllTypes();
-        obj.setDefaultData();
-        obj.enumArr = null;
-
-        PortableBuilder builder = builder(toPortable(obj));
-
-        builder.removeField("str");
-
-        assertNull(builder.build().<TestObjectAllTypes>deserialize().str);
-    }
-
-    /**
-     *
-     */
-    public void testRemoveFromExistingObjectAfterGet() {
-        TestObjectAllTypes obj = new TestObjectAllTypes();
-        obj.setDefaultData();
-        obj.enumArr = null;
-
-        PortableBuilderImpl builder = builder(toPortable(obj));
-
-        builder.getField("i_");
-
-        builder.removeField("str");
-
-        assertNull(builder.build().<TestObjectAllTypes>deserialize().str);
-    }
-
-    /**
-     * @throws IgniteCheckedException If any error occurs.
-     */
-    public void testDontBrokeCyclicDependency() throws IgniteCheckedException {
-        TestObjectOuter outer = new TestObjectOuter();
-        outer.inner = new TestObjectInner();
-        outer.inner.outer = outer;
-        outer.foo = "a";
-
-        PortableBuilder builder = builder(toPortable(outer));
-
-        builder.setField("foo", "b");
-
-        TestObjectOuter res = builder.build().deserialize();
-
-        assertEquals("b", res.foo);
-        assertSame(res, res.inner.outer);
-    }
-
-    /**
-     * @return Portables.
-     */
-    private IgnitePortables portables() {
-        return grid(0).portables();
-    }
-
-    /**
-     * @param obj Object.
-     * @return Portable object.
-     */
-    private PortableObject toPortable(Object obj) {
-        return portables().toPortable(obj);
-    }
-
-    /**
-     * @return Builder.
-     */
-    private <T> PortableBuilder builder(int typeId) {
-        return portables().builder(typeId);
-    }
-
-    /**
-     * @return Builder.
-     */
-    private <T> PortableBuilder builder(String clsName) {
-        return portables().builder(clsName);
-    }
-
-    /**
-     * @return Builder.
-     */
-    private <T> PortableBuilderImpl builder(PortableObject obj) {
-        return (PortableBuilderImpl)portables().builder(obj);
-    }
-
-    /**
-     *
-     */
-    private static class CustomIdMapper {
-        /** */
-        private String str = "a";
-
-        /** */
-        private int i = 10;
-    }
-
-    /**
-     */
-    private static class Key {
-        /** */
-        private int i;
-
-        /**
-         */
-        private Key() {
-            // No-op.
-        }
-
-        /**
-         * @param i Index.
-         */
-        private Key(int i) {
-            this.i = i;
-        }
-
-        /** {@inheritDoc} */
-        @Override public boolean equals(Object o) {
-            if (this == o)
-                return true;
-
-            if (o == null || getClass() != o.getClass())
-                return false;
-
-            Key key = (Key)o;
-
-            return i == key.i;
-        }
-
-        /** {@inheritDoc} */
-        @Override public int hashCode() {
-            return i;
-        }
-    }
-
-    /**
-     */
-    private static class Value {
-        /** */
-        private int i;
-
-        /**
-         */
-        private Value() {
-            // No-op.
-        }
-
-        /**
-         * @param i Index.
-         */
-        private Value(int i) {
-            this.i = i;
-        }
-    }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/71379a80/modules/core/src/test/java/org/apache/ignite/internal/portable/GridPortableBuilderStringAsCharsAdditionalSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/portable/GridPortableBuilderStringAsCharsAdditionalSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/portable/GridPortableBuilderStringAsCharsAdditionalSelfTest.java
deleted file mode 100644
index 2fce1a5..0000000
--- a/modules/core/src/test/java/org/apache/ignite/internal/portable/GridPortableBuilderStringAsCharsAdditionalSelfTest.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;
-
-/**
- *
- */
-public class GridPortableBuilderStringAsCharsAdditionalSelfTest extends GridPortableBuilderAdditionalSelfTest {
-    /** {@inheritDoc} */
-    @Override protected boolean useUtf8() {
-        return false;
-    }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/71379a80/modules/core/src/test/java/org/apache/ignite/internal/portable/GridPortableBuilderStringAsCharsSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/portable/GridPortableBuilderStringAsCharsSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/portable/GridPortableBuilderStringAsCharsSelfTest.java
deleted file mode 100644
index 5c53233..0000000
--- a/modules/core/src/test/java/org/apache/ignite/internal/portable/GridPortableBuilderStringAsCharsSelfTest.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;
-
-/**
- * Portable builder test.
- */
-public class GridPortableBuilderStringAsCharsSelfTest extends GridPortableBuilderSelfTest {
-    /** {@inheritDoc} */
-    @Override protected boolean useUtf8() {
-        return false;
-    }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/71379a80/modules/core/src/test/java/org/apache/ignite/internal/portable/GridPortableMarshallerCtxDisabledSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/portable/GridPortableMarshallerCtxDisabledSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/portable/GridPortableMarshallerCtxDisabledSelfTest.java
deleted file mode 100644
index bd9612c..0000000
--- a/modules/core/src/test/java/org/apache/ignite/internal/portable/GridPortableMarshallerCtxDisabledSelfTest.java
+++ /dev/null
@@ -1,256 +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.io.Externalizable;
-import java.io.IOException;
-import java.io.ObjectInput;
-import java.io.ObjectOutput;
-import java.util.Arrays;
-import org.apache.ignite.IgniteCheckedException;
-import org.apache.ignite.internal.MarshallerContextAdapter;
-import org.apache.ignite.internal.util.IgniteUtils;
-import org.apache.ignite.marshaller.portable.PortableMarshaller;
-import org.apache.ignite.portable.PortableException;
-import org.apache.ignite.portable.PortableMarshalAware;
-import org.apache.ignite.portable.PortableMetadata;
-import org.apache.ignite.portable.PortableReader;
-import org.apache.ignite.portable.PortableWriter;
-import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
-
-/**
- *
- */
-public class GridPortableMarshallerCtxDisabledSelfTest extends GridCommonAbstractTest {
-    /** */
-    protected static final PortableMetaDataHandler META_HND = new PortableMetaDataHandler() {
-        @Override public void addMeta(int typeId, PortableMetadata meta) {
-            // No-op.
-        }
-
-        @Override public PortableMetadata metadata(int typeId) {
-            return null;
-        }
-    };
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testObjectExchange() throws Exception {
-        PortableMarshaller marsh = new PortableMarshaller();
-        marsh.setContext(new MarshallerContextWithNoStorage());
-
-        PortableContext context = new PortableContext(META_HND, null);
-
-        IgniteUtils.invoke(PortableMarshaller.class, marsh, "setPortableContext", context);
-
-        SimpleObject simpleObj = new SimpleObject();
-
-        simpleObj.b = 2;
-        simpleObj.bArr = new byte[] {2, 3, 4, 5, 5};
-        simpleObj.c = 'A';
-        simpleObj.enumVal = TestEnum.D;
-        simpleObj.objArr = new Object[] {"hello", "world", "from", "me"};
-        simpleObj.enumArr = new TestEnum[] {TestEnum.C, TestEnum.B};
-
-        SimpleObject otherObj = new SimpleObject();
-
-        otherObj.b = 3;
-        otherObj.bArr = new byte[] {5, 3, 4};
-
-        simpleObj.otherObj = otherObj;
-
-        assertEquals(simpleObj, marsh.unmarshal(marsh.marshal(simpleObj), null));
-
-        SimplePortable simplePortable = new SimplePortable();
-
-        simplePortable.str = "portable";
-        simplePortable.arr = new long[] {100, 200, 300};
-
-        assertEquals(simplePortable, marsh.unmarshal(marsh.marshal(simplePortable), null));
-
-        SimpleExternalizable simpleExtr = new SimpleExternalizable();
-
-        simpleExtr.str = "externalizable";
-        simpleExtr.arr = new long[] {20000, 300000, 400000};
-
-        assertEquals(simpleExtr, marsh.unmarshal(marsh.marshal(simpleExtr), null));
-    }
-
-    /**
-     * Marshaller context with no storage. Platform has to work in such environment as well by marshalling class name of
-     * a portable object.
-     */
-    private static class MarshallerContextWithNoStorage extends MarshallerContextAdapter {
-        /** */
-        public MarshallerContextWithNoStorage() {
-            super(null);
-        }
-
-        /** {@inheritDoc} */
-        @Override protected boolean registerClassName(int id, String clsName) throws IgniteCheckedException {
-            return false;
-        }
-
-        /** {@inheritDoc} */
-        @Override protected String className(int id) throws IgniteCheckedException {
-            return null;
-        }
-    }
-
-    /**
-     */
-    private enum TestEnum {
-        A, B, C, D, E
-    }
-
-    /**
-     */
-    private static class SimpleObject {
-        /** */
-        private byte b;
-
-        /** */
-        private char c;
-
-        /** */
-        private byte[] bArr;
-
-        /** */
-        private Object[] objArr;
-
-        /** */
-        private TestEnum enumVal;
-
-        /** */
-        private TestEnum[] enumArr;
-
-        private SimpleObject otherObj;
-
-        /** {@inheritDoc} */
-        @Override public boolean equals(Object o) {
-            if (this == o)
-                return true;
-
-            if (o == null || getClass() != o.getClass())
-                return false;
-
-            SimpleObject object = (SimpleObject)o;
-
-            if (b != object.b)
-                return false;
-
-            if (c != object.c)
-                return false;
-
-            if (!Arrays.equals(bArr, object.bArr))
-                return false;
-
-            // Probably incorrect - comparing Object[] arrays with Arrays.equals
-            if (!Arrays.equals(objArr, object.objArr))
-                return false;
-
-            if (enumVal != object.enumVal)
-                return false;
-
-            // Probably incorrect - comparing Object[] arrays with Arrays.equals
-            if (!Arrays.equals(enumArr, object.enumArr))
-                return false;
-
-            return !(otherObj != null ? !otherObj.equals(object.otherObj) : object.otherObj != null);
-        }
-    }
-
-    /**
-     *
-     */
-    private static class SimplePortable implements PortableMarshalAware {
-        /** */
-        private String str;
-
-        /** */
-        private long[] arr;
-
-        /** {@inheritDoc} */
-        @Override public void writePortable(PortableWriter writer) throws PortableException {
-            writer.writeString("str", str);
-            writer.writeLongArray("longArr", arr);
-        }
-
-        /** {@inheritDoc} */
-        @Override public void readPortable(PortableReader reader) throws PortableException {
-            str = reader.readString("str");
-            arr = reader.readLongArray("longArr");
-        }
-
-        /** {@inheritDoc} */
-        @Override public boolean equals(Object o) {
-            if (this == o)
-                return true;
-
-            if (o == null || getClass() != o.getClass())
-                return false;
-
-            SimplePortable that = (SimplePortable)o;
-
-            if (str != null ? !str.equals(that.str) : that.str != null)
-                return false;
-
-            return Arrays.equals(arr, that.arr);
-        }
-    }
-
-    /**
-     *
-     */
-    private static class SimpleExternalizable implements Externalizable {
-        /** */
-        private String str;
-
-        /** */
-        private long[] arr;
-
-        /** {@inheritDoc} */
-        @Override public void writeExternal(ObjectOutput out) throws IOException {
-            out.writeUTF(str);
-            out.writeObject(arr);
-        }
-
-        /** {@inheritDoc} */
-        @Override public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException {
-            str = in.readUTF();
-            arr = (long[])in.readObject();
-        }
-
-        /** {@inheritDoc} */
-        @Override public boolean equals(Object o) {
-            if (this == o)
-                return true;
-
-            if (o == null || getClass() != o.getClass())
-                return false;
-
-            SimpleExternalizable that = (SimpleExternalizable)o;
-
-            if (str != null ? !str.equals(that.str) : that.str != null)
-                return false;
-
-            return Arrays.equals(arr, that.arr);
-        }
-    }
-}
\ No newline at end of file


[42/55] [abbrv] ignite git commit: Fixed test to really check query result.

Posted by ag...@apache.org.
Fixed test to really check query result.


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

Branch: refs/heads/ignite-1171
Commit: dcd8b42aa696bafbb4d7c11a9cf1520d54539001
Parents: ae1e1dc
Author: sboikov <sb...@gridgain.com>
Authored: Tue Sep 15 16:44:05 2015 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Tue Sep 15 16:44:05 2015 +0300

----------------------------------------------------------------------
 ...CacheScanPartitionQueryFallbackSelfTest.java | 54 ++++++++------------
 1 file changed, 21 insertions(+), 33 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/dcd8b42a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/CacheScanPartitionQueryFallbackSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/CacheScanPartitionQueryFallbackSelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/CacheScanPartitionQueryFallbackSelfTest.java
index df310b4..1ef470a 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/CacheScanPartitionQueryFallbackSelfTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/CacheScanPartitionQueryFallbackSelfTest.java
@@ -146,7 +146,7 @@ public class CacheScanPartitionQueryFallbackSelfTest extends GridCommonAbstractT
 
             CacheQuery<Map.Entry<Integer, Integer>> qry = cache.context().queries().createScanQuery(null, part, false);
 
-            doTestScanQuery(qry);
+            doTestScanQuery(qry, part);
         }
         finally {
             stopAllGrids();
@@ -176,7 +176,7 @@ public class CacheScanPartitionQueryFallbackSelfTest extends GridCommonAbstractT
 
             CacheQuery<Map.Entry<Integer, Integer>> qry = cache.context().queries().createScanQuery(null, part, false);
 
-            doTestScanQuery(qry);
+            doTestScanQuery(qry, part);
         }
         finally {
             stopAllGrids();
@@ -198,7 +198,7 @@ public class CacheScanPartitionQueryFallbackSelfTest extends GridCommonAbstractT
         try {
             Ignite ignite = startGrids(GRID_CNT);
 
-            final IgniteCacheProxy<Integer, Integer> cache = fillCache(ignite);
+            fillCache(ignite);
 
             final AtomicBoolean done = new AtomicBoolean(false);
 
@@ -211,6 +211,7 @@ public class CacheScanPartitionQueryFallbackSelfTest extends GridCommonAbstractT
 
                         while (!done.get()) {
                             startGrid(id);
+
                             Thread.sleep(3000);
 
                             stopGrid(id);
@@ -240,15 +241,10 @@ public class CacheScanPartitionQueryFallbackSelfTest extends GridCommonAbstractT
 
                             int part = tup.get1();
 
-                            try {
-                                CacheQuery<Map.Entry<Integer, Integer>> qry = cache.context().queries().createScanQuery(
-                                    null, part, false);
+                            CacheQuery<Map.Entry<Integer, Integer>> qry = cache.context().queries().createScanQuery(
+                                null, part, false);
 
-                                doTestScanQuery(qry);
-                            }
-                            catch (ClusterGroupEmptyCheckedException e) {
-                                log.warning("Invalid partition: " + part, e);
-                            }
+                            doTestScanQuery(qry, part);
                         }
 
                         return null;
@@ -315,17 +311,12 @@ public class CacheScanPartitionQueryFallbackSelfTest extends GridCommonAbstractT
                         while (!done.get()) {
                             int part = ThreadLocalRandom.current().nextInt(ignite(nodeId).affinity(null).partitions());
 
-                            try {
-                                QueryCursor<Cache.Entry<Integer, Integer>> cur =
-                                    cache.query(new ScanQuery<Integer, Integer>(part));
+                            QueryCursor<Cache.Entry<Integer, Integer>> cur =
+                                cache.query(new ScanQuery<Integer, Integer>(part));
 
-                                U.debug(log, "Running query [node=" + nodeId + ", part=" + part + ']');
+                            U.debug(log, "Running query [node=" + nodeId + ", part=" + part + ']');
 
-                                doTestScanQueryCursor(cur, part);
-                            }
-                            catch (ClusterGroupEmptyCheckedException e) {
-                                log.warning("Invalid partition: " + part, e);
-                            }
+                            doTestScanQueryCursor(cur, part);
                         }
 
                         return null;
@@ -404,7 +395,7 @@ public class CacheScanPartitionQueryFallbackSelfTest extends GridCommonAbstractT
                     }
                 };
 
-                int part;
+                Integer part = null;
                 CacheQuery<Map.Entry<Integer, Integer>> qry = null;
 
                 if (test.get()) {
@@ -416,7 +407,7 @@ public class CacheScanPartitionQueryFallbackSelfTest extends GridCommonAbstractT
                 new Thread(run).start();
 
                 if (test.get())
-                    doTestScanQuery(qry);
+                    doTestScanQuery(qry, part);
                 else
                     latch.await();
             }
@@ -454,20 +445,18 @@ public class CacheScanPartitionQueryFallbackSelfTest extends GridCommonAbstractT
     /**
      * @param qry Query.
      */
-    protected void doTestScanQuery(
-        CacheQuery<Map.Entry<Integer, Integer>> qry) throws IgniteCheckedException {
+    protected void doTestScanQuery(CacheQuery<Map.Entry<Integer, Integer>> qry, int part)
+        throws IgniteCheckedException {
         CacheQueryFuture<Map.Entry<Integer, Integer>> fut = qry.execute();
 
-        Collection<Map.Entry<Integer, Integer>> expEntries = fut.get();
+        Collection<Map.Entry<Integer, Integer>> qryEntries = fut.get();
 
-        for (Map.Entry<Integer, Integer> e : expEntries) {
-            Map<Integer, Integer> map = entries.get(((GridCacheQueryAdapter)qry).partition());
+        Map<Integer, Integer> map = entries.get(part);
 
-            if (map == null)
-                assertTrue(expEntries.isEmpty());
-            else
-                assertEquals(map.get(e.getKey()), e.getValue());
-        }
+        for (Map.Entry<Integer, Integer> e : qryEntries)
+            assertEquals(map.get(e.getKey()), e.getValue());
+
+        assertEquals("Invalid number of entries for partition: " + part, map.size(), qryEntries.size());
     }
 
     /**
@@ -484,7 +473,6 @@ public class CacheScanPartitionQueryFallbackSelfTest extends GridCommonAbstractT
         int cnt = 0;
 
         for (Cache.Entry<Integer, Integer> e : cur) {
-
             assertEquals(map.get(e.getKey()), e.getValue());
 
             cnt++;


[32/55] [abbrv] ignite git commit: ignite-1400 Concurrent cache/node stop

Posted by ag...@apache.org.
ignite-1400 Concurrent cache/node stop


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

Branch: refs/heads/ignite-1171
Commit: 91dd7c1cadfd70da8223a2755c2daf915ed69183
Parents: f977613
Author: sboikov <sb...@gridgain.com>
Authored: Tue Sep 15 14:26:33 2015 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Tue Sep 15 14:26:33 2015 +0300

----------------------------------------------------------------------
 .../processors/cache/GridCacheProcessor.java    |  19 +++-
 .../cache/IgniteCacheCreateRestartSelfTest.java | 106 +++++++++++++++++++
 .../testsuites/IgniteCacheRestartTestSuite.java |   3 +
 3 files changed, 126 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/91dd7c1c/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 59bb5f7..4317f70 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
@@ -936,6 +936,12 @@ public class GridCacheProcessor extends GridProcessorAdapter {
     @Override public void onKernalStop(boolean cancel) {
         cacheStartedLatch.countDown();
 
+        GridCachePartitionExchangeManager<Object, Object> exch = context().exchange();
+
+        // Stop exchange manager first so that we call onKernalStop on all caches.
+        // No new caches should be added after this point.
+        exch.onKernalStop(cancel);
+
         for (String cacheName : stopSeq) {
             GridCacheAdapter<?, ?> cache = caches.remove(maskNull(cacheName));
 
@@ -962,7 +968,8 @@ public class GridCacheProcessor extends GridProcessorAdapter {
             it.hasPrevious();) {
             GridCacheSharedManager<?, ?> mgr = it.previous();
 
-            mgr.onKernalStop(cancel);
+            if (mgr != exch)
+                mgr.onKernalStop(cancel);
         }
     }
 
@@ -2379,7 +2386,7 @@ public class GridCacheProcessor extends GridProcessorAdapter {
             if (ctx.localNodeId().equals(req.initiatingNodeId())) {
                 fut = (DynamicCacheStartFuture)pendingFuts.get(maskNull(req.cacheName()));
 
-                if (!req.deploymentId().equals(fut.deploymentId()))
+                if (fut != null && !req.deploymentId().equals(fut.deploymentId()))
                     fut = null;
             }
 
@@ -3101,6 +3108,14 @@ public class GridCacheProcessor extends GridProcessorAdapter {
      */
     public void cancelUserOperations() {
         sharedCtx.mvcc().cancelClientFutures();
+
+        Exception err = new IgniteCheckedException("Operation has been cancelled (node is stopping).");
+
+        for (IgniteInternalFuture fut : pendingFuts.values())
+            ((GridFutureAdapter)fut).onDone(err);
+
+        for (IgniteInternalFuture fut : pendingTemplateFuts.values())
+            ((GridFutureAdapter)fut).onDone(err);
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/91dd7c1c/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheCreateRestartSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheCreateRestartSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheCreateRestartSelfTest.java
new file mode 100644
index 0000000..0a2d778
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheCreateRestartSelfTest.java
@@ -0,0 +1,106 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.cache;
+
+import java.util.concurrent.Callable;
+import java.util.concurrent.ThreadLocalRandom;
+import javax.cache.CacheException;
+import org.apache.ignite.Ignite;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.internal.IgniteInternalFuture;
+import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi;
+import org.apache.ignite.spi.discovery.tcp.ipfinder.TcpDiscoveryIpFinder;
+import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder;
+import org.apache.ignite.testframework.GridTestUtils;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+
+/**
+ *
+ */
+public class IgniteCacheCreateRestartSelfTest extends GridCommonAbstractTest {
+    /** Cache name. */
+    private static final String CACHE_NAME = "partitioned";
+
+    /** IP finder. */
+    private static TcpDiscoveryIpFinder ipFinder = new TcpDiscoveryVmIpFinder(true);
+
+    /** */
+    private static final int NODES = 4;
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(gridName);
+
+        cfg.setPeerClassLoadingEnabled(false);
+
+        ((TcpDiscoverySpi)cfg.getDiscoverySpi()).setIpFinder(ipFinder);
+
+        return cfg;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTest() throws Exception {
+        stopAllGrids();
+    }
+
+    /** {@inheritDoc} */
+    @Override protected long getTestTimeout() {
+        return 5 * 60_000;
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testStopOriginatingNode() throws Exception {
+        startGrids(NODES);
+
+        ThreadLocalRandom rnd = ThreadLocalRandom.current();
+
+        for (int i = 0; i < 50; i++) {
+            int node = rnd.nextInt(0, NODES);
+
+            final Ignite ignite = ignite(node);
+
+            info("Running iteration on the node [idx=" + node + ", nodeId=" + ignite.cluster().localNode().id() + ']');
+
+            IgniteInternalFuture<?> fut = GridTestUtils.runAsync(new Callable<Void>() {
+                @Override public Void call() throws Exception {
+                    Thread.currentThread().setName("create-thread");
+
+                    try {
+                        ignite.createCache(new CacheConfiguration<>(CACHE_NAME));
+                    }
+                    catch (CacheException | IllegalStateException e) {
+                        log.info("Expected error: " + e);
+                    }
+
+                    return null;
+                }
+            });
+
+            ignite.close();
+
+            fut.get();
+
+            Ignite ignite0 = startGrid(node);
+
+            ignite0.destroyCache(CACHE_NAME);
+        }
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/91dd7c1c/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheRestartTestSuite.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheRestartTestSuite.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheRestartTestSuite.java
index 34ef79b..a6bd785 100644
--- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheRestartTestSuite.java
+++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheRestartTestSuite.java
@@ -18,6 +18,7 @@
 package org.apache.ignite.testsuites;
 
 import junit.framework.TestSuite;
+import org.apache.ignite.internal.processors.cache.IgniteCacheCreateRestartSelfTest;
 import org.apache.ignite.internal.processors.cache.distributed.dht.GridCachePartitionedNearDisabledOptimisticTxNodeRestartTest;
 import org.apache.ignite.internal.processors.cache.distributed.near.GridCachePartitionedNodeRestartTest;
 import org.apache.ignite.internal.processors.cache.distributed.near.GridCachePartitionedOptimisticTxNodeRestartTest;
@@ -39,6 +40,8 @@ public class IgniteCacheRestartTestSuite extends TestSuite {
         suite.addTestSuite(GridCacheReplicatedNodeRestartSelfTest.class);
         suite.addTestSuite(GridCachePartitionedNearDisabledOptimisticTxNodeRestartTest.class);
 
+        suite.addTestSuite(IgniteCacheCreateRestartSelfTest.class);
+
         return suite;
     }
 }
\ No newline at end of file


[37/55] [abbrv] 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/c257e070
Tree: http://git-wip-us.apache.org/repos/asf/ignite/tree/c257e070
Diff: http://git-wip-us.apache.org/repos/asf/ignite/diff/c257e070

Branch: refs/heads/ignite-1171
Commit: c257e070b1fcf29de4ac7192f8873c4e422b5d7e
Parents: a81cce7 2aa292c
Author: sboikov <sb...@gridgain.com>
Authored: Tue Sep 15 15:55:17 2015 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Tue Sep 15 15:55:17 2015 +0300

----------------------------------------------------------------------
 .../processors/igfs/IgfsFileWorkerBatch.java    | 15 ++++-
 .../processors/igfs/IgfsAbstractSelfTest.java   | 60 +++++++++-----------
 .../igfs/IgfsDualAbstractSelfTest.java          | 10 +++-
 .../testsuites/IgniteHadoopTestSuite.java       |  6 +-
 4 files changed, 52 insertions(+), 39 deletions(-)
----------------------------------------------------------------------



[55/55] [abbrv] ignite git commit: IGNITE-1171 - Attempt to fix discovery.

Posted by ag...@apache.org.
IGNITE-1171 - Attempt to fix discovery.


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

Branch: refs/heads/ignite-1171
Commit: 6aa0ee16c18086ba36f8f55735ce75c105a00cb3
Parents: c01f936
Author: Alexey Goncharuk <al...@gmail.com>
Authored: Wed Sep 16 19:03:31 2015 -0700
Committer: Alexey Goncharuk <al...@gmail.com>
Committed: Wed Sep 16 19:03:31 2015 -0700

----------------------------------------------------------------------
 .../GridCachePartitionExchangeManager.java      |  6 ++++
 .../ignite/spi/discovery/tcp/ServerImpl.java    | 31 ++++++++++++++++----
 .../spi/discovery/tcp/TcpDiscoverySpi.java      |  5 +++-
 .../TcpDiscoveryNodeAddFinishedMessage.java     | 26 +++++++++++++++-
 .../distributed/CacheAffEarlySelfTest.java      |  3 ++
 5 files changed, 64 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/6aa0ee16/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 20340d1..d7ada1c 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
@@ -1412,6 +1412,7 @@ public class GridCachePartitionExchangeManager<K, V> extends GridCacheSharedMana
         private static final long serialVersionUID = 0L;
 
         /** */
+        @GridToStringInclude
         private AffinityTopologyVersion topVer;
 
         /**
@@ -1432,5 +1433,10 @@ public class GridCachePartitionExchangeManager<K, V> extends GridCacheSharedMana
 
             return done;
         }
+
+        /** {@inheritDoc} */
+        @Override public String toString() {
+            return S.toString(AffinityReadyFuture.class, this, super.toString());
+        }
     }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/6aa0ee16/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ServerImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ServerImpl.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ServerImpl.java
index 3e50b94..24b30c9 100644
--- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ServerImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ServerImpl.java
@@ -3186,7 +3186,9 @@ class ServerImpl extends TcpDiscoveryImpl {
                     if (data != null)
                         spi.onExchange(node.id(), node.id(), data, U.gridClassLoader());
 
-                    msg.addDiscoveryData(locNodeId, spi.collectExchangeData(node.id()));
+                    // Collect discovery data the old way for older version nodes.
+                    if (TcpDiscoverySpi.DISCOVERY_DATA_COLLECT_ON_FINISH.compareTo(node.version()) > 0)
+                        msg.addDiscoveryData(locNodeId, spi.collectExchangeData(node.id()));
                 }
 
                 if (log.isDebugEnabled())
@@ -3253,9 +3255,12 @@ class ServerImpl extends TcpDiscoveryImpl {
                 }
 
                 // Notify outside of synchronized block.
-                if (dataMap != null) {
-                    for (Map.Entry<UUID, Map<Integer, byte[]>> entry : dataMap.entrySet())
-                        spi.onExchange(node.id(), entry.getKey(), entry.getValue(), U.gridClassLoader());
+                // Notify on node added message only if joining node is an old node.
+                if (TcpDiscoverySpi.DISCOVERY_DATA_COLLECT_ON_FINISH.compareTo(node.version()) > 0) {
+                    if (dataMap != null) {
+                        for (Map.Entry<UUID, Map<Integer, byte[]>> entry : dataMap.entrySet())
+                            spi.onExchange(node.id(), entry.getKey(), entry.getValue(), U.gridClassLoader());
+                    }
                 }
             }
 
@@ -3332,6 +3337,12 @@ class ServerImpl extends TcpDiscoveryImpl {
             }
 
             if (msg.verified() && !locNodeId.equals(nodeId) && spiStateCopy() == CONNECTED && fireEvt) {
+                if (TcpDiscoverySpi.DISCOVERY_DATA_COLLECT_ON_FINISH.compareTo(node.version()) <= 0) {
+                    Map<Integer, byte[]> data = spi.collectExchangeData(node.id());
+
+                    msg.addDiscoveryData(locNodeId, data);
+                }
+
                 spi.stats.onNodeJoined();
 
                 // Make sure that node with greater order will never get EVT_NODE_JOINED
@@ -3385,6 +3396,16 @@ class ServerImpl extends TcpDiscoveryImpl {
                     mux.notifyAll();
                 }
 
+                // Notify outside of synchronized block.
+                if (TcpDiscoverySpi.DISCOVERY_DATA_COLLECT_ON_FINISH.compareTo(node.version()) <= 0) {
+                    Map<UUID, Map<Integer, byte[]>> dataMap = msg.oldNodesDiscoveryData();
+
+                    if (dataMap != null) {
+                        for (Map.Entry<UUID, Map<Integer, byte[]>> entry : dataMap.entrySet())
+                            spi.onExchange(node.id(), entry.getKey(), entry.getValue(), U.gridClassLoader());
+                    }
+                }
+
                 // Discovery manager must create local joined event before spiStart completes.
                 notifyDiscovery(EVT_NODE_JOINED, topVer, locNode);
             }
@@ -5293,4 +5314,4 @@ class ServerImpl extends TcpDiscoveryImpl {
             this.sock = sock;
         }
     }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/6aa0ee16/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySpi.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySpi.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySpi.java
index 80fcc46..237e6d9 100644
--- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySpi.java
+++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySpi.java
@@ -222,6 +222,9 @@ public class TcpDiscoverySpi extends IgniteSpiAdapter implements DiscoverySpi, T
     /** Failure detection timeout feature minor version. */
     final static byte FAILURE_DETECTION_MINOR_VER = 4;
 
+    /** */
+    public static final IgniteProductVersion DISCOVERY_DATA_COLLECT_ON_FINISH = IgniteProductVersion.fromString("1.4.0");
+
     /** Failure detection timeout feature maintainance version. */
     final static byte FAILURE_DETECTION_MAINT_VER = 1;
 
@@ -2038,4 +2041,4 @@ public class TcpDiscoverySpi extends IgniteSpiAdapter implements DiscoverySpi, T
             return S.toString(SocketTimeoutObject.class, this);
         }
     }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/6aa0ee16/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryNodeAddFinishedMessage.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryNodeAddFinishedMessage.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryNodeAddFinishedMessage.java
index c6a469f..fc2a166 100644
--- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryNodeAddFinishedMessage.java
+++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryNodeAddFinishedMessage.java
@@ -17,6 +17,7 @@
 
 package org.apache.ignite.spi.discovery.tcp.messages;
 
+import java.util.LinkedHashMap;
 import java.util.Map;
 import java.util.UUID;
 import org.apache.ignite.internal.util.tostring.GridToStringExclude;
@@ -46,6 +47,9 @@ public class TcpDiscoveryNodeAddFinishedMessage extends TcpDiscoveryAbstractMess
     @GridToStringExclude
     private Map<String, Object> clientNodeAttrs;
 
+    /** Discovery data from old nodes. */
+    private Map<UUID, Map<Integer, byte[]>> oldNodesDiscoData;
+
     /**
      * Constructor.
      *
@@ -56,6 +60,8 @@ public class TcpDiscoveryNodeAddFinishedMessage extends TcpDiscoveryAbstractMess
         super(creatorNodeId);
 
         this.nodeId = nodeId;
+
+        oldNodesDiscoData = new LinkedHashMap<>();
     }
 
     /**
@@ -75,6 +81,24 @@ public class TcpDiscoveryNodeAddFinishedMessage extends TcpDiscoveryAbstractMess
     }
 
     /**
+     * @return Discovery data from old nodes.
+     */
+    public Map<UUID, Map<Integer, byte[]>> oldNodesDiscoveryData() {
+        return oldNodesDiscoData;
+    }
+
+    /**
+     * @param nodeId Node ID.
+     * @param discoData Discovery data to add.
+     */
+    public void addDiscoveryData(UUID nodeId, Map<Integer, byte[]> discoData) {
+        // Old nodes disco data may be null if message
+        // makes more than 1 pass due to stopping of the nodes in topology.
+        if (oldNodesDiscoData != null)
+            oldNodesDiscoData.put(nodeId, discoData);
+    }
+
+    /**
      * @param clientDiscoData Discovery data for joined client.
      */
     public void clientDiscoData(@Nullable Map<UUID, Map<Integer, byte[]>> clientDiscoData) {
@@ -101,4 +125,4 @@ public class TcpDiscoveryNodeAddFinishedMessage extends TcpDiscoveryAbstractMess
     @Override public String toString() {
         return S.toString(TcpDiscoveryNodeAddFinishedMessage.class, this, "super", super.toString());
     }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/6aa0ee16/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/CacheAffEarlySelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/CacheAffEarlySelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/CacheAffEarlySelfTest.java
index 7f0ca11..453b6f6 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/CacheAffEarlySelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/CacheAffEarlySelfTest.java
@@ -37,6 +37,7 @@ import org.apache.ignite.lang.IgniteFuture;
 import org.apache.ignite.lang.IgniteFutureTimeoutException;
 import org.apache.ignite.lang.IgniteInClosure;
 import org.apache.ignite.marshaller.optimized.OptimizedMarshaller;
+import org.apache.ignite.spi.checkpoint.noop.NoopCheckpointSpi;
 import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi;
 import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder;
 import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
@@ -83,6 +84,8 @@ public class CacheAffEarlySelfTest extends GridCommonAbstractTest {
 
         cfg.setMarshaller(marsh);
 
+        cfg.setCheckpointSpi(new NoopCheckpointSpi());
+
         return cfg;
     }
 


[04/55] [abbrv] ignite git commit: ignite-1462: hid portable API in 1.4 release

Posted by ag...@apache.org.
http://git-wip-us.apache.org/repos/asf/ignite/blob/71379a80/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
deleted file mode 100644
index 05df23b..0000000
--- a/modules/core/src/test/java/org/apache/ignite/internal/portable/GridPortableMetaDataDisabledSelfTest.java
+++ /dev/null
@@ -1,238 +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.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;
-import org.apache.ignite.portable.PortableTypeConfiguration;
-import org.apache.ignite.portable.PortableWriter;
-import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
-
-/**
- * Test for disabled meta data.
- */
-public class GridPortableMetaDataDisabledSelfTest extends GridCommonAbstractTest {
-    /** */
-    private PortableMarshaller marsh;
-
-    /** {@inheritDoc} */
-    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
-        IgniteConfiguration cfg = super.getConfiguration(gridName);
-
-        cfg.setMarshaller(marsh);
-
-        return cfg;
-    }
-
-    /**
-     * @return Portables.
-     */
-    private IgnitePortables portables() {
-        return grid().portables();
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testDisableGlobal() throws Exception {
-        marsh = new PortableMarshaller();
-
-        marsh.setClassNames(Arrays.asList(
-            TestObject1.class.getName(),
-            TestObject2.class.getName()
-        ));
-
-        marsh.setMetaDataEnabled(false);
-
-        try {
-            startGrid();
-
-            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();
-        }
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testDisableGlobalSimpleClass() throws Exception {
-        marsh = new PortableMarshaller();
-
-        PortableTypeConfiguration typeCfg = new PortableTypeConfiguration(TestObject2.class.getName());
-
-        typeCfg.setMetaDataEnabled(true);
-
-        marsh.setTypeConfigurations(Arrays.asList(
-            new PortableTypeConfiguration(TestObject1.class.getName()),
-            typeCfg
-        ));
-
-        marsh.setMetaDataEnabled(false);
-
-        try {
-            startGrid();
-
-            portables().toPortable(new TestObject1());
-            portables().toPortable(new TestObject2());
-
-            assertEquals(0, portables().metadata(TestObject1.class).fields().size());
-            assertEquals(1, portables().metadata(TestObject2.class).fields().size());
-        }
-        finally {
-            stopGrid();
-        }
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testDisableGlobalMarshalAwareClass() throws Exception {
-        marsh = new PortableMarshaller();
-
-        PortableTypeConfiguration typeCfg = new PortableTypeConfiguration(TestObject1.class.getName());
-
-        typeCfg.setMetaDataEnabled(true);
-
-        marsh.setTypeConfigurations(Arrays.asList(
-            new PortableTypeConfiguration(TestObject2.class.getName()),
-            typeCfg
-        ));
-
-        marsh.setMetaDataEnabled(false);
-
-        try {
-            startGrid();
-
-            portables().toPortable(new TestObject1());
-            portables().toPortable(new TestObject2());
-
-            assertEquals(1, portables().metadata(TestObject1.class).fields().size());
-            assertEquals(0, portables().metadata(TestObject2.class).fields().size());
-        }
-        finally {
-            stopGrid();
-        }
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testDisableSimpleClass() throws Exception {
-        marsh = new PortableMarshaller();
-
-        PortableTypeConfiguration typeCfg = new PortableTypeConfiguration(TestObject1.class.getName());
-
-        typeCfg.setMetaDataEnabled(false);
-
-        marsh.setTypeConfigurations(Arrays.asList(
-            new PortableTypeConfiguration(TestObject2.class.getName()),
-            typeCfg
-        ));
-
-        try {
-            startGrid();
-
-            portables().toPortable(new TestObject1());
-            portables().toPortable(new TestObject2());
-
-            assertEquals(0, portables().metadata(TestObject1.class).fields().size());
-            assertEquals(1, portables().metadata(TestObject2.class).fields().size());
-        }
-        finally {
-            stopGrid();
-        }
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testDisableMarshalAwareClass() throws Exception {
-        marsh = new PortableMarshaller();
-
-        PortableTypeConfiguration typeCfg = new PortableTypeConfiguration(TestObject2.class.getName());
-
-        typeCfg.setMetaDataEnabled(false);
-
-        marsh.setTypeConfigurations(Arrays.asList(
-            new PortableTypeConfiguration(TestObject1.class.getName()),
-            typeCfg
-        ));
-
-        try {
-            startGrid();
-
-            portables().toPortable(new TestObject1());
-            portables().toPortable(new TestObject2());
-
-            assertEquals(1, portables().metadata(TestObject1.class).fields().size());
-            assertEquals(0, portables().metadata(TestObject2.class).fields().size());
-        }
-        finally {
-            stopGrid();
-        }
-    }
-
-    /**
-     */
-    @SuppressWarnings("UnusedDeclaration")
-    private static class TestObject1 {
-        /** */
-        private int field;
-    }
-
-    /**
-     */
-    private static class TestObject2 implements PortableMarshalAware {
-        /** {@inheritDoc} */
-        @Override public void writePortable(PortableWriter writer) throws PortableException {
-            writer.writeInt("field", 1);
-        }
-
-        /** {@inheritDoc} */
-        @Override public void readPortable(PortableReader reader) throws PortableException {
-            // 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/71379a80/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
deleted file mode 100644
index fa3c9a7..0000000
--- a/modules/core/src/test/java/org/apache/ignite/internal/portable/GridPortableMetaDataSelfTest.java
+++ /dev/null
@@ -1,371 +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.math.BigDecimal;
-import java.util.Arrays;
-import java.util.Collection;
-import java.util.Date;
-import java.util.HashMap;
-import org.apache.ignite.IgnitePortables;
-import org.apache.ignite.configuration.CacheConfiguration;
-import org.apache.ignite.configuration.IgniteConfiguration;
-import org.apache.ignite.marshaller.portable.PortableMarshaller;
-import org.apache.ignite.portable.PortableException;
-import org.apache.ignite.portable.PortableMarshalAware;
-import org.apache.ignite.portable.PortableMetadata;
-import org.apache.ignite.portable.PortableObject;
-import org.apache.ignite.portable.PortableRawWriter;
-import org.apache.ignite.portable.PortableReader;
-import org.apache.ignite.portable.PortableWriter;
-import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
-
-/**
- * Portable meta data test.
- */
-public class GridPortableMetaDataSelfTest extends GridCommonAbstractTest {
-    /** */
-    private static int idx;
-
-    /** {@inheritDoc} */
-    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
-        IgniteConfiguration cfg = super.getConfiguration(gridName);
-
-        PortableMarshaller marsh = new PortableMarshaller();
-
-        marsh.setClassNames(Arrays.asList(TestObject1.class.getName(), TestObject2.class.getName()));
-
-        cfg.setMarshaller(marsh);
-
-        CacheConfiguration ccfg = new CacheConfiguration();
-
-        cfg.setCacheConfiguration(ccfg);
-
-        return cfg;
-    }
-
-    /** {@inheritDoc} */
-    @Override protected void beforeTest() throws Exception {
-        idx = 0;
-
-        startGrid();
-    }
-
-    /** {@inheritDoc} */
-    @Override protected void afterTest() throws Exception {
-        stopGrid();
-    }
-
-    /**
-     * @return Portables API.
-     */
-    protected IgnitePortables portables() {
-        return grid().portables();
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testGetAll() throws Exception {
-        portables().toPortable(new TestObject2());
-
-        Collection<PortableMetadata> metas = portables().metadata();
-
-        assertEquals(2, metas.size());
-
-        for (PortableMetadata meta : metas) {
-            Collection<String> fields;
-
-            switch (meta.typeName()) {
-                case "TestObject1":
-                    fields = meta.fields();
-
-                    assertEquals(7, fields.size());
-
-                    assertTrue(fields.contains("intVal"));
-                    assertTrue(fields.contains("strVal"));
-                    assertTrue(fields.contains("arrVal"));
-                    assertTrue(fields.contains("obj1Val"));
-                    assertTrue(fields.contains("obj2Val"));
-                    assertTrue(fields.contains("decVal"));
-                    assertTrue(fields.contains("decArrVal"));
-
-                    assertEquals("int", meta.fieldTypeName("intVal"));
-                    assertEquals("String", meta.fieldTypeName("strVal"));
-                    assertEquals("byte[]", meta.fieldTypeName("arrVal"));
-                    assertEquals("Object", meta.fieldTypeName("obj1Val"));
-                    assertEquals("Object", meta.fieldTypeName("obj2Val"));
-                    assertEquals("decimal", meta.fieldTypeName("decVal"));
-                    assertEquals("decimal[]", meta.fieldTypeName("decArrVal"));
-
-                    break;
-
-                case "TestObject2":
-                    fields = meta.fields();
-
-                    assertEquals(7, fields.size());
-
-                    assertTrue(fields.contains("boolVal"));
-                    assertTrue(fields.contains("dateVal"));
-                    assertTrue(fields.contains("uuidArrVal"));
-                    assertTrue(fields.contains("objVal"));
-                    assertTrue(fields.contains("mapVal"));
-                    assertTrue(fields.contains("decVal"));
-                    assertTrue(fields.contains("decArrVal"));
-
-                    assertEquals("boolean", meta.fieldTypeName("boolVal"));
-                    assertEquals("Date", meta.fieldTypeName("dateVal"));
-                    assertEquals("UUID[]", meta.fieldTypeName("uuidArrVal"));
-                    assertEquals("Object", meta.fieldTypeName("objVal"));
-                    assertEquals("Map", meta.fieldTypeName("mapVal"));
-                    assertEquals("decimal", meta.fieldTypeName("decVal"));
-                    assertEquals("decimal[]", meta.fieldTypeName("decArrVal"));
-
-                    break;
-
-                default:
-                    assert false : meta.typeName();
-            }
-        }
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testNoConfiguration() throws Exception {
-        fail("https://issues.apache.org/jira/browse/IGNITE-1377");
-
-        portables().toPortable(new TestObject3());
-
-        assertNotNull(portables().metadata(TestObject3.class));
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testReflection() throws Exception {
-        PortableMetadata meta = portables().metadata(TestObject1.class);
-
-        assertNotNull(meta);
-
-        assertEquals("TestObject1", meta.typeName());
-
-        Collection<String> fields = meta.fields();
-
-        assertEquals(7, fields.size());
-
-        assertTrue(fields.contains("intVal"));
-        assertTrue(fields.contains("strVal"));
-        assertTrue(fields.contains("arrVal"));
-        assertTrue(fields.contains("obj1Val"));
-        assertTrue(fields.contains("obj2Val"));
-        assertTrue(fields.contains("decVal"));
-        assertTrue(fields.contains("decArrVal"));
-
-        assertEquals("int", meta.fieldTypeName("intVal"));
-        assertEquals("String", meta.fieldTypeName("strVal"));
-        assertEquals("byte[]", meta.fieldTypeName("arrVal"));
-        assertEquals("Object", meta.fieldTypeName("obj1Val"));
-        assertEquals("Object", meta.fieldTypeName("obj2Val"));
-        assertEquals("decimal", meta.fieldTypeName("decVal"));
-        assertEquals("decimal[]", meta.fieldTypeName("decArrVal"));
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testPortableMarshalAware() throws Exception {
-        portables().toPortable(new TestObject2());
-
-        PortableMetadata meta = portables().metadata(TestObject2.class);
-
-        assertNotNull(meta);
-
-        assertEquals("TestObject2", meta.typeName());
-
-        Collection<String> fields = meta.fields();
-
-        assertEquals(7, fields.size());
-
-        assertTrue(fields.contains("boolVal"));
-        assertTrue(fields.contains("dateVal"));
-        assertTrue(fields.contains("uuidArrVal"));
-        assertTrue(fields.contains("objVal"));
-        assertTrue(fields.contains("mapVal"));
-        assertTrue(fields.contains("decVal"));
-        assertTrue(fields.contains("decArrVal"));
-
-        assertEquals("boolean", meta.fieldTypeName("boolVal"));
-        assertEquals("Date", meta.fieldTypeName("dateVal"));
-        assertEquals("UUID[]", meta.fieldTypeName("uuidArrVal"));
-        assertEquals("Object", meta.fieldTypeName("objVal"));
-        assertEquals("Map", meta.fieldTypeName("mapVal"));
-        assertEquals("decimal", meta.fieldTypeName("decVal"));
-        assertEquals("decimal[]", meta.fieldTypeName("decArrVal"));
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testMerge() throws Exception {
-        portables().toPortable(new TestObject2());
-
-        idx = 1;
-
-        portables().toPortable(new TestObject2());
-
-        PortableMetadata meta = portables().metadata(TestObject2.class);
-
-        assertNotNull(meta);
-
-        assertEquals("TestObject2", meta.typeName());
-
-        Collection<String> fields = meta.fields();
-
-        assertEquals(9, fields.size());
-
-        assertTrue(fields.contains("boolVal"));
-        assertTrue(fields.contains("dateVal"));
-        assertTrue(fields.contains("uuidArrVal"));
-        assertTrue(fields.contains("objVal"));
-        assertTrue(fields.contains("mapVal"));
-        assertTrue(fields.contains("charVal"));
-        assertTrue(fields.contains("colVal"));
-        assertTrue(fields.contains("decVal"));
-        assertTrue(fields.contains("decArrVal"));
-
-        assertEquals("boolean", meta.fieldTypeName("boolVal"));
-        assertEquals("Date", meta.fieldTypeName("dateVal"));
-        assertEquals("UUID[]", meta.fieldTypeName("uuidArrVal"));
-        assertEquals("Object", meta.fieldTypeName("objVal"));
-        assertEquals("Map", meta.fieldTypeName("mapVal"));
-        assertEquals("char", meta.fieldTypeName("charVal"));
-        assertEquals("Collection", meta.fieldTypeName("colVal"));
-        assertEquals("decimal", meta.fieldTypeName("decVal"));
-        assertEquals("decimal[]", meta.fieldTypeName("decArrVal"));
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testSerializedObject() throws Exception {
-        TestObject1 obj = new TestObject1();
-
-        obj.intVal = 10;
-        obj.strVal = "str";
-        obj.arrVal = new byte[] {2, 4, 6};
-        obj.obj1Val = null;
-        obj.obj2Val = new TestObject2();
-        obj.decVal = BigDecimal.ZERO;
-        obj.decArrVal = new BigDecimal[] { BigDecimal.ONE };
-
-        PortableObject po = portables().toPortable(obj);
-
-        info(po.toString());
-
-        PortableMetadata meta = po.metaData();
-
-        assertNotNull(meta);
-
-        assertEquals("TestObject1", meta.typeName());
-
-        Collection<String> fields = meta.fields();
-
-        assertEquals(7, fields.size());
-
-        assertTrue(fields.contains("intVal"));
-        assertTrue(fields.contains("strVal"));
-        assertTrue(fields.contains("arrVal"));
-        assertTrue(fields.contains("obj1Val"));
-        assertTrue(fields.contains("obj2Val"));
-        assertTrue(fields.contains("decVal"));
-        assertTrue(fields.contains("decArrVal"));
-
-        assertEquals("int", meta.fieldTypeName("intVal"));
-        assertEquals("String", meta.fieldTypeName("strVal"));
-        assertEquals("byte[]", meta.fieldTypeName("arrVal"));
-        assertEquals("Object", meta.fieldTypeName("obj1Val"));
-        assertEquals("Object", meta.fieldTypeName("obj2Val"));
-        assertEquals("decimal", meta.fieldTypeName("decVal"));
-        assertEquals("decimal[]", meta.fieldTypeName("decArrVal"));
-    }
-
-    /**
-     */
-    @SuppressWarnings("UnusedDeclaration")
-    private static class TestObject1 {
-        /** */
-        private int intVal;
-
-        /** */
-        private String strVal;
-
-        /** */
-        private byte[] arrVal;
-
-        /** */
-        private TestObject1 obj1Val;
-
-        /** */
-        private TestObject2 obj2Val;
-
-        /** */
-        private BigDecimal decVal;
-
-        /** */
-        private BigDecimal[] decArrVal;
-    }
-
-    /**
-     */
-    private static class TestObject2 implements PortableMarshalAware {
-        /** {@inheritDoc} */
-        @Override public void writePortable(PortableWriter writer) throws PortableException {
-            writer.writeBoolean("boolVal", false);
-            writer.writeDate("dateVal", new Date());
-            writer.writeUuidArray("uuidArrVal", null);
-            writer.writeObject("objVal", null);
-            writer.writeMap("mapVal", new HashMap<>());
-            writer.writeDecimal("decVal", BigDecimal.ZERO);
-            writer.writeDecimalArray("decArrVal", new BigDecimal[] { BigDecimal.ONE });
-
-            if (idx == 1) {
-                writer.writeChar("charVal", (char)0);
-                writer.writeCollection("colVal", null);
-            }
-
-            PortableRawWriter raw = writer.rawWriter();
-
-            raw.writeChar((char)0);
-            raw.writeCollection(null);
-        }
-
-        /** {@inheritDoc} */
-        @Override public void readPortable(PortableReader reader) throws PortableException {
-            // 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/71379a80/modules/core/src/test/java/org/apache/ignite/internal/portable/GridPortableWildcardsSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/portable/GridPortableWildcardsSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/portable/GridPortableWildcardsSelfTest.java
deleted file mode 100644
index 349f152..0000000
--- a/modules/core/src/test/java/org/apache/ignite/internal/portable/GridPortableWildcardsSelfTest.java
+++ /dev/null
@@ -1,482 +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.Arrays;
-import java.util.Map;
-import org.apache.ignite.internal.util.typedef.internal.U;
-import org.apache.ignite.marshaller.MarshallerContextTestImpl;
-import org.apache.ignite.marshaller.portable.PortableMarshaller;
-import org.apache.ignite.portable.PortableIdMapper;
-import org.apache.ignite.portable.PortableMetadata;
-import org.apache.ignite.portable.PortableTypeConfiguration;
-import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
-
-/**
- * Wildcards test.
- */
-public class GridPortableWildcardsSelfTest extends GridCommonAbstractTest {
-    /** */
-    private static final PortableMetaDataHandler META_HND = new PortableMetaDataHandler() {
-        @Override public void addMeta(int typeId, PortableMetadata meta) {
-            // No-op.
-        }
-
-        @Override public PortableMetadata metadata(int typeId) {
-            return null;
-        }
-    };
-
-    /**
-     * @return Portable context.
-     */
-    private PortableContext portableContext() {
-        return new PortableContext(META_HND, null);
-    }
-
-    /**
-     * @return Portable marshaller.
-     */
-    private PortableMarshaller portableMarshaller() {
-        PortableMarshaller marsh = new PortableMarshaller();
-        marsh.setContext(new MarshallerContextTestImpl(null));
-
-        return marsh;
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testClassNames() throws Exception {
-        PortableContext ctx = portableContext();
-
-        PortableMarshaller marsh = portableMarshaller();
-
-        marsh.setClassNames(Arrays.asList(
-            "org.apache.ignite.internal.portable.test.*",
-            "unknown.*"
-        ));
-
-        ctx.configure(marsh);
-
-        Map<Integer, Class> typeIds = U.field(ctx, "userTypes");
-
-        assertEquals(3, typeIds.size());
-
-        assertTrue(typeIds.containsKey("gridportabletestclass1".hashCode()));
-        assertTrue(typeIds.containsKey("gridportabletestclass2".hashCode()));
-        assertTrue(typeIds.containsKey("innerclass".hashCode()));
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testClassNamesWithMapper() throws Exception {
-        PortableContext ctx = portableContext();
-
-        PortableMarshaller marsh = portableMarshaller();
-
-        marsh.setIdMapper(new PortableIdMapper() {
-            @SuppressWarnings("IfMayBeConditional")
-            @Override public int typeId(String clsName) {
-                if (clsName.endsWith("1"))
-                    return 300;
-                else if (clsName.endsWith("2"))
-                    return 400;
-                else if (clsName.endsWith("InnerClass"))
-                    return 500;
-                else
-                    return -500;
-            }
-
-            @Override public int fieldId(int typeId, String fieldName) {
-                return 0;
-            }
-        });
-
-        marsh.setClassNames(Arrays.asList(
-            "org.apache.ignite.internal.portable.test.*",
-            "unknown.*"
-        ));
-
-        ctx.configure(marsh);
-
-        Map<String, PortableIdMapper> typeMappers = U.field(ctx, "typeMappers");
-
-        assertEquals(3, typeMappers.size());
-
-        assertEquals(300, typeMappers.get("GridPortableTestClass1").typeId("GridPortableTestClass1"));
-        assertEquals(400, typeMappers.get("GridPortableTestClass2").typeId("GridPortableTestClass2"));
-        assertEquals(500, typeMappers.get("InnerClass").typeId("InnerClass"));
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testTypeConfigurations() throws Exception {
-        PortableContext ctx = portableContext();
-
-        PortableMarshaller marsh = portableMarshaller();
-
-        marsh.setTypeConfigurations(Arrays.asList(
-            new PortableTypeConfiguration("org.apache.ignite.internal.portable.test.*"),
-            new PortableTypeConfiguration("unknown.*")
-        ));
-
-        ctx.configure(marsh);
-
-        Map<Integer, Class> typeIds = U.field(ctx, "userTypes");
-
-        assertEquals(3, typeIds.size());
-
-        assertTrue(typeIds.containsKey("gridportabletestclass1".hashCode()));
-        assertTrue(typeIds.containsKey("gridportabletestclass2".hashCode()));
-        assertTrue(typeIds.containsKey("innerclass".hashCode()));
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testTypeConfigurationsWithGlobalMapper() throws Exception {
-        PortableContext ctx = portableContext();
-
-        PortableMarshaller marsh = portableMarshaller();
-
-        marsh.setIdMapper(new PortableIdMapper() {
-            @SuppressWarnings("IfMayBeConditional")
-            @Override public int typeId(String clsName) {
-                if (clsName.endsWith("1"))
-                    return 300;
-                else if (clsName.endsWith("2"))
-                    return 400;
-                else if (clsName.endsWith("InnerClass"))
-                    return 500;
-                else
-                    return -500;
-            }
-
-            @Override public int fieldId(int typeId, String fieldName) {
-                return 0;
-            }
-        });
-
-        marsh.setTypeConfigurations(Arrays.asList(
-            new PortableTypeConfiguration("org.apache.ignite.internal.portable.test.*"),
-            new PortableTypeConfiguration("unknown.*")
-        ));
-
-        ctx.configure(marsh);
-
-        Map<String, PortableIdMapper> typeMappers = U.field(ctx, "typeMappers");
-
-        assertEquals(3, typeMappers.size());
-
-        assertEquals(300, typeMappers.get("GridPortableTestClass1").typeId("GridPortableTestClass1"));
-        assertEquals(400, typeMappers.get("GridPortableTestClass2").typeId("GridPortableTestClass2"));
-        assertEquals(500, typeMappers.get("InnerClass").typeId("InnerClass"));
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testTypeConfigurationsWithNonGlobalMapper() throws Exception {
-        PortableContext ctx = portableContext();
-
-        PortableMarshaller marsh = portableMarshaller();
-
-        marsh.setIdMapper(new PortableIdMapper() {
-            @SuppressWarnings("IfMayBeConditional")
-            @Override public int typeId(String clsName) {
-                if (clsName.endsWith("1"))
-                    return 300;
-                else if (clsName.endsWith("2"))
-                    return 400;
-                else if (clsName.endsWith("InnerClass"))
-                    return 500;
-                else
-                    return -500;
-            }
-
-            @Override public int fieldId(int typeId, String fieldName) {
-                return 0;
-            }
-        });
-
-        marsh.setTypeConfigurations(Arrays.asList(
-            new PortableTypeConfiguration("org.apache.ignite.internal.portable.test.*"),
-            new PortableTypeConfiguration("unknown.*")
-        ));
-
-        ctx.configure(marsh);
-
-        Map<String, PortableIdMapper> typeMappers = U.field(ctx, "typeMappers");
-
-        assertEquals(3, typeMappers.size());
-
-        assertEquals(300, typeMappers.get("GridPortableTestClass1").typeId("GridPortableTestClass1"));
-        assertEquals(400, typeMappers.get("GridPortableTestClass2").typeId("GridPortableTestClass2"));
-        assertEquals(500, typeMappers.get("InnerClass").typeId("InnerClass"));
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testOverride() throws Exception {
-        PortableContext ctx = portableContext();
-
-        PortableMarshaller marsh = portableMarshaller();
-
-        marsh.setClassNames(Arrays.asList(
-            "org.apache.ignite.internal.portable.test.*"
-        ));
-
-        PortableTypeConfiguration typeCfg = new PortableTypeConfiguration();
-
-        typeCfg.setClassName("org.apache.ignite.internal.portable.test.GridPortableTestClass2");
-        typeCfg.setIdMapper(new PortableIdMapper() {
-            @Override public int typeId(String clsName) {
-                return 100;
-            }
-
-            @Override public int fieldId(int typeId, String fieldName) {
-                return 0;
-            }
-        });
-
-        marsh.setTypeConfigurations(Arrays.asList(typeCfg));
-
-        ctx.configure(marsh);
-
-        Map<Integer, Class> typeIds = U.field(ctx, "userTypes");
-
-        assertEquals(3, typeIds.size());
-
-        assertTrue(typeIds.containsKey("gridportabletestclass1".hashCode()));
-        assertTrue(typeIds.containsKey("innerclass".hashCode()));
-        assertFalse(typeIds.containsKey("gridportabletestclass2".hashCode()));
-
-        Map<String, PortableIdMapper> typeMappers = U.field(ctx, "typeMappers");
-
-        assertEquals(100, typeMappers.get("GridPortableTestClass2").typeId("GridPortableTestClass2"));
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testClassNamesJar() throws Exception {
-        PortableContext ctx = portableContext();
-
-        PortableMarshaller marsh = portableMarshaller();
-
-        marsh.setClassNames(Arrays.asList(
-            "org.apache.ignite.portable.testjar.*",
-            "unknown.*"
-        ));
-
-        ctx.configure(marsh);
-
-        Map<Integer, Class> typeIds = U.field(ctx, "userTypes");
-
-        assertEquals(3, typeIds.size());
-
-        assertTrue(typeIds.containsKey("gridportabletestclass1".hashCode()));
-        assertTrue(typeIds.containsKey("gridportabletestclass2".hashCode()));
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testClassNamesWithMapperJar() throws Exception {
-        PortableContext ctx = portableContext();
-
-        PortableMarshaller marsh = portableMarshaller();
-
-        marsh.setIdMapper(new PortableIdMapper() {
-            @SuppressWarnings("IfMayBeConditional")
-            @Override public int typeId(String clsName) {
-                if (clsName.endsWith("1"))
-                    return 300;
-                else if (clsName.endsWith("2"))
-                    return 400;
-                else
-                    return -500;
-            }
-
-            @Override public int fieldId(int typeId, String fieldName) {
-                return 0;
-            }
-        });
-
-        marsh.setClassNames(Arrays.asList(
-            "org.apache.ignite.portable.testjar.*",
-            "unknown.*"
-        ));
-
-        ctx.configure(marsh);
-
-        Map<String, PortableIdMapper> typeMappers = U.field(ctx, "typeMappers");
-
-        assertEquals(3, typeMappers.size());
-
-        assertEquals(300, typeMappers.get("GridPortableTestClass1").typeId("GridPortableTestClass1"));
-        assertEquals(400, typeMappers.get("GridPortableTestClass2").typeId("GridPortableTestClass2"));
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testTypeConfigurationsJar() throws Exception {
-        PortableContext ctx = portableContext();
-
-        PortableMarshaller marsh = portableMarshaller();
-
-        marsh.setTypeConfigurations(Arrays.asList(
-            new PortableTypeConfiguration("org.apache.ignite.portable.testjar.*"),
-            new PortableTypeConfiguration("unknown.*")
-        ));
-
-        ctx.configure(marsh);
-
-        Map<Integer, Class> typeIds = U.field(ctx, "userTypes");
-
-        assertEquals(3, typeIds.size());
-
-        assertTrue(typeIds.containsKey("gridportabletestclass1".hashCode()));
-        assertTrue(typeIds.containsKey("gridportabletestclass2".hashCode()));
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testTypeConfigurationsWithGlobalMapperJar() throws Exception {
-        PortableContext ctx = portableContext();
-
-        PortableMarshaller marsh = portableMarshaller();
-
-        marsh.setIdMapper(new PortableIdMapper() {
-            @SuppressWarnings("IfMayBeConditional")
-            @Override public int typeId(String clsName) {
-                if (clsName.endsWith("1"))
-                    return 300;
-                else if (clsName.endsWith("2"))
-                    return 400;
-                else
-                    return -500;
-            }
-
-            @Override public int fieldId(int typeId, String fieldName) {
-                return 0;
-            }
-        });
-
-        marsh.setTypeConfigurations(Arrays.asList(
-            new PortableTypeConfiguration("org.apache.ignite.portable.testjar.*"),
-            new PortableTypeConfiguration("unknown.*")
-        ));
-
-        ctx.configure(marsh);
-
-        Map<String, PortableIdMapper> typeMappers = U.field(ctx, "typeMappers");
-
-        assertEquals(3, typeMappers.size());
-
-        assertEquals(300, typeMappers.get("GridPortableTestClass1").typeId("GridPortableTestClass1"));
-        assertEquals(400, typeMappers.get("GridPortableTestClass2").typeId("GridPortableTestClass2"));
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testTypeConfigurationsWithNonGlobalMapperJar() throws Exception {
-        PortableContext ctx = portableContext();
-
-        PortableMarshaller marsh = portableMarshaller();
-
-        marsh.setIdMapper(new PortableIdMapper() {
-            @SuppressWarnings("IfMayBeConditional")
-            @Override public int typeId(String clsName) {
-                if (clsName.endsWith("1"))
-                    return 300;
-                else if (clsName.endsWith("2"))
-                    return 400;
-                else
-                    return -500;
-            }
-
-            @Override public int fieldId(int typeId, String fieldName) {
-                return 0;
-            }
-        });
-
-        marsh.setTypeConfigurations(Arrays.asList(
-            new PortableTypeConfiguration("org.apache.ignite.portable.testjar.*"),
-            new PortableTypeConfiguration("unknown.*")
-        ));
-
-        ctx.configure(marsh);
-
-        Map<String, PortableIdMapper> typeMappers = U.field(ctx, "typeMappers");
-
-        assertEquals(3, typeMappers.size());
-
-        assertEquals(300, typeMappers.get("GridPortableTestClass1").typeId("GridPortableTestClass1"));
-        assertEquals(400, typeMappers.get("GridPortableTestClass2").typeId("GridPortableTestClass2"));
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testOverrideJar() throws Exception {
-        PortableContext ctx = portableContext();
-
-        PortableMarshaller marsh = portableMarshaller();
-
-        marsh.setClassNames(Arrays.asList(
-            "org.apache.ignite.portable.testjar.*"
-        ));
-
-        PortableTypeConfiguration typeCfg = new PortableTypeConfiguration(
-            "org.apache.ignite.portable.testjar.GridPortableTestClass2");
-
-        typeCfg.setIdMapper(new PortableIdMapper() {
-            @Override public int typeId(String clsName) {
-                return 100;
-            }
-
-            @Override public int fieldId(int typeId, String fieldName) {
-                return 0;
-            }
-        });
-
-        marsh.setTypeConfigurations(Arrays.asList(typeCfg));
-
-        ctx.configure(marsh);
-
-        Map<Integer, Class> typeIds = U.field(ctx, "userTypes");
-
-        assertEquals(3, typeIds.size());
-
-        assertTrue(typeIds.containsKey("gridportabletestclass1".hashCode()));
-
-        Map<String, PortableIdMapper> typeMappers = U.field(ctx, "typeMappers");
-
-        assertEquals(3, typeMappers.size());
-
-        assertEquals(100, typeMappers.get("GridPortableTestClass2").typeId("GridPortableTestClass2"));
-    }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/71379a80/modules/core/src/test/java/org/apache/ignite/internal/portable/mutabletest/GridPortableMarshalerAwareTestClass.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/portable/mutabletest/GridPortableMarshalerAwareTestClass.java b/modules/core/src/test/java/org/apache/ignite/internal/portable/mutabletest/GridPortableMarshalerAwareTestClass.java
deleted file mode 100644
index 3244331..0000000
--- a/modules/core/src/test/java/org/apache/ignite/internal/portable/mutabletest/GridPortableMarshalerAwareTestClass.java
+++ /dev/null
@@ -1,67 +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.mutabletest;
-
-import org.apache.ignite.internal.util.typedef.internal.S;
-import org.apache.ignite.portable.PortableException;
-import org.apache.ignite.portable.PortableMarshalAware;
-import org.apache.ignite.portable.PortableRawReader;
-import org.apache.ignite.portable.PortableRawWriter;
-import org.apache.ignite.portable.PortableReader;
-import org.apache.ignite.portable.PortableWriter;
-import org.apache.ignite.testframework.GridTestUtils;
-
-/**
- *
- */
-public class GridPortableMarshalerAwareTestClass implements PortableMarshalAware {
-    /** */
-    public String s;
-
-    /** */
-    public String sRaw;
-
-    /** {@inheritDoc} */
-    @Override public void writePortable(PortableWriter writer) throws PortableException {
-        writer.writeString("s", s);
-
-        PortableRawWriter raw = writer.rawWriter();
-
-        raw.writeString(sRaw);
-    }
-
-    /** {@inheritDoc} */
-    @Override public void readPortable(PortableReader reader) throws PortableException {
-        s = reader.readString("s");
-
-        PortableRawReader raw = reader.rawReader();
-
-        sRaw = raw.readString();
-    }
-
-    /** {@inheritDoc} */
-    @SuppressWarnings("FloatingPointEquality")
-    @Override public boolean equals(Object other) {
-        return this == other || GridTestUtils.deepEquals(this, other);
-    }
-
-    /** {@inheritDoc} */
-    @Override public String toString() {
-        return S.toString(GridPortableMarshalerAwareTestClass.class, this);
-    }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/71379a80/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
deleted file mode 100644
index e49514b..0000000
--- a/modules/core/src/test/java/org/apache/ignite/internal/portable/mutabletest/GridPortableTestClasses.java
+++ /dev/null
@@ -1,434 +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.mutabletest;
-
-import com.google.common.base.Throwables;
-import java.io.ByteArrayOutputStream;
-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;
-import java.util.Map;
-import java.util.TreeMap;
-import java.util.UUID;
-import org.apache.ignite.internal.util.lang.GridMapEntry;
-import org.apache.ignite.portable.PortableObject;
-
-/**
- *
- */
-@SuppressWarnings({"PublicInnerClass", "PublicField"})
-public class GridPortableTestClasses {
-    /**
-     *
-     */
-    public static class TestObjectContainer {
-        /** */
-        public Object foo;
-
-        /**
-         *
-         */
-        public TestObjectContainer() {
-            // No-op.
-        }
-
-        /**
-         * @param foo Object.
-         */
-        public TestObjectContainer(Object foo) {
-            this.foo = foo;
-        }
-    }
-
-    /**
-     *
-     */
-    public static class TestObjectOuter {
-        /** */
-        public TestObjectInner inner;
-
-        /** */
-        public String foo;
-
-        /**
-         *
-         */
-        public TestObjectOuter() {
-
-        }
-
-        /**
-         * @param inner Inner object.
-         */
-        public TestObjectOuter(TestObjectInner inner) {
-            this.inner = inner;
-        }
-    }
-
-    /** */
-    public static class TestObjectInner {
-        /** */
-        public Object foo;
-
-        /** */
-        public TestObjectOuter outer;
-    }
-
-    /** */
-    public static class TestObjectArrayList {
-        /** */
-        public List<String> list = new ArrayList<>();
-    }
-
-    /**
-     *
-     */
-    public static class TestObjectPlainPortable {
-        /** */
-        public PortableObject plainPortable;
-
-        /**
-         *
-         */
-        public TestObjectPlainPortable() {
-            // No-op.
-        }
-
-        /**
-         * @param plainPortable Object.
-         */
-        public TestObjectPlainPortable(PortableObject plainPortable) {
-            this.plainPortable = plainPortable;
-        }
-    }
-
-    /**
-     *
-     */
-    public static class TestObjectAllTypes implements Serializable {
-        /** */
-        public Byte b_;
-
-        /** */
-        public Short s_;
-
-        /** */
-        public Integer i_;
-
-        /** */
-        public Long l_;
-
-        /** */
-        public Float f_;
-
-        /** */
-        public Double d_;
-
-        /** */
-        public Character c_;
-
-        /** */
-        public Boolean z_;
-
-        /** */
-        public byte b;
-
-        /** */
-        public short s;
-
-        /** */
-        public int i;
-
-        /** */
-        public long l;
-
-        /** */
-        public float f;
-
-        /** */
-        public double d;
-
-        /** */
-        public char c;
-
-        /** */
-        public boolean z;
-
-        /** */
-        public String str;
-
-        /** */
-        public UUID uuid;
-
-        /** */
-        public Date date;
-
-        /** */
-        public byte[] bArr;
-
-        /** */
-        public short[] sArr;
-
-        /** */
-        public int[] iArr;
-
-        /** */
-        public long[] lArr;
-
-        /** */
-        public float[] fArr;
-
-        /** */
-        public double[] dArr;
-
-        /** */
-        public char[] cArr;
-
-        /** */
-        public boolean[] zArr;
-
-        /** */
-        public BigDecimal[] bdArr;
-
-        /** */
-        public String[] strArr;
-
-        /** */
-        public UUID[] uuidArr;
-
-        /** */
-        public Date[] dateArr;
-
-        /** */
-        public TestObjectEnum anEnum;
-
-        /** */
-        public TestObjectEnum[] enumArr;
-
-        /** */
-        public Map.Entry entry;
-
-        /**
-         * @return Array.
-         */
-        private byte[] serialize() {
-            ByteArrayOutputStream byteOut = new ByteArrayOutputStream();
-
-            try {
-                ObjectOutput out = new ObjectOutputStream(byteOut);
-
-                out.writeObject(this);
-
-                out.close();
-            }
-            catch (IOException e) {
-                Throwables.propagate(e);
-            }
-
-            return byteOut.toByteArray();
-        }
-
-        /**
-         *
-         */
-        public void setDefaultData() {
-            b_ = 11;
-            s_ = 22;
-            i_ = 33;
-            l_ = 44L;
-            f_ = 55f;
-            d_ = 66d;
-            c_ = 'e';
-            z_ = true;
-
-            b = 1;
-            s = 2;
-            i = 3;
-            l = 4;
-            f = 5;
-            d = 6;
-            c = 7;
-            z = true;
-
-            str = "abc";
-            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};
-
-            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};
-
-            entry = new GridMapEntry<>(1, "a");
-        }
-    }
-
-    /**
-     *
-     */
-    public enum TestObjectEnum {
-        A, B, C
-    }
-
-    /**
-     *
-     */
-    public static class Address {
-        /** City. */
-        public String city;
-
-        /** Street. */
-        public String street;
-
-        /** Street number. */
-        public int streetNumber;
-
-        /** Flat number. */
-        public int flatNumber;
-
-        /**
-         * Default constructor.
-         */
-        public Address() {
-            // No-op.
-        }
-
-        /**
-         * Constructor.
-         *
-         * @param city City.
-         * @param street Street.
-         * @param streetNumber Street number.
-         * @param flatNumber Flat number.
-         */
-        public Address(String city, String street, int streetNumber, int flatNumber) {
-            this.city = city;
-            this.street = street;
-            this.streetNumber = streetNumber;
-            this.flatNumber = flatNumber;
-        }
-    }
-
-    /**
-     *
-     */
-    public static class Company {
-        /** ID. */
-        public int id;
-
-        /** Name. */
-        public String name;
-
-        /** Size. */
-        public int size;
-
-        /** Address. */
-        public Address address;
-
-        /** Occupation. */
-        public String occupation;
-
-        /**
-         * Default constructor.
-         */
-        public Company() {
-            // No-op.
-        }
-
-        /**
-         * Constructor.
-         *
-         * @param id ID.
-         * @param name Name.
-         * @param size Size.
-         * @param address Address.
-         * @param occupation Occupation.
-         */
-        public Company(int id, String name, int size, Address address, String occupation) {
-            this.id = id;
-            this.name = name;
-            this.size = size;
-            this.address = address;
-            this.occupation = occupation;
-        }
-    }
-
-    /**
-     *
-     */
-    public static class AddressBook {
-        /** */
-        private Map<String, List<Company>> companyByStreet = new TreeMap<>();
-
-        /**
-         * @param street Street.
-         * @return Company.
-         */
-        public List<Company> findCompany(String street) {
-            return companyByStreet.get(street);
-        }
-
-        /**
-         * @param company Company.
-         */
-        public void addCompany(Company company) {
-            List<Company> list = companyByStreet.get(company.address.street);
-
-            if (list == null) {
-                list = new ArrayList<>();
-
-                companyByStreet.put(company.address.street, list);
-            }
-
-            list.add(company);
-        }
-
-        /**
-         * @return map
-         */
-        public Map<String, List<Company>> getCompanyByStreet() {
-            return companyByStreet;
-        }
-
-        /**
-         * @param companyByStreet map
-         */
-        public void setCompanyByStreet(Map<String, List<Company>> companyByStreet) {
-            this.companyByStreet = companyByStreet;
-        }
-    }
-
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/71379a80/modules/core/src/test/java/org/apache/ignite/internal/portable/mutabletest/package-info.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/portable/mutabletest/package-info.java b/modules/core/src/test/java/org/apache/ignite/internal/portable/mutabletest/package-info.java
deleted file mode 100644
index daa13d5..0000000
--- a/modules/core/src/test/java/org/apache/ignite/internal/portable/mutabletest/package-info.java
+++ /dev/null
@@ -1,22 +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 description. -->
- * Contains internal tests or test related classes and interfaces.
- */
-package org.apache.ignite.internal.portable.mutabletest;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/71379a80/modules/core/src/test/java/org/apache/ignite/internal/portable/package-info.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/portable/package-info.java b/modules/core/src/test/java/org/apache/ignite/internal/portable/package-info.java
deleted file mode 100644
index 26897e6..0000000
--- a/modules/core/src/test/java/org/apache/ignite/internal/portable/package-info.java
+++ /dev/null
@@ -1,22 +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 description. -->
- * Contains internal tests or test related classes and interfaces.
- */
-package org.apache.ignite.internal.portable;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/71379a80/modules/core/src/test/java/org/apache/ignite/internal/portable/test/GridPortableTestClass1.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/portable/test/GridPortableTestClass1.java b/modules/core/src/test/java/org/apache/ignite/internal/portable/test/GridPortableTestClass1.java
deleted file mode 100644
index 05a8c33..0000000
--- a/modules/core/src/test/java/org/apache/ignite/internal/portable/test/GridPortableTestClass1.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.test;
-
-/**
- */
-public class GridPortableTestClass1 {
-    /**
-     */
-    private static class InnerClass {
-        // No-op.
-    }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/71379a80/modules/core/src/test/java/org/apache/ignite/internal/portable/test/GridPortableTestClass2.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/portable/test/GridPortableTestClass2.java b/modules/core/src/test/java/org/apache/ignite/internal/portable/test/GridPortableTestClass2.java
deleted file mode 100644
index ba69991..0000000
--- a/modules/core/src/test/java/org/apache/ignite/internal/portable/test/GridPortableTestClass2.java
+++ /dev/null
@@ -1,24 +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.test;
-
-/**
- */
-public class GridPortableTestClass2 {
-    // No-op.
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/71379a80/modules/core/src/test/java/org/apache/ignite/internal/portable/test/package-info.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/portable/test/package-info.java b/modules/core/src/test/java/org/apache/ignite/internal/portable/test/package-info.java
deleted file mode 100644
index e63b814..0000000
--- a/modules/core/src/test/java/org/apache/ignite/internal/portable/test/package-info.java
+++ /dev/null
@@ -1,22 +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 description. -->
- * Contains internal tests or test related classes and interfaces.
- */
-package org.apache.ignite.internal.portable.test;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/71379a80/modules/core/src/test/java/org/apache/ignite/internal/portable/test/subpackage/GridPortableTestClass3.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/portable/test/subpackage/GridPortableTestClass3.java b/modules/core/src/test/java/org/apache/ignite/internal/portable/test/subpackage/GridPortableTestClass3.java
deleted file mode 100644
index cf3aa2d..0000000
--- a/modules/core/src/test/java/org/apache/ignite/internal/portable/test/subpackage/GridPortableTestClass3.java
+++ /dev/null
@@ -1,24 +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.test.subpackage;
-
-/**
- */
-public class GridPortableTestClass3 {
-    // No-op.
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/71379a80/modules/core/src/test/java/org/apache/ignite/internal/portable/test/subpackage/package-info.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/portable/test/subpackage/package-info.java b/modules/core/src/test/java/org/apache/ignite/internal/portable/test/subpackage/package-info.java
deleted file mode 100644
index ae8ee73..0000000
--- a/modules/core/src/test/java/org/apache/ignite/internal/portable/test/subpackage/package-info.java
+++ /dev/null
@@ -1,22 +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 description. -->
- * Contains internal tests or test related classes and interfaces.
- */
-package org.apache.ignite.internal.portable.test.subpackage;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/71379a80/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
deleted file mode 100644
index 1ba3d4d..0000000
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/GridCacheClientNodePortableMetadataMultinodeTest.java
+++ /dev/null
@@ -1,295 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.ignite.internal.processors.cache.portable;
-
-import java.util.Collection;
-import java.util.HashSet;
-import java.util.Set;
-import java.util.concurrent.Callable;
-import java.util.concurrent.CyclicBarrier;
-import java.util.concurrent.ThreadLocalRandom;
-import java.util.concurrent.atomic.AtomicBoolean;
-import java.util.concurrent.atomic.AtomicInteger;
-import org.apache.ignite.Ignite;
-import org.apache.ignite.IgniteCache;
-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;
-import org.apache.ignite.portable.PortableMetadata;
-import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi;
-import org.apache.ignite.spi.discovery.tcp.ipfinder.TcpDiscoveryIpFinder;
-import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder;
-import org.apache.ignite.testframework.GridTestUtils;
-import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
-import org.eclipse.jetty.util.ConcurrentHashSet;
-
-import static org.apache.ignite.cache.CacheWriteSynchronizationMode.FULL_SYNC;
-
-/**
- *
- */
-public class GridCacheClientNodePortableMetadataMultinodeTest extends GridCommonAbstractTest {
-    /** */
-    protected static TcpDiscoveryIpFinder ipFinder = new TcpDiscoveryVmIpFinder(true);
-
-    /** */
-    private boolean client;
-
-    /** {@inheritDoc} */
-    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
-        IgniteConfiguration cfg = super.getConfiguration(gridName);
-
-        cfg.setPeerClassLoadingEnabled(false);
-
-        ((TcpDiscoverySpi)cfg.getDiscoverySpi()).setIpFinder(ipFinder).setForceServerMode(true);
-
-        cfg.setMarshaller(new PortableMarshaller());
-
-        CacheConfiguration ccfg = new CacheConfiguration();
-
-        ccfg.setWriteSynchronizationMode(FULL_SYNC);
-
-        cfg.setCacheConfiguration(ccfg);
-
-        cfg.setClientMode(client);
-
-        return cfg;
-    }
-
-    /** {@inheritDoc} */
-    @Override protected void afterTest() throws Exception {
-        super.afterTest();
-
-        stopAllGrids();
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testClientMetadataInitialization() throws Exception {
-        startGrids(2);
-
-        final AtomicBoolean stop = new AtomicBoolean();
-
-        final ConcurrentHashSet<String> allTypes = new ConcurrentHashSet<>();
-
-        IgniteInternalFuture<?> fut;
-
-        try {
-            // Update portable metadata concurrently with client nodes start.
-            fut = GridTestUtils.runMultiThreadedAsync(new Callable<Object>() {
-                @Override public Object call() throws Exception {
-                    IgnitePortables portables = ignite(0).portables();
-
-                    IgniteCache<Object, Object> cache = ignite(0).cache(null).withKeepPortable();
-
-                    ThreadLocalRandom rnd = ThreadLocalRandom.current();
-
-                    for (int i = 0; i < 1000; i++) {
-                        log.info("Iteration: " + i);
-
-                        String type = "portable-type-" + i;
-
-                        allTypes.add(type);
-
-                        for (int f = 0; f < 10; f++) {
-                            PortableBuilder builder = portables.builder(type);
-
-                            String fieldName = "f" + f;
-
-                            builder.setField(fieldName, i);
-
-                            cache.put(rnd.nextInt(0, 100_000), builder.build());
-
-                            if (f % 100 == 0)
-                                log.info("Put iteration: " + f);
-                        }
-
-                        if (stop.get())
-                            break;
-                    }
-
-                    return null;
-                }
-            }, 5, "update-thread");
-        }
-        finally {
-            stop.set(true);
-        }
-
-        client = true;
-
-        startGridsMultiThreaded(2, 5);
-
-        fut.get();
-
-        assertFalse(allTypes.isEmpty());
-
-        log.info("Expected portable types: " + allTypes.size());
-
-        assertEquals(7, ignite(0).cluster().nodes().size());
-
-        for (int i = 0; i < 7; i++) {
-            log.info("Check metadata on node: " + i);
-
-            boolean client = i > 1;
-
-            assertEquals((Object)client, ignite(i).configuration().isClientMode());
-
-            IgnitePortables portables = ignite(i).portables();
-
-            Collection<PortableMetadata> metaCol = portables.metadata();
-
-            assertEquals(allTypes.size(), metaCol.size());
-
-            Set<String> names = new HashSet<>();
-
-            for (PortableMetadata meta : metaCol) {
-                assertTrue(names.add(meta.typeName()));
-
-                assertNull(meta.affinityKeyFieldName());
-
-                assertEquals(10, meta.fields().size());
-            }
-
-            assertEquals(allTypes.size(), names.size());
-        }
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testFailoverOnStart() throws Exception {
-        startGrids(4);
-
-        IgnitePortables portables = ignite(0).portables();
-
-        IgniteCache<Object, Object> cache = ignite(0).cache(null).withKeepPortable();
-
-        for (int i = 0; i < 1000; i++) {
-            PortableBuilder builder = portables.builder("type-" + i);
-
-            builder.setField("f0", i);
-
-            cache.put(i, builder.build());
-        }
-
-        client = true;
-
-        final CyclicBarrier barrier = new CyclicBarrier(6);
-
-        final AtomicInteger startIdx = new AtomicInteger(4);
-
-        IgniteInternalFuture<?> fut = GridTestUtils.runMultiThreadedAsync(new Callable<Object>() {
-            @Override public Object call() throws Exception {
-                barrier.await();
-
-                Ignite ignite = startGrid(startIdx.getAndIncrement());
-
-                assertTrue(ignite.configuration().isClientMode());
-
-                log.info("Started node: " + ignite.name());
-
-                return null;
-            }
-        }, 5, "start-thread");
-
-        barrier.await();
-
-        U.sleep(ThreadLocalRandom.current().nextInt(10, 100));
-
-        for (int i = 0; i < 3; i++)
-            stopGrid(i);
-
-        fut.get();
-
-        assertEquals(6, ignite(3).cluster().nodes().size());
-
-        for (int i = 3; i < 7; i++) {
-            log.info("Check metadata on node: " + i);
-
-            boolean client = i > 3;
-
-            assertEquals((Object) client, ignite(i).configuration().isClientMode());
-
-            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());
-
-            Set<String> names = new HashSet<>();
-
-            for (PortableMetadata meta : metaCol) {
-                assertTrue(names.add(meta.typeName()));
-
-                assertNull(meta.affinityKeyFieldName());
-
-                assertEquals(1, meta.fields().size());
-            }
-
-            assertEquals(1000, names.size());
-        }
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testClientStartsFirst() throws Exception {
-        client = true;
-
-        Ignite ignite0 = startGrid(0);
-
-        assertTrue(ignite0.configuration().isClientMode());
-
-        client = false;
-
-        Ignite ignite1 = startGrid(1);
-
-        assertFalse(ignite1.configuration().isClientMode());
-
-        IgnitePortables portables = ignite(1).portables();
-
-        IgniteCache<Object, Object> cache = ignite(1).cache(null).withKeepPortable();
-
-        for (int i = 0; i < 100; i++) {
-            PortableBuilder builder = portables.builder("type-" + i);
-
-            builder.setField("f0", i);
-
-            cache.put(i, builder.build());
-        }
-
-        assertEquals(100, ignite(0).portables().metadata().size());
-    }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/71379a80/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/GridCacheClientNodePortableMetadataTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/GridCacheClientNodePortableMetadataTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/GridCacheClientNodePortableMetadataTest.java
deleted file mode 100644
index a66d940..0000000
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/GridCacheClientNodePortableMetadataTest.java
+++ /dev/null
@@ -1,286 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.ignite.internal.processors.cache.portable;
-
-import java.util.Arrays;
-import java.util.Collection;
-import org.apache.ignite.Ignite;
-import org.apache.ignite.IgniteCache;
-import org.apache.ignite.cache.CacheAtomicityMode;
-import org.apache.ignite.cache.CacheMode;
-import org.apache.ignite.cache.affinity.Affinity;
-import org.apache.ignite.configuration.IgniteConfiguration;
-import org.apache.ignite.configuration.NearCacheConfiguration;
-import org.apache.ignite.internal.processors.cache.GridCacheAbstractSelfTest;
-import org.apache.ignite.marshaller.portable.PortableMarshaller;
-import org.apache.ignite.portable.PortableBuilder;
-import org.apache.ignite.portable.PortableMetadata;
-import org.apache.ignite.portable.PortableObject;
-import org.apache.ignite.portable.PortableTypeConfiguration;
-import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi;
-
-import static org.apache.ignite.cache.CacheAtomicityMode.ATOMIC;
-
-/**
- *
- */
-public class GridCacheClientNodePortableMetadataTest extends GridCacheAbstractSelfTest {
-    /** {@inheritDoc} */
-    @Override protected int gridCount() {
-        return 4;
-    }
-
-    /** {@inheritDoc} */
-    @Override protected CacheMode cacheMode() {
-        return CacheMode.PARTITIONED;
-    }
-
-    /** {@inheritDoc} */
-    @Override protected CacheAtomicityMode atomicityMode() {
-        return ATOMIC;
-    }
-
-    /** {@inheritDoc} */
-    @Override protected NearCacheConfiguration nearConfiguration() {
-        return null;
-    }
-
-    /** {@inheritDoc} */
-    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
-        IgniteConfiguration cfg = super.getConfiguration(gridName);
-
-        PortableMarshaller marsh = new PortableMarshaller();
-
-        marsh.setClassNames(Arrays.asList(TestObject1.class.getName(), TestObject2.class.getName()));
-
-        PortableTypeConfiguration typeCfg = new PortableTypeConfiguration();
-
-        typeCfg.setClassName(TestObject1.class.getName());
-        typeCfg.setAffinityKeyFieldName("val2");
-
-        marsh.setTypeConfigurations(Arrays.asList(typeCfg));
-
-        if (gridName.equals(getTestGridName(gridCount() - 1)))
-            cfg.setClientMode(true);
-
-        cfg.setMarshaller(marsh);
-
-        ((TcpDiscoverySpi)cfg.getDiscoverySpi()).setForceServerMode(true);
-
-        return cfg;
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testPortableMetadataOnClient() throws Exception {
-        Ignite ignite0 = ignite(gridCount() - 1);
-
-        assertTrue(ignite0.configuration().isClientMode());
-
-        Ignite ignite1 = ignite(0);
-
-        assertFalse(ignite1.configuration().isClientMode());
-
-        Affinity<Object> aff0 = ignite0.affinity(null);
-        Affinity<Object> aff1 = ignite1.affinity(null);
-
-        for (int i = 0 ; i < 100; i++) {
-            TestObject1 obj1 = new TestObject1(i, i + 1);
-
-            assertEquals(aff1.mapKeyToPrimaryAndBackups(obj1),
-                aff0.mapKeyToPrimaryAndBackups(obj1));
-
-            TestObject2 obj2 = new TestObject2(i, i + 1);
-
-            assertEquals(aff1.mapKeyToPrimaryAndBackups(obj2),
-                aff0.mapKeyToPrimaryAndBackups(obj2));
-        }
-
-        {
-            PortableBuilder builder = ignite0.portables().builder("TestObject3");
-
-            builder.setField("f1", 1);
-
-            ignite0.cache(null).put(0, builder.build());
-
-            IgniteCache<Integer, PortableObject> cache = ignite0.cache(null).withKeepPortable();
-
-            PortableObject obj = cache.get(0);
-
-            PortableMetadata meta = obj.metaData();
-
-            assertNotNull(meta);
-            assertEquals(1, meta.fields().size());
-
-            meta = ignite0.portables().metadata(TestObject1.class);
-
-            assertNotNull(meta);
-            assertEquals("val2", meta.affinityKeyFieldName());
-
-            meta = ignite0.portables().metadata(TestObject2.class);
-
-            assertNotNull(meta);
-            assertNull(meta.affinityKeyFieldName());
-        }
-
-        {
-            PortableBuilder builder = ignite1.portables().builder("TestObject3");
-
-            builder.setField("f2", 2);
-
-            ignite1.cache(null).put(1, builder.build());
-
-            IgniteCache<Integer, PortableObject> cache = ignite1.cache(null).withKeepPortable();
-
-            PortableObject obj = cache.get(0);
-
-            PortableMetadata meta = obj.metaData();
-
-            assertNotNull(meta);
-            assertEquals(2, meta.fields().size());
-
-            meta = ignite1.portables().metadata(TestObject1.class);
-
-            assertNotNull(meta);
-            assertEquals("val2", meta.affinityKeyFieldName());
-
-            meta = ignite1.portables().metadata(TestObject2.class);
-
-            assertNotNull(meta);
-            assertNull(meta.affinityKeyFieldName());
-        }
-
-        PortableMetadata meta = ignite0.portables().metadata("TestObject3");
-
-        assertNotNull(meta);
-        assertEquals(2, meta.fields().size());
-
-        IgniteCache<Integer, PortableObject> cache = ignite0.cache(null).withKeepPortable();
-
-        PortableObject obj = cache.get(1);
-
-        assertEquals(Integer.valueOf(2), obj.field("f2"));
-        assertNull(obj.field("f1"));
-
-        meta = obj.metaData();
-
-        assertNotNull(meta);
-        assertEquals(2, meta.fields().size());
-
-        Collection<PortableMetadata> meta1 = ignite1.portables().metadata();
-        Collection<PortableMetadata> meta2 = ignite1.portables().metadata();
-
-        assertEquals(meta1.size(), meta2.size());
-
-        for (PortableMetadata m1 : meta1) {
-            boolean found = false;
-
-            for (PortableMetadata m2 : meta1) {
-                if (m1.typeName().equals(m2.typeName())) {
-                    assertEquals(m1.affinityKeyFieldName(), m2.affinityKeyFieldName());
-                    assertEquals(m1.fields(), m2.fields());
-
-                    found = true;
-
-                    break;
-                }
-            }
-
-            assertTrue(found);
-        }
-    }
-
-    /**
-     *
-     */
-    static class TestObject1 {
-        /** */
-        private int val1;
-
-        /** */
-        private int val2;
-
-        /**
-         * @param val1 Value 1.
-         * @param val2 Value 2.
-         */
-        public TestObject1(int val1, int val2) {
-            this.val1 = val1;
-            this.val2 = val2;
-        }
-
-        /** {@inheritDoc} */
-        @Override public boolean equals(Object o) {
-            if (this == o)
-                return true;
-
-            if (o == null || getClass() != o.getClass())
-                return false;
-
-            TestObject1 that = (TestObject1)o;
-
-            return val1 == that.val1;
-
-        }
-
-        /** {@inheritDoc} */
-        @Override public int hashCode() {
-            return val1;
-        }
-    }
-
-    /**
-     *
-     */
-    static class TestObject2 {
-        /** */
-        private int val1;
-
-        /** */
-        private int val2;
-
-        /**
-         * @param val1 Value 1.
-         * @param val2 Value 2.
-         */
-        public TestObject2(int val1, int val2) {
-            this.val1 = val1;
-            this.val2 = val2;
-        }
-
-        /** {@inheritDoc} */
-        @Override public boolean equals(Object o) {
-            if (this == o)
-                return true;
-
-            if (o == null || getClass() != o.getClass())
-                return false;
-
-            TestObject2 that = (TestObject2)o;
-
-            return val2 == that.val2;
-
-        }
-
-        /** {@inheritDoc} */
-        @Override public int hashCode() {
-            return val2;
-        }
-    }
-}
\ No newline at end of file


[45/55] [abbrv] ignite git commit: IGFS: Reverted back some test changes.

Posted by ag...@apache.org.
IGFS: Reverted back some test changes.


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

Branch: refs/heads/ignite-1171
Commit: 126e68bee16d7f31a7c1ae6810fe40d5ea3f8a34
Parents: ae1e1dc
Author: vozerov-gridgain <vo...@gridgain.com>
Authored: Tue Sep 15 17:03:00 2015 +0300
Committer: vozerov-gridgain <vo...@gridgain.com>
Committed: Tue Sep 15 17:03:00 2015 +0300

----------------------------------------------------------------------
 .../ignite/internal/processors/igfs/IgfsAbstractSelfTest.java | 7 ++++---
 .../internal/processors/igfs/IgfsDualAbstractSelfTest.java    | 6 +++---
 2 files changed, 7 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/126e68be/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsAbstractSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsAbstractSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsAbstractSelfTest.java
index 076c269..3e63cdc 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsAbstractSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsAbstractSelfTest.java
@@ -2295,13 +2295,13 @@ public abstract class IgfsAbstractSelfTest extends IgfsCommonAbstractTest {
     /**
      * Create the file in the given IGFS and write provided data chunks to it.
      *
-     * @param uni File system adapter.
      * @param file File.
+     * @param overwrite Overwrite flag.
      * @param chunks Data chunks.
      * @throws IOException In case of IO exception.
      */
-    protected static void createFile(UniversalFileSystemAdapter uni, IgfsPath file, @Nullable byte[]... chunks)
-        throws IOException {
+    protected static void createFile(UniversalFileSystemAdapter uni, IgfsPath file, boolean overwrite,
+        @Nullable byte[]... chunks) throws IOException {
         OutputStream os = null;
 
         try {
@@ -2313,6 +2313,7 @@ public abstract class IgfsAbstractSelfTest extends IgfsCommonAbstractTest {
             U.closeQuiet(os);
 
             IgfsEx igfsEx = uni.getAdapter(IgfsEx.class);
+
             if (igfsEx != null)
                 awaitFileClose(igfsEx.asSecondary(), file);
         }

http://git-wip-us.apache.org/repos/asf/ignite/blob/126e68be/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsDualAbstractSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsDualAbstractSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsDualAbstractSelfTest.java
index 683054b..9527459 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsDualAbstractSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsDualAbstractSelfTest.java
@@ -1129,7 +1129,7 @@ public abstract class IgfsDualAbstractSelfTest extends IgfsAbstractSelfTest {
         create(igfsSecondary, paths(DIR, SUBDIR), null);
         create(igfs, null, null);
 
-        createFile(igfsSecondary, FILE, chunk);
+        createFile(igfsSecondary, FILE, true, chunk);
 
         checkFileContent(igfs, FILE, chunk);
     }
@@ -1348,7 +1348,7 @@ public abstract class IgfsDualAbstractSelfTest extends IgfsAbstractSelfTest {
 
         igfsSecondaryFileSystem.update(SUBDIR, props);
 
-        createFile(igfsSecondary, FILE, /*BLOCK_SIZE,*/ chunk);
+        createFile(igfsSecondary, FILE, true, /*BLOCK_SIZE,*/ chunk);
 
         appendFile(igfs, FILE, chunk);
 
@@ -1375,7 +1375,7 @@ public abstract class IgfsDualAbstractSelfTest extends IgfsAbstractSelfTest {
         igfsSecondaryFileSystem.update(DIR, propsDir);
         igfsSecondaryFileSystem.update(SUBDIR, propsSubDir);
 
-        createFile(igfsSecondary, FILE, /*BLOCK_SIZE,*/ chunk);
+        createFile(igfsSecondary, FILE, true, /*BLOCK_SIZE,*/ chunk);
 
         appendFile(igfs, FILE, chunk);
 


[48/55] [abbrv] ignite git commit: added near eviction policy for yardstick

Posted by ag...@apache.org.
added near eviction policy for yardstick


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

Branch: refs/heads/ignite-1171
Commit: 8e2520474fdf6e073a54e5d8a23b32fb266d5b4c
Parents: 6bd0f8f
Author: Yakov Zhdanov <yz...@gridgain.com>
Authored: Tue Sep 15 17:09:23 2015 +0300
Committer: Yakov Zhdanov <yz...@gridgain.com>
Committed: Tue Sep 15 17:09:23 2015 +0300

----------------------------------------------------------------------
 .../yardstick/IgniteBenchmarkArguments.java     | 24 ++++++++++----------
 .../org/apache/ignite/yardstick/IgniteNode.java | 12 +++++++---
 2 files changed, 21 insertions(+), 15 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/8e252047/modules/yardstick/src/main/java/org/apache/ignite/yardstick/IgniteBenchmarkArguments.java
----------------------------------------------------------------------
diff --git a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/IgniteBenchmarkArguments.java b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/IgniteBenchmarkArguments.java
index 9710443..6e9059d 100644
--- a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/IgniteBenchmarkArguments.java
+++ b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/IgniteBenchmarkArguments.java
@@ -53,6 +53,10 @@ public class IgniteBenchmarkArguments {
     private boolean nearCacheFlag = false;
 
     /** */
+    @Parameter(names = {"-ncs", "--nearCacheSize"}, description = "Near cache size")
+    private int nearCacheSize;
+
+    /** */
     @Parameter(names = {"-wom", "--writeOrderMode"}, description = "Write ordering mode")
     private CacheAtomicWriteOrderMode orderMode;
 
@@ -81,10 +85,6 @@ public class IgniteBenchmarkArguments {
     private String restTcpHost;
 
     /** */
-    @Parameter(names = {"-ss", "--syncSend"}, description = "Synchronous send")
-    private boolean syncSnd;
-
-    /** */
     @Parameter(names = {"-r", "--range"}, description = "Key range")
     private int range = 1_000_000;
 
@@ -162,6 +162,13 @@ public class IgniteBenchmarkArguments {
     }
 
     /**
+     * @return Near cache size ({@code 0} for unlimited).
+     */
+    public int getNearCacheSize() {
+        return nearCacheSize;
+    }
+
+    /**
      * @return Synchronization.
      */
     public CacheWriteSynchronizationMode syncMode() {
@@ -211,13 +218,6 @@ public class IgniteBenchmarkArguments {
     }
 
     /**
-     * @return {@code True} if sending is synchronous.
-     */
-    public boolean isSyncSend() {
-        return syncSnd;
-    }
-
-    /**
      * @return Key range, from {@code 0} to this number.
      */
     public int range() {
@@ -278,4 +278,4 @@ public class IgniteBenchmarkArguments {
     @Override public String toString() {
         return GridToStringBuilder.toString(IgniteBenchmarkArguments.class, this);
     }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/8e252047/modules/yardstick/src/main/java/org/apache/ignite/yardstick/IgniteNode.java
----------------------------------------------------------------------
diff --git a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/IgniteNode.java b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/IgniteNode.java
index 8c3bc45..ea3bd07 100644
--- a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/IgniteNode.java
+++ b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/IgniteNode.java
@@ -86,8 +86,14 @@ public class IgniteNode implements BenchmarkServer {
             if (cl)
                 c.setClientMode(true);
 
-            if (args.isNearCache())
-                cc.setNearConfiguration(new NearCacheConfiguration());
+            if (args.isNearCache()) {
+                NearCacheConfiguration nearCfg = new NearCacheConfiguration();
+
+                if (args.getNearCacheSize() != 0)
+                    nearCfg.setNearEvictionPolicy(new LruEvictionPolicy(args.getNearCacheSize()));
+
+                cc.setNearConfiguration(nearCfg);
+            }
 
             cc.setWriteSynchronizationMode(args.syncMode());
 
@@ -204,4 +210,4 @@ public class IgniteNode implements BenchmarkServer {
     public Ignite ignite() {
         return ignite;
     }
-}
\ No newline at end of file
+}


[43/55] [abbrv] ignite git commit: fixed javadoc ignite-1.4

Posted by ag...@apache.org.
fixed javadoc 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/72e270c6
Tree: http://git-wip-us.apache.org/repos/asf/ignite/tree/72e270c6
Diff: http://git-wip-us.apache.org/repos/asf/ignite/diff/72e270c6

Branch: refs/heads/ignite-1171
Commit: 72e270c63e99ac17a077b26fb39302f603d15188
Parents: 3f75c1c
Author: Yakov Zhdanov <yz...@gridgain.com>
Authored: Tue Sep 15 16:55:00 2015 +0300
Committer: Yakov Zhdanov <yz...@gridgain.com>
Committed: Tue Sep 15 16:55:00 2015 +0300

----------------------------------------------------------------------
 .../configuration/NearCacheConfiguration.java     | 18 ++++++++++++++++--
 1 file changed, 16 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/72e270c6/modules/core/src/main/java/org/apache/ignite/configuration/NearCacheConfiguration.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/configuration/NearCacheConfiguration.java b/modules/core/src/main/java/org/apache/ignite/configuration/NearCacheConfiguration.java
index bbaa030..3c59bd6 100644
--- a/modules/core/src/main/java/org/apache/ignite/configuration/NearCacheConfiguration.java
+++ b/modules/core/src/main/java/org/apache/ignite/configuration/NearCacheConfiguration.java
@@ -25,7 +25,12 @@ import org.apache.ignite.internal.util.typedef.internal.S;
 import static org.apache.ignite.configuration.CacheConfiguration.DFLT_NEAR_START_SIZE;
 
 /**
- * Client cache configuration.
+ * Client (near) cache configuration.
+ * <p>
+ * Distributed cache can also be fronted by a Near cache,
+ * which is a smaller local cache that stores most recently
+ * or most frequently accessed data. Just like with a partitioned cache,
+ * the user can control the size of the near cache and its eviction policies.
  */
 public class NearCacheConfiguration<K, V> extends MutableConfiguration<K, V> {
     /** */
@@ -45,6 +50,8 @@ public class NearCacheConfiguration<K, V> extends MutableConfiguration<K, V> {
     }
 
     /**
+     * Creates near cache configuration copying properties from passed in configuration.
+     *
      * @param ccfg Configuration to copy.
      */
     public NearCacheConfiguration(NearCacheConfiguration<K, V> ccfg) {
@@ -55,13 +62,20 @@ public class NearCacheConfiguration<K, V> extends MutableConfiguration<K, V> {
     }
 
     /**
+     * Gets near eviction policy. By default, returns {@code null}
+     * which means that evictions are disabled for near cache.
+     *
      * @return Near eviction policy.
+     * @see CacheConfiguration#getEvictionPolicy()
+     * @see CacheConfiguration#isEvictSynchronized()
      */
     public EvictionPolicy<K, V> getNearEvictionPolicy() {
         return nearEvictPlc;
     }
 
     /**
+     * Sets near eviction policy.
+     *
      * @param nearEvictPlc Near eviction policy.
      * @return {@code this} for chaining.
      */
@@ -97,4 +111,4 @@ public class NearCacheConfiguration<K, V> extends MutableConfiguration<K, V> {
     @Override public String toString() {
         return S.toString(NearCacheConfiguration.class, this, super.toString());
     }
-}
\ No newline at end of file
+}


[51/55] [abbrv] ignite git commit: IGNITE-1239 - Fixed scan query failover on changing topology.

Posted by ag...@apache.org.
IGNITE-1239 - Fixed scan query failover on changing topology.


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

Branch: refs/heads/ignite-1171
Commit: 5f5220af0449fd3957d011fcef954b19da852e18
Parents: d928ef4
Author: sboikov <sb...@gridgain.com>
Authored: Wed Sep 16 18:06:40 2015 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Wed Sep 16 18:06:40 2015 +0300

----------------------------------------------------------------------
 .../query/GridCacheDistributedQueryFuture.java  |  27 ++-
 .../cache/query/GridCacheLocalQueryFuture.java  |   5 +
 .../cache/query/GridCacheQueryAdapter.java      | 170 ++++++++------
 .../query/GridCacheQueryFutureAdapter.java      |  11 +-
 .../cache/query/GridCacheQueryManager.java      |  30 ++-
 .../GridCacheSwapScanQueryAbstractSelfTest.java | 118 +++++-----
 ...CacheScanPartitionQueryFallbackSelfTest.java | 224 ++++++-------------
 7 files changed, 287 insertions(+), 298 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/5f5220af/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheDistributedQueryFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheDistributedQueryFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheDistributedQueryFuture.java
index 1d547c5..e745e30 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheDistributedQueryFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheDistributedQueryFuture.java
@@ -27,6 +27,7 @@ import java.util.concurrent.CountDownLatch;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.cluster.ClusterNode;
 import org.apache.ignite.internal.IgniteInterruptedCheckedException;
+import org.apache.ignite.internal.cluster.ClusterTopologyCheckedException;
 import org.apache.ignite.internal.processors.cache.GridCacheContext;
 import org.apache.ignite.internal.util.typedef.F;
 import org.apache.ignite.internal.util.typedef.P1;
@@ -142,9 +143,24 @@ public class GridCacheDistributedQueryFuture<K, V, R> extends GridCacheQueryFutu
         }
 
         if (callOnPage)
-            // We consider node departure as a reception of last empty
-            // page from this node.
-            onPage(nodeId, Collections.emptyList(), null, true);
+            onPage(nodeId, Collections.emptyList(),
+                new ClusterTopologyCheckedException("Remote node has left topology: " + nodeId), true);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void awaitFirstPage() throws IgniteCheckedException {
+        try {
+            firstPageLatch.await();
+
+            if (isDone() && error() != null)
+                // Throw the exception if future failed.
+                get();
+        }
+        catch (InterruptedException e) {
+            Thread.currentThread().interrupt();
+
+            throw new IgniteInterruptedCheckedException(e);
+        }
     }
 
     /** {@inheritDoc} */
@@ -229,9 +245,12 @@ public class GridCacheDistributedQueryFuture<K, V, R> extends GridCacheQueryFutu
 
     /** {@inheritDoc} */
     @Override public boolean onDone(Collection<R> res, Throwable err) {
+        boolean done = super.onDone(res, err);
+
+        // Must release the lath after onDone() in order for a waiting thread to see an exception, if any.
         firstPageLatch.countDown();
 
-        return super.onDone(res, err);
+        return done;
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/5f5220af/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheLocalQueryFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheLocalQueryFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheLocalQueryFuture.java
index 46af18a..248dfa8 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheLocalQueryFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheLocalQueryFuture.java
@@ -77,6 +77,11 @@ public class GridCacheLocalQueryFuture<K, V, R> extends GridCacheQueryFutureAdap
         // No-op.
     }
 
+    /** {@inheritDoc} */
+    @Override public void awaitFirstPage() throws IgniteCheckedException {
+        get();
+    }
+
     /** */
     private class LocalQueryRunnable implements GridPlainRunnable {
         /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/5f5220af/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryAdapter.java
index 3ac5746..855e239 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryAdapter.java
@@ -19,6 +19,7 @@ package org.apache.ignite.internal.processors.cache.query;
 
 import java.util.Collection;
 import java.util.Collections;
+import java.util.Deque;
 import java.util.HashSet;
 import java.util.LinkedList;
 import java.util.Queue;
@@ -34,6 +35,7 @@ import org.apache.ignite.cluster.ClusterNode;
 import org.apache.ignite.internal.IgniteClientDisconnectedCheckedException;
 import org.apache.ignite.internal.IgniteInternalFuture;
 import org.apache.ignite.internal.cluster.ClusterGroupEmptyCheckedException;
+import org.apache.ignite.internal.cluster.ClusterTopologyCheckedException;
 import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
 import org.apache.ignite.internal.processors.cache.GridCacheContext;
 import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtUnreservedPartitionException;
@@ -41,14 +43,13 @@ import org.apache.ignite.internal.processors.query.GridQueryProcessor;
 import org.apache.ignite.internal.util.future.GridFutureAdapter;
 import org.apache.ignite.internal.util.typedef.F;
 import org.apache.ignite.internal.util.typedef.P1;
+import org.apache.ignite.internal.util.typedef.X;
 import org.apache.ignite.internal.util.typedef.internal.A;
 import org.apache.ignite.internal.util.typedef.internal.CU;
 import org.apache.ignite.internal.util.typedef.internal.S;
 import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.lang.IgniteBiPredicate;
 import org.apache.ignite.lang.IgniteClosure;
-import org.apache.ignite.lang.IgniteInClosure;
-import org.apache.ignite.lang.IgnitePredicate;
 import org.apache.ignite.lang.IgniteReducer;
 import org.apache.ignite.plugin.security.SecurityPermission;
 import org.jetbrains.annotations.Nullable;
@@ -63,13 +64,6 @@ import static org.apache.ignite.internal.processors.cache.query.GridCacheQueryTy
  * Query adapter.
  */
 public class GridCacheQueryAdapter<T> implements CacheQuery<T> {
-    /** Is local node predicate. */
-    private static final IgnitePredicate<ClusterNode> IS_LOC_NODE = new IgnitePredicate<ClusterNode>() {
-        @Override public boolean apply(ClusterNode n) {
-            return n.isLocal();
-        }
-    };
-
     /** */
     private final GridCacheContext<?, ?> cctx;
 
@@ -446,7 +440,7 @@ public class GridCacheQueryAdapter<T> implements CacheQuery<T> {
 
         cctx.checkSecurity(SecurityPermission.CACHE_READ);
 
-        if (nodes.isEmpty())
+        if (nodes.isEmpty() && (type != SCAN || part == null))
             return new GridCacheQueryErrorFuture<>(cctx.kernalContext(), new ClusterGroupEmptyCheckedException());
 
         if (log.isDebugEnabled())
@@ -477,8 +471,8 @@ public class GridCacheQueryAdapter<T> implements CacheQuery<T> {
         if (type == SQL_FIELDS || type == SPI)
             return (CacheQueryFuture<R>)(loc ? qryMgr.queryFieldsLocal(bean) :
                 qryMgr.queryFieldsDistributed(bean, nodes));
-        else if (type == SCAN && part != null && nodes.size() > 1)
-            return new CacheQueryFallbackFuture<>(nodes, part, bean, qryMgr, cctx);
+        else if (type == SCAN && part != null && !cctx.isLocal())
+            return new CacheQueryFallbackFuture<>(part, bean, qryMgr, cctx);
         else
             return (CacheQueryFuture<R>)(loc ? qryMgr.queryLocal(bean) : qryMgr.queryDistributed(bean, nodes));
     }
@@ -581,37 +575,48 @@ public class GridCacheQueryAdapter<T> implements CacheQuery<T> {
         /** Partition. */
         private final int part;
 
+        /** Flag indicating that a first item has been returned to a user. */
+        private boolean firstItemReturned;
+
         /**
-         * @param nodes Backups.
          * @param part Partition.
          * @param bean Bean.
          * @param qryMgr Query manager.
          * @param cctx Cache context.
          */
-        public CacheQueryFallbackFuture(Collection<ClusterNode> nodes, int part, GridCacheQueryBean bean,
+        private CacheQueryFallbackFuture(int part, GridCacheQueryBean bean,
             GridCacheQueryManager qryMgr, GridCacheContext cctx) {
-            this.nodes = fallbacks(nodes);
             this.bean = bean;
             this.qryMgr = qryMgr;
             this.cctx = cctx;
             this.part = part;
 
+            nodes = fallbacks(cctx.discovery().topologyVersionEx());
+
             init();
         }
 
         /**
-         * @param nodes Nodes.
+         * @param topVer Topology version.
          * @return Nodes for query execution.
          */
-        private Queue<ClusterNode> fallbacks(Collection<ClusterNode> nodes) {
-            Queue<ClusterNode> fallbacks = new LinkedList<>();
+        private Queue<ClusterNode> fallbacks(AffinityTopologyVersion topVer) {
+            Deque<ClusterNode> fallbacks = new LinkedList<>();
+            Collection<ClusterNode> owners = new HashSet<>();
 
-            ClusterNode node = F.first(F.view(nodes, IS_LOC_NODE));
+            for (ClusterNode node : cctx.topology().owners(part, topVer)) {
+                if (node.isLocal())
+                    fallbacks.addFirst(node);
+                else
+                    fallbacks.add(node);
 
-            if (node != null)
-                fallbacks.add(node);
+                owners.add(node);
+            }
 
-            fallbacks.addAll(node != null ? F.view(nodes, F.not(IS_LOC_NODE)) : nodes);
+            for (ClusterNode node : cctx.topology().moving(part)) {
+                if (!owners.contains(node))
+                    fallbacks.add(node);
+            }
 
             return fallbacks;
         }
@@ -623,53 +628,9 @@ public class GridCacheQueryAdapter<T> implements CacheQuery<T> {
         private void init() {
             final ClusterNode node = nodes.poll();
 
-            GridCacheQueryFutureAdapter<?, ?, R> fut0 = (GridCacheQueryFutureAdapter<?, ?, R>)(node.isLocal() ?
+            fut = (GridCacheQueryFutureAdapter<?, ?, R>)(node.isLocal() ?
                 qryMgr.queryLocal(bean) :
                 qryMgr.queryDistributed(bean, Collections.singleton(node)));
-
-            fut0.listen(new IgniteInClosure<IgniteInternalFuture<Collection<R>>>() {
-                @Override public void apply(IgniteInternalFuture<Collection<R>> fut) {
-                    try {
-                        onDone(fut.get());
-                    }
-                    catch (IgniteClientDisconnectedCheckedException e) {
-                        onDone(e);
-                    }
-                    catch (IgniteCheckedException e) {
-                        if (e.hasCause(GridDhtUnreservedPartitionException.class)) {
-                            unreservedTopVer = ((GridDhtUnreservedPartitionException)e.getCause()).topologyVersion();
-
-                            assert unreservedTopVer != null;
-                        }
-
-                        if (F.isEmpty(nodes)) {
-                            final AffinityTopologyVersion topVer = unreservedTopVer;
-
-                            if (topVer != null && --unreservedNodesRetryCnt > 0) {
-                                cctx.affinity().affinityReadyFuture(topVer).listen(
-                                    new IgniteInClosure<IgniteInternalFuture<AffinityTopologyVersion>>() {
-                                        @Override public void apply(
-                                            IgniteInternalFuture<AffinityTopologyVersion> future) {
-
-                                            nodes = fallbacks(cctx.topology().owners(part, topVer));
-
-                                            // Race is impossible here because query retries are executed one by one.
-                                            unreservedTopVer = null;
-
-                                            init();
-                                        }
-                                    });
-                            }
-                            else
-                                onDone(e);
-                        }
-                        else
-                            init();
-                    }
-                }
-            });
-
-            fut = fut0;
         }
 
         /** {@inheritDoc} */
@@ -683,8 +644,81 @@ public class GridCacheQueryAdapter<T> implements CacheQuery<T> {
         }
 
         /** {@inheritDoc} */
+        @Override public Collection<R> get() throws IgniteCheckedException {
+            assert false;
+
+            return super.get();
+        }
+
+        /** {@inheritDoc} */
         @Override public R next() {
-            return fut.next();
+            if (firstItemReturned)
+                return fut.next();
+
+            while (true) {
+                try {
+                    fut.awaitFirstPage();
+
+                    firstItemReturned = true;
+
+                    return fut.next();
+                }
+                catch (IgniteClientDisconnectedCheckedException e) {
+                    throw CU.convertToCacheException(e);
+                }
+                catch (IgniteCheckedException e) {
+                    retryIfPossible(e);
+                }
+            }
+        }
+
+        /**
+         * @param e Exception for query run.
+         */
+        private void retryIfPossible(IgniteCheckedException e) {
+            try {
+                IgniteInternalFuture<?> retryFut;
+
+                if (e.hasCause(GridDhtUnreservedPartitionException.class)) {
+                    AffinityTopologyVersion waitVer = ((GridDhtUnreservedPartitionException)e.getCause()).topologyVersion();
+
+                    assert waitVer != null;
+
+                    retryFut = cctx.affinity().affinityReadyFuture(waitVer);
+                }
+                else if (e.hasCause(ClusterTopologyCheckedException.class)) {
+                    ClusterTopologyCheckedException topEx = X.cause(e, ClusterTopologyCheckedException.class);
+
+                    retryFut = topEx.retryReadyFuture();
+                }
+                else if (e.hasCause(ClusterGroupEmptyCheckedException.class)) {
+                    ClusterGroupEmptyCheckedException ex = X.cause(e, ClusterGroupEmptyCheckedException.class);
+
+                    retryFut = ex.retryReadyFuture();
+                }
+                else
+                    throw CU.convertToCacheException(e);
+
+                if (F.isEmpty(nodes)) {
+                    if (--unreservedNodesRetryCnt > 0) {
+                        if (retryFut != null)
+                            retryFut.get();
+
+                        nodes = fallbacks(unreservedTopVer == null ? cctx.discovery().topologyVersionEx() : unreservedTopVer);
+
+                        unreservedTopVer = null;
+
+                        init();
+                    }
+                    else
+                        throw CU.convertToCacheException(e);
+                }
+                else
+                    init();
+            }
+            catch (IgniteCheckedException ex) {
+                throw CU.convertToCacheException(ex);
+            }
         }
     }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/5f5220af/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryFutureAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryFutureAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryFutureAdapter.java
index ad9ee39..2a4fbda 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryFutureAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryFutureAdapter.java
@@ -183,6 +183,13 @@ public abstract class GridCacheQueryFutureAdapter<K, V, R> extends GridFutureAda
     }
 
     /**
+     * Waits for the first page to be received from remote node(s), if any.
+     *
+     * @throws IgniteCheckedException If query execution failed with an error.
+     */
+    public abstract void awaitFirstPage() throws IgniteCheckedException;
+
+    /**
      * Returns next page for the query.
      *
      * @return Next page or {@code null} if no more pages available.
@@ -380,13 +387,13 @@ public abstract class GridCacheQueryFutureAdapter<K, V, R> extends GridFutureAda
                 synchronized (mux) {
                     enqueue(Collections.emptyList());
 
-                    onPage(nodeId, true);
-
                     onDone(nodeId != null ?
                         new IgniteCheckedException("Failed to execute query on node [query=" + qry +
                             ", nodeId=" + nodeId + "]", err) :
                         new IgniteCheckedException("Failed to execute query locally: " + qry, err));
 
+                    onPage(nodeId, true);
+
                     mux.notifyAll();
                 }
             else {

http://git-wip-us.apache.org/repos/asf/ignite/blob/5f5220af/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryManager.java
index 1d934d8..25ace1b 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryManager.java
@@ -33,8 +33,9 @@ import org.apache.ignite.internal.managers.eventstorage.GridLocalEventListener;
 import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
 import org.apache.ignite.internal.processors.cache.CacheMetricsImpl;
 import org.apache.ignite.internal.processors.cache.CacheObject;
-import org.apache.ignite.internal.processors.cache.CachePeekModes;
 import org.apache.ignite.internal.processors.cache.GridCacheAdapter;
+import org.apache.ignite.internal.processors.cache.GridCacheEntryEx;
+import org.apache.ignite.internal.processors.cache.GridCacheEntryRemovedException;
 import org.apache.ignite.internal.processors.cache.GridCacheInternal;
 import org.apache.ignite.internal.processors.cache.GridCacheManagerAdapter;
 import org.apache.ignite.internal.processors.cache.GridCacheOffheapSwapEntry;
@@ -819,8 +820,12 @@ public abstract class GridCacheQueryManager<K, V> extends GridCacheManagerAdapte
 
         final GridDhtCacheAdapter dht = cctx.isLocal() ? null : (cctx.isNear() ? cctx.near().dht() : cctx.dht());
 
+        final GridCacheAdapter cache = dht != null ? dht : cctx.cache();
+
         final ExpiryPolicy plc = cctx.expiry();
 
+        final AffinityTopologyVersion topVer = cctx.affinity().affinityTopologyVersion();
+
         final boolean backups = qry.includeBackups() || cctx.isReplicated();
 
         final GridCloseableIteratorAdapter<IgniteBiTuple<K, V>> heapIt =
@@ -841,8 +846,6 @@ public abstract class GridCacheQueryManager<K, V> extends GridCacheManagerAdapte
                     else if (part < 0 || part >= cctx.affinity().partitions())
                         iter = F.emptyIterator();
                     else {
-                        AffinityTopologyVersion topVer = cctx.affinity().affinityTopologyVersion();
-
                         locPart = dht.topology().localPartition(part, topVer, false);
 
                         // double check for owning state
@@ -899,7 +902,15 @@ public abstract class GridCacheQueryManager<K, V> extends GridCacheManagerAdapte
                         V val;
 
                         try {
-                            val = prj.localPeek(key, CachePeekModes.ONHEAP_ONLY, expiryPlc);
+                            GridCacheEntryEx entry = cache.peekEx(key);
+
+                            CacheObject cacheVal =
+                                entry != null ? entry.peek(true, false, false, topVer, expiryPlc) : null;
+
+                            val = cacheVal != null ? (V)cacheVal.value(cctx.cacheObjectContext(), false) : null;
+                        }
+                        catch (GridCacheEntryRemovedException e) {
+                            val = null;
                         }
                         catch (IgniteCheckedException e) {
                             if (log.isDebugEnabled())
@@ -1557,7 +1568,8 @@ public abstract class GridCacheQueryManager<K, V> extends GridCacheManagerAdapte
                 }
             }
             catch (Throwable e) {
-                U.error(log, "Failed to run query [qry=" + qryInfo + ", node=" + cctx.nodeId() + "]", e);
+                if (!X.hasCause(e, GridDhtUnreservedPartitionException.class))
+                    U.error(log, "Failed to run query [qry=" + qryInfo + ", node=" + cctx.nodeId() + "]", e);
 
                 onPageReady(loc, qryInfo, null, true, e);
 
@@ -1572,8 +1584,9 @@ public abstract class GridCacheQueryManager<K, V> extends GridCacheManagerAdapte
                             res.closeIfNotShared(recipient(qryInfo.senderId(), qryInfo.requestId()));
                         }
                         catch (IgniteCheckedException e) {
-                            U.error(log, "Failed to close local iterator [qry=" + qryInfo + ", node=" +
-                                cctx.nodeId() + "]", e);
+                            if (!X.hasCause(e, GridDhtUnreservedPartitionException.class))
+                                U.error(log, "Failed to close local iterator [qry=" + qryInfo + ", node=" +
+                                    cctx.nodeId() + "]", e);
                         }
                     }
                 }
@@ -1694,7 +1707,8 @@ public abstract class GridCacheQueryManager<K, V> extends GridCacheManagerAdapte
                     fut.get().closeIfNotShared(recipient(sndId, reqId));
                 }
                 catch (IgniteCheckedException e) {
-                    U.error(log, "Failed to close iterator.", e);
+                    if (!X.hasCause(e, GridDhtUnreservedPartitionException.class))
+                        U.error(log, "Failed to close iterator.", e);
                 }
             }
         }

http://git-wip-us.apache.org/repos/asf/ignite/blob/5f5220af/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/GridCacheSwapScanQueryAbstractSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/GridCacheSwapScanQueryAbstractSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/GridCacheSwapScanQueryAbstractSelfTest.java
index 6f4a4ab..94c8d00 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/GridCacheSwapScanQueryAbstractSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/GridCacheSwapScanQueryAbstractSelfTest.java
@@ -19,13 +19,16 @@ package org.apache.ignite.internal.processors.cache.query;
 
 import java.util.Collection;
 import java.util.HashMap;
+import java.util.List;
 import java.util.Map;
 import java.util.concurrent.Callable;
+import javax.cache.Cache;
+import org.apache.ignite.IgniteCache;
 import org.apache.ignite.cache.CacheAtomicityMode;
+import org.apache.ignite.cache.affinity.Affinity;
+import org.apache.ignite.cache.query.ScanQuery;
 import org.apache.ignite.configuration.CacheConfiguration;
 import org.apache.ignite.configuration.IgniteConfiguration;
-import org.apache.ignite.internal.IgniteKernal;
-import org.apache.ignite.internal.processors.cache.GridCacheAdapter;
 import org.apache.ignite.lang.IgniteBiPredicate;
 import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi;
 import org.apache.ignite.spi.discovery.tcp.ipfinder.TcpDiscoveryIpFinder;
@@ -115,13 +118,13 @@ public abstract class GridCacheSwapScanQueryAbstractSelfTest extends GridCommonA
      * @throws Exception If failed.
      */
     public void testQuery() throws Exception {
-        checkQuery(((IgniteKernal)grid(0)).internalCache(ATOMIC_CACHE_NAME), false);
+        checkQuery(grid(0).cache(ATOMIC_CACHE_NAME), false);
 
-        checkQuery(((IgniteKernal)grid(0)).internalCache(TRANSACTIONAL_CACHE_NAME), false);
+        checkQuery(grid(0).cache(TRANSACTIONAL_CACHE_NAME), false);
 
-        checkQuery(((IgniteKernal)grid(0)).internalCache(ATOMIC_CACHE_NAME), true);
+        checkQuery(grid(0).cache(ATOMIC_CACHE_NAME), true);
 
-        checkQuery(((IgniteKernal)grid(0)).internalCache(TRANSACTIONAL_CACHE_NAME), true);
+        checkQuery(grid(0).cache(TRANSACTIONAL_CACHE_NAME), true);
     }
 
     /**
@@ -130,16 +133,18 @@ public abstract class GridCacheSwapScanQueryAbstractSelfTest extends GridCommonA
      * @throws Exception If failed.
      */
     @SuppressWarnings("unchecked")
-    private void checkQuery(GridCacheAdapter cache, boolean scanPartitions) throws Exception {
+    private void checkQuery(IgniteCache cache, boolean scanPartitions) throws Exception {
         final int ENTRY_CNT = 500;
 
         Map<Integer, Map<Key, Person>> entries = new HashMap<>();
 
+        Affinity<Object> aff = ignite(0).affinity(cache.getName());
+
         for (int i = 0; i < ENTRY_CNT; i++) {
             Key key = new Key(i);
             Person val = new Person("p-" + i, i);
 
-            int part = cache.context().affinity().partition(key);
+            int part = aff.partition(key);
 
             cache.getAndPut(key, val);
 
@@ -152,24 +157,26 @@ public abstract class GridCacheSwapScanQueryAbstractSelfTest extends GridCommonA
         }
 
         try {
-            int partitions = scanPartitions ? cache.context().affinity().partitions() : 1;
+            int partitions = scanPartitions ? aff.partitions() : 1;
 
             for (int i = 0; i < partitions; i++) {
-                CacheQuery<Map.Entry<Key, Person>> qry = cache.context().queries().createScanQuery(
-                    new IgniteBiPredicate<Key, Person>() {
-                        @Override public boolean apply(Key key, Person p) {
-                            assertEquals(key.id, (Integer)p.salary);
+                ScanQuery<Key, Person> qry = new ScanQuery<>(new IgniteBiPredicate<Key, Person>() {
+                    @Override public boolean apply(Key key, Person p) {
+                        assertEquals(key.id, (Integer)p.salary);
+
+                        return key.id % 2 == 0;
+                    }
+                });
 
-                            return key.id % 2 == 0;
-                        }
-                    }, (scanPartitions ? i : null), false);
+                if (scanPartitions)
+                    qry.setPartition(i);
 
-                Collection<Map.Entry<Key, Person>> res = qry.execute().get();
+                List<Cache.Entry<Key, Person>> res = cache.query(qry).getAll();
 
                 if (!scanPartitions)
                     assertEquals(ENTRY_CNT / 2, res.size());
 
-                for (Map.Entry<Key, Person> e : res) {
+                for (Cache.Entry<Key, Person> e : res) {
                     Key k = e.getKey();
                     Person p = e.getValue();
 
@@ -183,9 +190,12 @@ public abstract class GridCacheSwapScanQueryAbstractSelfTest extends GridCommonA
                     }
                 }
 
-                qry = cache.context().queries().createScanQuery(null, (scanPartitions ? i : null), false);
+                qry = new ScanQuery<>();
+
+                if (scanPartitions)
+                    qry.setPartition(i);
 
-                res = qry.execute().get();
+                res = cache.query(qry).getAll();
 
                 if (!scanPartitions)
                     assertEquals(ENTRY_CNT, res.size());
@@ -204,23 +214,22 @@ public abstract class GridCacheSwapScanQueryAbstractSelfTest extends GridCommonA
      * @param expCnt Expected entries in query result.
      * @throws Exception If failed.
      */
-    private void testMultithreaded(final GridCacheAdapter cache, final int expCnt) throws Exception {
+    private void testMultithreaded(final IgniteCache cache, final int expCnt) throws Exception {
         log.info("Starting multithreaded queries.");
 
         GridTestUtils.runMultiThreaded(new Callable<Void>() {
             @SuppressWarnings("unchecked")
             @Override public Void call() throws Exception {
-                CacheQuery<Map.Entry<Key, Person>> qry = cache.context().queries().createScanQuery(
-                    new IgniteBiPredicate<Key, Person>() {
-                        @Override public boolean apply(Key key, Person p) {
-                            assertEquals(key.id, (Integer)p.salary);
+                ScanQuery<Key, Person> qry = new ScanQuery<>(new IgniteBiPredicate<Key, Person>() {
+                    @Override public boolean apply(Key key, Person p) {
+                        assertEquals(key.id, (Integer)p.salary);
 
-                            return key.id % 2 == 0;
-                        }
-                    }, null, false);
+                        return key.id % 2 == 0;
+                    }
+                });
 
                 for (int i = 0; i < 250; i++) {
-                    Collection<Map.Entry<Key, Person>> res = qry.execute().get();
+                    List<Cache.Entry<Key, Person>> res = cache.query(qry).getAll();
 
                     assertEquals(expCnt, res.size());
 
@@ -237,9 +246,9 @@ public abstract class GridCacheSwapScanQueryAbstractSelfTest extends GridCommonA
      * @throws Exception If failed.
      */
     public void testQueryPrimitives() throws Exception {
-        checkQueryPrimitives(((IgniteKernal)grid(0)).internalCache(ATOMIC_CACHE_NAME));
+        checkQueryPrimitives(grid(0).cache(ATOMIC_CACHE_NAME));
 
-        checkQueryPrimitives(((IgniteKernal)grid(0)).internalCache(TRANSACTIONAL_CACHE_NAME));
+        checkQueryPrimitives(grid(0).cache(TRANSACTIONAL_CACHE_NAME));
     }
 
     /**
@@ -247,27 +256,26 @@ public abstract class GridCacheSwapScanQueryAbstractSelfTest extends GridCommonA
      * @throws Exception If failed.
      */
     @SuppressWarnings("unchecked")
-    private void checkQueryPrimitives(GridCacheAdapter cache) throws Exception {
+    private void checkQueryPrimitives(IgniteCache cache) throws Exception {
         final int ENTRY_CNT = 500;
 
         for (int i = 0; i < ENTRY_CNT; i++)
-            cache.getAndPut(String.valueOf(i), (long) i);
+            cache.getAndPut(String.valueOf(i), (long)i);
 
         try {
-            CacheQuery<Map.Entry<String, Long>> qry = cache.context().queries().createScanQuery(
-                new IgniteBiPredicate<String, Long>() {
-                    @Override public boolean apply(String key, Long val) {
-                        assertEquals(key, String.valueOf(val));
+            ScanQuery<String, Long> qry = new ScanQuery<>(new IgniteBiPredicate<String, Long>() {
+                @Override public boolean apply(String key, Long val) {
+                    assertEquals(key, String.valueOf(val));
 
-                        return val % 2 == 0;
-                    }
-                }, null, false);
+                    return val % 2 == 0;
+                }
+            });
 
-            Collection<Map.Entry<String, Long>> res = qry.execute().get();
+            Collection<Cache.Entry<String, Long>> res = cache.query(qry).getAll();
 
             assertEquals(ENTRY_CNT / 2, res.size());
 
-            for (Map.Entry<String, Long> e : res) {
+            for (Cache.Entry<String, Long> e : res) {
                 String key = e.getKey();
                 Long val = e.getValue();
 
@@ -276,9 +284,9 @@ public abstract class GridCacheSwapScanQueryAbstractSelfTest extends GridCommonA
                 assertEquals(0, val % 2);
             }
 
-            qry = cache.context().queries().createScanQuery(null, null, false);
+            qry = new ScanQuery<>();
 
-            res = qry.execute().get();
+            res = cache.query(qry).getAll();
 
             assertEquals(ENTRY_CNT, res.size());
         }
@@ -292,9 +300,9 @@ public abstract class GridCacheSwapScanQueryAbstractSelfTest extends GridCommonA
      * @throws Exception If failed.
      */
     public void testQueryValueByteArray() throws Exception {
-        checkQueryValueByteArray(((IgniteKernal)grid(0)).internalCache(ATOMIC_CACHE_NAME));
+        checkQueryValueByteArray(grid(0).cache(ATOMIC_CACHE_NAME));
 
-        checkQueryValueByteArray(((IgniteKernal)grid(0)).internalCache(TRANSACTIONAL_CACHE_NAME));
+        checkQueryValueByteArray(grid(0).cache(TRANSACTIONAL_CACHE_NAME));
     }
 
     /**
@@ -302,27 +310,27 @@ public abstract class GridCacheSwapScanQueryAbstractSelfTest extends GridCommonA
      * @throws Exception If failed.
      */
     @SuppressWarnings("unchecked")
-    private void checkQueryValueByteArray(GridCacheAdapter cache) throws Exception {
+    private void checkQueryValueByteArray(IgniteCache cache) throws Exception {
         final int ENTRY_CNT = 100;
 
         for (int i = 0; i < ENTRY_CNT; i++)
             cache.getAndPut(i, new byte[i]);
 
         try {
-            CacheQuery<Map.Entry<Integer, byte[]>> qry = cache.context().queries().createScanQuery(
+            ScanQuery<Integer, byte[]> qry = new ScanQuery<>(
                 new IgniteBiPredicate<Integer, byte[]>() {
                     @Override public boolean apply(Integer key, byte[] val) {
                         assertEquals(key, (Integer)val.length);
 
                         return key % 2 == 0;
                     }
-                }, null, false);
+                });
 
-            Collection<Map.Entry<Integer, byte[]>> res = qry.execute().get();
+            Collection<Cache.Entry<Integer, byte[]>> res = cache.query(qry).getAll();
 
             assertEquals(ENTRY_CNT / 2, res.size());
 
-            for (Map.Entry<Integer, byte[]> e : res) {
+            for (Cache.Entry<Integer, byte[]> e : res) {
                 Integer key = e.getKey();
                 byte[] val = e.getValue();
 
@@ -331,9 +339,9 @@ public abstract class GridCacheSwapScanQueryAbstractSelfTest extends GridCommonA
                 assertEquals(0, key % 2);
             }
 
-            qry = cache.context().queries().createScanQuery(null, null, false);
+            qry = new ScanQuery<>();
 
-            res = qry.execute().get();
+            res = cache.query(qry).getAll();
 
             assertEquals(ENTRY_CNT, res.size());
         }
@@ -419,9 +427,9 @@ public abstract class GridCacheSwapScanQueryAbstractSelfTest extends GridCommonA
 
         /** {@inheritDoc} */
         @Override public int hashCode() {
-            int result = name != null ? name.hashCode() : 0;
+            int res = name != null ? name.hashCode() : 0;
 
-            return 31 * result + salary;
+            return 31 * res + salary;
         }
     }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/5f5220af/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/CacheScanPartitionQueryFallbackSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/CacheScanPartitionQueryFallbackSelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/CacheScanPartitionQueryFallbackSelfTest.java
index 1ef470a..02b213e 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/CacheScanPartitionQueryFallbackSelfTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/CacheScanPartitionQueryFallbackSelfTest.java
@@ -22,36 +22,29 @@ import java.util.HashMap;
 import java.util.HashSet;
 import java.util.Map;
 import java.util.Set;
-import java.util.TreeSet;
 import java.util.UUID;
 import java.util.concurrent.Callable;
-import java.util.concurrent.CountDownLatch;
 import java.util.concurrent.ThreadLocalRandom;
 import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.concurrent.atomic.AtomicInteger;
 import javax.cache.Cache;
 import org.apache.ignite.Ignite;
 import org.apache.ignite.IgniteCache;
-import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.IgniteException;
 import org.apache.ignite.cache.CacheAtomicityMode;
 import org.apache.ignite.cache.CacheMode;
+import org.apache.ignite.cache.CacheRebalanceMode;
 import org.apache.ignite.cache.affinity.rendezvous.RendezvousAffinityFunction;
 import org.apache.ignite.cache.query.QueryCursor;
 import org.apache.ignite.cache.query.ScanQuery;
 import org.apache.ignite.cluster.ClusterNode;
 import org.apache.ignite.configuration.CacheConfiguration;
 import org.apache.ignite.configuration.IgniteConfiguration;
-import org.apache.ignite.internal.IgniteEx;
 import org.apache.ignite.internal.IgniteInternalFuture;
-import org.apache.ignite.internal.cluster.ClusterGroupEmptyCheckedException;
 import org.apache.ignite.internal.managers.communication.GridIoMessage;
 import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
 import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtLocalPartition;
 import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtPartitionState;
-import org.apache.ignite.internal.processors.cache.query.CacheQuery;
-import org.apache.ignite.internal.processors.cache.query.CacheQueryFuture;
-import org.apache.ignite.internal.processors.cache.query.GridCacheQueryAdapter;
 import org.apache.ignite.internal.processors.cache.query.GridCacheQueryRequest;
 import org.apache.ignite.internal.util.typedef.F;
 import org.apache.ignite.internal.util.typedef.internal.U;
@@ -91,18 +84,15 @@ public class CacheScanPartitionQueryFallbackSelfTest extends GridCommonAbstractT
     /** Expected first node ID. */
     private static UUID expNodeId;
 
-    /** Expected fallback node ID. */
-    private static UUID expFallbackNodeId;
-
     /** Communication SPI factory. */
     private CommunicationSpiFactory commSpiFactory;
 
-    /** Latch. */
-    private static CountDownLatch latch;
-
     /** Test entries. */
     private Map<Integer, Map<Integer, Integer>> entries = new HashMap<>();
 
+    /** */
+    private boolean syncRebalance;
+
     /** {@inheritDoc} */
     @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
         IgniteConfiguration cfg = super.getConfiguration(gridName);
@@ -120,6 +110,10 @@ public class CacheScanPartitionQueryFallbackSelfTest extends GridCommonAbstractT
         ccfg.setCacheMode(cacheMode);
         ccfg.setAtomicityMode(CacheAtomicityMode.ATOMIC);
         ccfg.setBackups(backups);
+
+        if (syncRebalance)
+            ccfg.setRebalanceMode(CacheRebalanceMode.SYNC);
+
         ccfg.setNearConfiguration(null);
 
         cfg.setCacheConfiguration(ccfg);
@@ -144,7 +138,8 @@ public class CacheScanPartitionQueryFallbackSelfTest extends GridCommonAbstractT
 
             int part = anyLocalPartition(cache.context());
 
-            CacheQuery<Map.Entry<Integer, Integer>> qry = cache.context().queries().createScanQuery(null, part, false);
+            QueryCursor<Cache.Entry<Integer, Integer>> qry =
+                cache.query(new ScanQuery<Integer, Integer>().setPartition(part));
 
             doTestScanQuery(qry, part);
         }
@@ -174,7 +169,8 @@ public class CacheScanPartitionQueryFallbackSelfTest extends GridCommonAbstractT
 
             expNodeId = tup.get2();
 
-            CacheQuery<Map.Entry<Integer, Integer>> qry = cache.context().queries().createScanQuery(null, part, false);
+            QueryCursor<Cache.Entry<Integer, Integer>> qry =
+                cache.query(new ScanQuery<Integer, Integer>().setPartition(part));
 
             doTestScanQuery(qry, part);
         }
@@ -184,16 +180,22 @@ public class CacheScanPartitionQueryFallbackSelfTest extends GridCommonAbstractT
     }
 
     /**
-     * Scan should activate fallback mechanism when new nodes join topology and rebalancing happens in parallel with
-     * scan query.
-     *
      * @throws Exception In case of error.
      */
     public void testScanFallbackOnRebalancing() throws Exception {
+        scanFallbackOnRebalancing(false);
+    }
+
+    /**
+     * @param cur If {@code true} tests query cursor.
+     * @throws Exception In case of error.
+     */
+    private void scanFallbackOnRebalancing(final boolean cur) throws Exception {
         cacheMode = CacheMode.PARTITIONED;
         clientMode = false;
-        backups = 1;
+        backups = 2;
         commSpiFactory = new TestFallbackOnRebalancingCommunicationSpiFactory();
+        syncRebalance = true;
 
         try {
             Ignite ignite = startGrids(GRID_CNT);
@@ -214,6 +216,8 @@ public class CacheScanPartitionQueryFallbackSelfTest extends GridCommonAbstractT
 
                             Thread.sleep(3000);
 
+                            info("Will stop grid: " + getTestGridName(id));
+
                             stopGrid(id);
 
                             if (done.get())
@@ -224,7 +228,7 @@ public class CacheScanPartitionQueryFallbackSelfTest extends GridCommonAbstractT
 
                         return null;
                     }
-                }, GRID_CNT);
+                }, 2);
 
             final AtomicInteger nodeIdx = new AtomicInteger();
 
@@ -233,18 +237,24 @@ public class CacheScanPartitionQueryFallbackSelfTest extends GridCommonAbstractT
                     @Override public Object call() throws Exception {
                         int nodeId = nodeIdx.getAndIncrement();
 
-                        IgniteCacheProxy<Integer, Integer> cache = (IgniteCacheProxy<Integer, Integer>)
-                            grid(nodeId).<Integer, Integer>cache(null);
+                        IgniteCache<Integer, Integer> cache = grid(nodeId).cache(null);
+
+                        int cntr = 0;
 
                         while (!done.get()) {
-                            IgniteBiTuple<Integer, UUID> tup = remotePartition(cache.context());
+                            int part = ThreadLocalRandom.current().nextInt(ignite(nodeId).affinity(null).partitions());
 
-                            int part = tup.get1();
+                            if (cntr++ % 100 == 0)
+                                info("Running query [node=" + nodeId + ", part=" + part + ']');
 
-                            CacheQuery<Map.Entry<Integer, Integer>> qry = cache.context().queries().createScanQuery(
-                                null, part, false);
+                            try (QueryCursor<Cache.Entry<Integer, Integer>> cur0 =
+                                     cache.query(new ScanQuery<Integer, Integer>(part).setPageSize(5))) {
 
-                            doTestScanQuery(qry, part);
+                                if (cur)
+                                    doTestScanQueryCursor(cur0, part);
+                                else
+                                    doTestScanQuery(cur0, part);
+                            }
                         }
 
                         return null;
@@ -269,9 +279,7 @@ public class CacheScanPartitionQueryFallbackSelfTest extends GridCommonAbstractT
      *
      * @throws Exception In case of error.
      */
-    public void testScanFallbackOnRebalancingCursor() throws Exception {
-        fail("https://issues.apache.org/jira/browse/IGNITE-1239");
-
+    public void testScanFallbackOnRebalancingCursor1() throws Exception {
         cacheMode = CacheMode.PARTITIONED;
         clientMode = false;
         backups = 1;
@@ -308,15 +316,19 @@ public class CacheScanPartitionQueryFallbackSelfTest extends GridCommonAbstractT
 
                         IgniteCache<Integer, Integer> cache = grid(nodeId).cache(null);
 
+                        int cntr = 0;
+
                         while (!done.get()) {
                             int part = ThreadLocalRandom.current().nextInt(ignite(nodeId).affinity(null).partitions());
 
-                            QueryCursor<Cache.Entry<Integer, Integer>> cur =
-                                cache.query(new ScanQuery<Integer, Integer>(part));
+                            if (cntr++ % 100 == 0)
+                                info("Running query [node=" + nodeId + ", part=" + part + ']');
 
-                            U.debug(log, "Running query [node=" + nodeId + ", part=" + part + ']');
+                            try (QueryCursor<Cache.Entry<Integer, Integer>> cur =
+                                     cache.query(new ScanQuery<Integer, Integer>(part).setPageSize(5))) {
 
-                            doTestScanQueryCursor(cur, part);
+                                doTestScanQueryCursor(cur, part);
+                            }
                         }
 
                         return null;
@@ -332,95 +344,15 @@ public class CacheScanPartitionQueryFallbackSelfTest extends GridCommonAbstractT
     }
 
     /**
-     * Scan should try first remote node and fallbacks to second remote node.
-     *
      * @throws Exception If failed.
      */
-    public void testScanFallback() throws Exception {
-        cacheMode = CacheMode.PARTITIONED;
-        backups = 1;
-        commSpiFactory = new TestFallbackCommunicationSpiFactory();
-
-        final Set<Integer> candidates = new TreeSet<>();
-
-        final AtomicBoolean test = new AtomicBoolean(false);
-
-        for(int j = 0; j < 2; j++) {
-            clientMode = true;
-
-            latch = new CountDownLatch(1);
-
-            try {
-                final Ignite ignite0 = startGrid(0);
-
-                clientMode = false;
-
-                final IgniteEx ignite1 = startGrid(1);
-                final IgniteEx ignite2 = startGrid(2);
-                startGrid(3);
-
-                if (test.get()) {
-                    expNodeId = ignite1.localNode().id();
-                    expFallbackNodeId = ignite2.localNode().id();
-                }
-
-                final IgniteCacheProxy<Integer, Integer> cache = fillCache(ignite0);
-
-                if (!test.get()) {
-                    candidates.addAll(localPartitions(ignite1));
-
-                    candidates.retainAll(localPartitions(ignite2));
-                }
-
-                Runnable run = new Runnable() {
-                    @Override public void run() {
-                        try {
-                            startGrid(4);
-                            startGrid(5);
-
-                            awaitPartitionMapExchange();
-
-                            if (!test.get()) {
-                                candidates.removeAll(localPartitions(ignite1));
-
-                                F.retain(candidates, false, localPartitions(ignite2));
-                            }
-
-                            latch.countDown();
-                        }
-                        catch (Exception e) {
-                            e.printStackTrace();
-                        }
-
-                    }
-                };
-
-                Integer part = null;
-                CacheQuery<Map.Entry<Integer, Integer>> qry = null;
-
-                if (test.get()) {
-                    part = F.first(candidates);
-
-                    qry = cache.context().queries().createScanQuery(null, part, false);
-                }
-
-                new Thread(run).start();
-
-                if (test.get())
-                    doTestScanQuery(qry, part);
-                else
-                    latch.await();
-            }
-            finally {
-                test.set(true);
-
-                stopAllGrids();
-            }
-        }
+    public void testScanFallbackOnRebalancingCursor2() throws Exception {
+        scanFallbackOnRebalancing(true);
     }
 
     /**
      * @param ignite Ignite.
+     * @return Cache.
      */
     protected IgniteCacheProxy<Integer, Integer> fillCache(Ignite ignite) {
         IgniteCacheProxy<Integer, Integer> cache =
@@ -444,16 +376,14 @@ public class CacheScanPartitionQueryFallbackSelfTest extends GridCommonAbstractT
 
     /**
      * @param qry Query.
+     * @param part Partition.
      */
-    protected void doTestScanQuery(CacheQuery<Map.Entry<Integer, Integer>> qry, int part)
-        throws IgniteCheckedException {
-        CacheQueryFuture<Map.Entry<Integer, Integer>> fut = qry.execute();
-
-        Collection<Map.Entry<Integer, Integer>> qryEntries = fut.get();
+    protected void doTestScanQuery(QueryCursor<Cache.Entry<Integer, Integer>> qry, int part) {
+        Collection<Cache.Entry<Integer, Integer>> qryEntries = qry.getAll();
 
         Map<Integer, Integer> map = entries.get(part);
 
-        for (Map.Entry<Integer, Integer> e : qryEntries)
+        for (Cache.Entry<Integer, Integer> e : qryEntries)
             assertEquals(map.get(e.getKey()), e.getValue());
 
         assertEquals("Invalid number of entries for partition: " + part, map.size(), qryEntries.size());
@@ -464,7 +394,7 @@ public class CacheScanPartitionQueryFallbackSelfTest extends GridCommonAbstractT
      * @param part Partition number.
      */
     protected void doTestScanQueryCursor(
-        QueryCursor<Cache.Entry<Integer, Integer>> cur, int part) throws IgniteCheckedException {
+        QueryCursor<Cache.Entry<Integer, Integer>> cur, int part) {
 
         Map<Integer, Integer> map = entries.get(part);
 
@@ -483,6 +413,7 @@ public class CacheScanPartitionQueryFallbackSelfTest extends GridCommonAbstractT
 
     /**
      * @param cctx Cctx.
+     * @return Local partition.
      */
     private static int anyLocalPartition(GridCacheContext<?, ?> cctx) {
         return F.first(cctx.topology().localPartitions()).id();
@@ -490,6 +421,7 @@ public class CacheScanPartitionQueryFallbackSelfTest extends GridCommonAbstractT
 
     /**
      * @param cctx Cctx.
+     * @return Remote partition.
      */
     private IgniteBiTuple<Integer, UUID> remotePartition(final GridCacheContext cctx) {
         ClusterNode node = F.first(cctx.kernalContext().grid().cluster().forRemotes().nodes());
@@ -505,6 +437,7 @@ public class CacheScanPartitionQueryFallbackSelfTest extends GridCommonAbstractT
 
     /**
      * @param ignite Ignite.
+     * @return Local partitions.
      */
     private Set<Integer> localPartitions(Ignite ignite) {
         GridCacheContext cctx = ((IgniteCacheProxy)ignite.cache(null)).context();
@@ -528,7 +461,7 @@ public class CacheScanPartitionQueryFallbackSelfTest extends GridCommonAbstractT
      */
     private interface CommunicationSpiFactory {
         /**
-         * Creates communication SPI instance.
+         * @return Communication SPI instance.
          */
         TcpCommunicationSpi create();
     }
@@ -541,13 +474,13 @@ public class CacheScanPartitionQueryFallbackSelfTest extends GridCommonAbstractT
         @Override public TcpCommunicationSpi create() {
             return new TcpCommunicationSpi() {
                 @Override public void sendMessage(ClusterNode node, Message msg,
-                    IgniteInClosure<IgniteException> ackClosure) throws IgniteSpiException {
+                    IgniteInClosure<IgniteException> ackC) throws IgniteSpiException {
                     Object origMsg = ((GridIoMessage)msg).message();
 
                     if (origMsg instanceof GridCacheQueryRequest)
                         fail(); //should use local node
 
-                    super.sendMessage(node, msg, ackClosure);
+                    super.sendMessage(node, msg, ackC);
                 }
             };
         }
@@ -561,44 +494,13 @@ public class CacheScanPartitionQueryFallbackSelfTest extends GridCommonAbstractT
         @Override public TcpCommunicationSpi create() {
             return new TcpCommunicationSpi() {
                 @Override public void sendMessage(ClusterNode node, Message msg,
-                    IgniteInClosure<IgniteException> ackClosure) throws IgniteSpiException {
+                    IgniteInClosure<IgniteException> ackC) throws IgniteSpiException {
                     Object origMsg = ((GridIoMessage)msg).message();
 
                     if (origMsg instanceof GridCacheQueryRequest)
                         assertEquals(expNodeId, node.id());
 
-                    super.sendMessage(node, msg, ackClosure);
-                }
-            };
-        }
-    }
-
-    /**
-     *
-     */
-    private static class TestFallbackCommunicationSpiFactory implements CommunicationSpiFactory {
-        /** {@inheritDoc} */
-        @Override public TcpCommunicationSpi create() {
-            return new TcpCommunicationSpi() {
-                @Override public void sendMessage(ClusterNode node, Message msg,
-                    IgniteInClosure<IgniteException> ackClosure) throws IgniteSpiException {
-                    Object origMsg = ((GridIoMessage)msg).message();
-
-                    if (origMsg instanceof GridCacheQueryRequest) {
-                        if (latch.getCount() > 0)
-                            assertEquals(expNodeId, node.id());
-                        else
-                            assertEquals(expFallbackNodeId, node.id());
-
-                        try {
-                            latch.await();
-                        }
-                        catch (InterruptedException e) {
-                            throw new IgniteSpiException(e);
-                        }
-                    }
-
-                    super.sendMessage(node, msg, ackClosure);
+                    super.sendMessage(node, msg, ackC);
                 }
             };
         }


[50/55] [abbrv] ignite git commit: import optimize

Posted by ag...@apache.org.
import optimize


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

Branch: refs/heads/ignite-1171
Commit: d928ef4ae0ae7fadbb05dacfc97ca6402657894d
Parents: f8577ca
Author: Yakov Zhdanov <yz...@gridgain.com>
Authored: Wed Sep 16 02:28:35 2015 +0300
Committer: Yakov Zhdanov <yz...@gridgain.com>
Committed: Wed Sep 16 02:28:35 2015 +0300

----------------------------------------------------------------------
 .../distributed/dht/GridDhtLocalPartition.java  | 26 ++++++++++----------
 1 file changed, 13 insertions(+), 13 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/d928ef4a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtLocalPartition.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtLocalPartition.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtLocalPartition.java
index 956f2bf..2deabfe 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtLocalPartition.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtLocalPartition.java
@@ -17,18 +17,6 @@
 
 package org.apache.ignite.internal.processors.cache.distributed.dht;
 
-import java.util.Collection;
-import java.util.HashMap;
-import java.util.Iterator;
-import java.util.Map;
-import java.util.NoSuchElementException;
-import java.util.Set;
-import java.util.concurrent.ConcurrentMap;
-import java.util.concurrent.CopyOnWriteArrayList;
-import java.util.concurrent.atomic.AtomicReference;
-import java.util.concurrent.atomic.AtomicStampedReference;
-import java.util.concurrent.locks.ReentrantLock;
-import javax.cache.CacheException;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.IgniteLogger;
 import org.apache.ignite.internal.IgniteInternalFuture;
@@ -58,6 +46,18 @@ import org.jetbrains.annotations.NotNull;
 import org.jsr166.ConcurrentHashMap8;
 import org.jsr166.LongAdder8;
 
+import javax.cache.CacheException;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.CopyOnWriteArrayList;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.concurrent.atomic.AtomicStampedReference;
+import java.util.concurrent.locks.ReentrantLock;
+
 import static org.apache.ignite.IgniteSystemProperties.IGNITE_ATOMIC_CACHE_DELETE_HISTORY_SIZE;
 import static org.apache.ignite.events.EventType.EVT_CACHE_REBALANCE_OBJECT_UNLOADED;
 import static org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtPartitionState.EVICTED;
@@ -787,4 +787,4 @@ public class GridDhtLocalPartition implements Comparable<GridDhtLocalPartition>,
             "createTime", U.format(createTime),
             "mapPubSize", mapPubSize);
     }
-}
\ No newline at end of file
+}


[54/55] [abbrv] ignite git commit: Merge branch 'ignite-1.4' of https://git-wip-us.apache.org/repos/asf/ignite into ignite-1171

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


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

Branch: refs/heads/ignite-1171
Commit: c01f936cf938f2603228e51902330c13a79a9006
Parents: 91115c8 6187241
Author: Alexey Goncharuk <al...@gmail.com>
Authored: Wed Sep 16 19:00:44 2015 -0700
Committer: Alexey Goncharuk <al...@gmail.com>
Committed: Wed Sep 16 19:00:44 2015 -0700

----------------------------------------------------------------------
 DEVNOTES.txt                                    |   22 +-
 DISCLAIMER.txt                                  |   15 -
 LICENSE                                         |    2 +-
 NOTICE                                          |    2 +-
 RELEASE_NOTES.txt                               |   13 +-
 assembly/LICENSE_FABRIC                         |    2 +-
 assembly/LICENSE_HADOOP                         |    2 +-
 assembly/NOTICE_FABRIC                          |    2 +-
 assembly/NOTICE_HADOOP                          |    2 +-
 assembly/dependencies-hadoop.xml                |    1 +
 assembly/release-base.xml                       |    5 -
 dev-tools/slurp.sh                              |    2 +-
 dev-tools/src/main/groovy/jiraslurp.groovy      |    2 +-
 examples/config/example-cache.xml               |    2 -
 examples/config/filesystem/README.txt           |    2 +-
 examples/config/filesystem/example-igfs.xml     |    7 -
 examples/pom.xml                                |    2 +-
 .../org/apache/ignite/schema/CacheConfig.java   |    9 +-
 .../java/org/apache/ignite/schema/Demo.java     |   22 +-
 .../java/org/apache/ignite/schema/DemoNode.java |    5 +-
 .../java/org/apache/ignite/schema/Person.java   |    1 -
 .../org/apache/ignite/schema/PersonKey.java     |    1 -
 .../ignite/examples/ExampleNodeStartup.java     |    5 +-
 .../apache/ignite/examples/ExamplesUtils.java   |   11 +-
 .../computegrid/ComputeAsyncExample.java        |   16 +-
 .../computegrid/ComputeBroadcastExample.java    |   16 +-
 .../computegrid/ComputeCallableExample.java     |   14 +-
 .../computegrid/ComputeClosureExample.java      |   14 +-
 .../ComputeContinuousMapperExample.java         |   32 +-
 .../ComputeFibonacciContinuationExample.java    |   32 +-
 .../computegrid/ComputeReducerExample.java      |   16 +-
 .../computegrid/ComputeRunnableExample.java     |   11 +-
 .../computegrid/ComputeTaskMapExample.java      |   23 +-
 .../computegrid/ComputeTaskSplitExample.java    |   20 +-
 .../cluster/ClusterGroupExample.java            |   16 +-
 .../computegrid/cluster/package-info.java       |    2 +-
 .../failover/ComputeFailoverExample.java        |   23 +-
 .../failover/ComputeFailoverNodeStartup.java    |   17 +-
 .../computegrid/failover/package-info.java      |    2 +-
 .../examples/computegrid/montecarlo/Credit.java |    4 +-
 .../montecarlo/CreditRiskExample.java           |   16 +-
 .../montecarlo/CreditRiskManager.java           |    5 +-
 .../computegrid/montecarlo/package-info.java    |    2 +-
 .../examples/computegrid/package-info.java      |    2 +-
 .../examples/datagrid/CacheAffinityExample.java |   20 +-
 .../examples/datagrid/CacheApiExample.java      |   15 +-
 .../examples/datagrid/CacheAsyncApiExample.java |   16 +-
 .../datagrid/CacheContinuousQueryExample.java   |   22 +-
 .../datagrid/CacheDataStreamerExample.java      |   11 +-
 .../examples/datagrid/CacheEventsExample.java   |   21 +-
 .../examples/datagrid/CachePutGetExample.java   |   13 +-
 .../examples/datagrid/CacheQueryExample.java    |   35 +-
 .../datagrid/CacheTransactionExample.java       |   25 +-
 .../hibernate/HibernateL2CacheExample.java      |   35 +-
 .../examples/datagrid/hibernate/Post.java       |   10 +-
 .../examples/datagrid/hibernate/User.java       |   13 +-
 .../datagrid/hibernate/package-info.java        |    2 +-
 .../ignite/examples/datagrid/package-info.java  |    2 +-
 .../starschema/CacheStarSchemaExample.java      |   26 +-
 .../datagrid/starschema/DimProduct.java         |    6 +-
 .../examples/datagrid/starschema/DimStore.java  |    6 +-
 .../datagrid/starschema/FactPurchase.java       |    6 +-
 .../datagrid/starschema/package-info.java       |    2 +-
 .../ignite/examples/datagrid/store/Person.java  |    3 +-
 .../store/auto/CacheAutoStoreExample.java       |   18 +-
 .../auto/CacheAutoStoreLoadDataExample.java     |   16 +-
 .../datagrid/store/auto/CacheConfig.java        |   27 +-
 .../datagrid/store/auto/DbH2ServerStartup.java  |   15 +-
 .../datagrid/store/auto/package-info.java       |    2 +-
 .../store/dummy/CacheDummyPersonStore.java      |   24 +-
 .../store/dummy/CacheDummyStoreExample.java     |   25 +-
 .../datagrid/store/dummy/package-info.java      |    2 +-
 .../hibernate/CacheHibernatePersonStore.java    |   22 +-
 .../hibernate/CacheHibernateStoreExample.java   |   30 +-
 .../datagrid/store/hibernate/package-info.java  |    2 +-
 .../store/jdbc/CacheJdbcPersonStore.java        |   29 +-
 .../store/jdbc/CacheJdbcStoreExample.java       |   30 +-
 .../datagrid/store/jdbc/package-info.java       |    2 +-
 .../examples/datagrid/store/package-info.java   |    2 +-
 .../store/spring/CacheSpringPersonStore.java    |   31 +-
 .../store/spring/CacheSpringStoreExample.java   |   30 +-
 .../datagrid/store/spring/package-info.java     |    2 +-
 .../datastructures/IgniteAtomicLongExample.java |   13 +-
 .../IgniteAtomicReferenceExample.java           |   13 +-
 .../IgniteAtomicSequenceExample.java            |   10 +-
 .../IgniteAtomicStampedExample.java             |   13 +-
 .../IgniteCountDownLatchExample.java            |   13 +-
 .../IgniteExecutorServiceExample.java           |   13 +-
 .../datastructures/IgniteQueueExample.java      |   18 +-
 .../datastructures/IgniteSetExample.java        |   22 +-
 .../examples/datastructures/package-info.java   |    2 +-
 .../ignite/examples/events/EventsExample.java   |   25 +-
 .../ignite/examples/events/package-info.java    |    2 +-
 .../ignite/examples/igfs/IgfsExample.java       |   20 +-
 .../examples/igfs/IgfsMapReduceExample.java     |   38 +-
 .../ignite/examples/igfs/IgfsNodeStartup.java   |    5 +-
 .../ignite/examples/igfs/package-info.java      |    2 +-
 .../examples/messaging/MessagingExample.java    |   21 +-
 .../messaging/MessagingPingPongExample.java     |   20 +-
 .../MessagingPingPongListenActorExample.java    |   20 +-
 .../ignite/examples/messaging/package-info.java |    2 +-
 .../client/memcache/MemcacheRestExample.java    |   17 +-
 .../MemcacheRestExampleNodeStartup.java         |   28 +-
 .../misc/client/memcache/package-info.java      |    2 +-
 .../examples/misc/client/package-info.java      |    2 +-
 .../misc/deployment/DeploymentExample.java      |   25 +-
 .../examples/misc/deployment/package-info.java  |    2 +-
 .../misc/lifecycle/LifecycleExample.java        |   16 +-
 .../examples/misc/lifecycle/package-info.java   |    2 +-
 .../ignite/examples/misc/package-info.java      |    2 +-
 .../misc/schedule/ComputeScheduleExample.java   |   15 +-
 .../examples/misc/schedule/package-info.java    |    2 +-
 .../misc/springbean/SpringBeanExample.java      |   13 +-
 .../examples/misc/springbean/package-info.java  |    2 +-
 .../apache/ignite/examples/package-info.java    |    2 +-
 .../examples/servicegrid/ServicesExample.java   |   29 +-
 .../examples/servicegrid/SimpleMapService.java  |    2 +-
 .../servicegrid/SimpleMapServiceImpl.java       |   12 +-
 .../examples/servicegrid/package-info.java      |    2 +-
 .../streaming/StreamTransformerExample.java     |   22 +-
 .../streaming/StreamVisitorExample.java         |   25 +-
 .../ignite/examples/streaming/package-info.java |    2 +-
 .../streaming/wordcount/CacheConfig.java        |   14 +-
 .../streaming/wordcount/QueryWords.java         |   16 +-
 .../streaming/wordcount/StreamWords.java        |   17 +-
 .../streaming/wordcount/package-info.java       |    2 +-
 .../socket/WordsSocketStreamerClient.java       |   19 +-
 .../socket/WordsSocketStreamerServer.java       |   30 +-
 .../wordcount/socket/package-info.java          |    2 +-
 .../java8/cluster/ClusterGroupExample.java      |   12 +-
 .../examples/java8/cluster/package-info.java    |    2 +-
 .../java8/computegrid/ComputeAsyncExample.java  |   16 +-
 .../computegrid/ComputeBroadcastExample.java    |   11 +-
 .../computegrid/ComputeCallableExample.java     |   14 +-
 .../computegrid/ComputeClosureExample.java      |   12 +-
 .../computegrid/ComputeRunnableExample.java     |   13 +-
 .../java8/computegrid/package-info.java         |    2 +-
 .../java8/datagrid/CacheAffinityExample.java    |   24 +-
 .../java8/datagrid/CacheApiExample.java         |   16 +-
 .../java8/datagrid/CacheAsyncApiExample.java    |   19 +-
 .../examples/java8/datagrid/package-info.java   |    2 +-
 .../IgniteExecutorServiceExample.java           |   13 +-
 .../java8/datastructures/package-info.java      |    2 +-
 .../examples/java8/events/EventsExample.java    |   25 +-
 .../examples/java8/events/package-info.java     |    2 +-
 .../java8/messaging/MessagingExample.java       |   13 +-
 .../messaging/MessagingPingPongExample.java     |   12 +-
 .../examples/java8/messaging/package-info.java  |    2 +-
 .../misc/schedule/ComputeScheduleExample.java   |   10 +-
 .../java8/misc/schedule/package-info.java       |    2 +-
 .../ignite/examples/java8/package-info.java     |    2 +-
 .../streaming/StreamTransformerExample.java     |   20 +-
 .../java8/streaming/StreamVisitorExample.java   |   24 +-
 .../examples/java8/streaming/package-info.java  |    2 +-
 .../BasicExamplesMultiNodeSelfTest.java         |    2 +-
 .../ignite/examples/BasicExamplesSelfTest.java  |   14 +-
 .../CacheExamplesMultiNodeSelfTest.java         |    2 +-
 .../ignite/examples/CacheExamplesSelfTest.java  |   24 +-
 .../examples/CheckpointExamplesSelfTest.java    |    7 +-
 .../examples/ClusterGroupExampleSelfTest.java   |    6 +-
 .../ContinuationExamplesMultiNodeSelfTest.java  |    2 +-
 .../examples/ContinuationExamplesSelfTest.java  |    6 +-
 ...ntinuousMapperExamplesMultiNodeSelfTest.java |    2 +-
 .../ContinuousMapperExamplesSelfTest.java       |    6 +-
 .../DeploymentExamplesMultiNodeSelfTest.java    |    2 +-
 .../examples/DeploymentExamplesSelfTest.java    |    6 +-
 .../EventsExamplesMultiNodeSelfTest.java        |    2 +-
 .../ignite/examples/EventsExamplesSelfTest.java |    6 +-
 ...ibernateL2CacheExampleMultiNodeSelfTest.java |    4 +-
 .../HibernateL2CacheExampleSelfTest.java        |    6 +-
 .../ignite/examples/IgfsExamplesSelfTest.java   |    8 +-
 .../examples/LifecycleExamplesSelfTest.java     |    6 +-
 .../MemcacheRestExamplesMultiNodeSelfTest.java  |    4 +-
 .../examples/MemcacheRestExamplesSelfTest.java  |    7 +-
 .../examples/MessagingExamplesSelfTest.java     |    7 +-
 .../MonteCarloExamplesMultiNodeSelfTest.java    |    2 +-
 .../examples/MonteCarloExamplesSelfTest.java    |    6 +-
 .../examples/SpringBeanExamplesSelfTest.java    |    6 +-
 .../examples/TaskExamplesMultiNodeSelfTest.java |    2 +-
 .../ignite/examples/TaskExamplesSelfTest.java   |    7 +-
 .../testsuites/IgniteExamplesSelfTestSuite.java |   35 +-
 .../BasicExamplesMultiNodeSelfTest.java         |    2 +-
 .../java8/examples/BasicExamplesSelfTest.java   |    9 +-
 .../CacheExamplesMultiNodeSelfTest.java         |    2 +-
 .../java8/examples/CacheExamplesSelfTest.java   |    8 +-
 .../examples/CheckpointExamplesSelfTest.java    |    7 +-
 .../examples/ClusterGroupExampleSelfTest.java   |    2 +-
 .../ContinuationExamplesMultiNodeSelfTest.java  |    2 +-
 .../examples/ContinuationExamplesSelfTest.java  |    7 +-
 ...ntinuousMapperExamplesMultiNodeSelfTest.java |    2 +-
 .../ContinuousMapperExamplesSelfTest.java       |    6 +-
 .../DeploymentExamplesMultiNodeSelfTest.java    |    2 +-
 .../examples/DeploymentExamplesSelfTest.java    |    5 +-
 .../EventsExamplesMultiNodeSelfTest.java        |    2 +-
 .../java8/examples/EventsExamplesSelfTest.java  |    4 +-
 ...ibernateL2CacheExampleMultiNodeSelfTest.java |    2 +-
 .../HibernateL2CacheExampleSelfTest.java        |    7 +-
 .../java8/examples/IgfsExamplesSelfTest.java    |    5 +-
 .../examples/LifecycleExamplesSelfTest.java     |    7 +-
 .../MemcacheRestExamplesMultiNodeSelfTest.java  |    2 +-
 .../examples/MemcacheRestExamplesSelfTest.java  |    3 +-
 .../examples/MessagingExamplesSelfTest.java     |    8 +-
 .../MonteCarloExamplesMultiNodeSelfTest.java    |    2 +-
 .../examples/MonteCarloExamplesSelfTest.java    |    7 +-
 .../examples/SpringBeanExamplesSelfTest.java    |    7 +-
 .../examples/TaskExamplesMultiNodeSelfTest.java |    2 +-
 .../java8/examples/TaskExamplesSelfTest.java    |    3 +-
 .../IgniteExamplesJ8SelfTestSuite.java          |   14 +-
 idea/ignite_codeStyle.xml                       |   11 +-
 modules/aop/pom.xml                             |    3 +-
 .../aop/aspectj/GridifyAspectJAspect.java       |   32 +-
 .../aspectj/GridifySetToSetAspectJAspect.java   |   34 +-
 .../aspectj/GridifySetToValueAspectJAspect.java |   34 +-
 .../aop/spring/GridifySetToSetSpringAspect.java |   30 +-
 .../spring/GridifySetToValueSpringAspect.java   |   30 +-
 .../gridify/aop/spring/GridifySpringAspect.java |   28 +-
 .../aop/spring/GridifySpringEnhancer.java       |    6 +-
 .../aop/spring/GridifySpringPointcut.java       |   13 +-
 .../apache/ignite/gridify/AbstractAopTest.java  |   35 +-
 .../apache/ignite/gridify/BasicAopSelfTest.java |   21 +-
 .../ignite/gridify/GridifySetToSetTarget.java   |   17 +-
 .../gridify/GridifySetToSetTargetInterface.java |   11 +-
 .../ignite/gridify/GridifySetToValueTarget.java |   14 +-
 .../GridifySetToValueTargetInterface.java       |   10 +-
 .../GridifySetToXXXNonSpringAopSelfTest.java    |   13 +-
 .../GridifySetToXXXSpringAopSelfTest.java       |   15 +-
 .../ignite/gridify/NonSpringAopSelfTest.java    |    4 +-
 .../ignite/gridify/SpringAopSelfTest.java       |    6 +-
 .../apache/ignite/gridify/TestAopTarget.java    |   10 +-
 .../ignite/gridify/TestAopTargetInterface.java  |    7 +-
 .../ignite/gridify/TestGridifyException.java    |    2 +-
 .../apache/ignite/gridify/TestGridifyJob.java   |   11 +-
 .../apache/ignite/gridify/TestGridifyTask.java  |   14 +-
 .../gridify/hierarchy/GridifyHierarchyTest.java |    4 +-
 .../ignite/gridify/hierarchy/SuperTarget.java   |    4 +-
 .../apache/ignite/gridify/hierarchy/Target.java |    2 +-
 .../optimized/OptimizedMarshallerAopTest.java   |   20 +-
 .../apache/ignite/p2p/P2PGridifySelfTest.java   |   22 +-
 .../testsuites/IgniteAopSelfTestSuite.java      |   16 +-
 .../singlesplit/SingleSplitTestJobTarget.java   |    8 +-
 .../direct/singlesplit/SingleSplitTestTask.java |   22 +-
 .../singlesplit/SingleSplitsLoadTest.java       |   29 +-
 .../gridify/GridifyLoadTestJobTarget.java       |    4 +-
 .../loadtests/gridify/GridifyLoadTestTask.java  |   18 +-
 .../gridify/GridifySingleSplitLoadTest.java     |   31 +-
 .../org/test/gridify/ExternalAopTarget.java     |   10 +-
 .../test/gridify/ExternalGridifyException.java  |    2 +-
 .../org/test/gridify/ExternalGridifyTask.java   |   20 +-
 .../gridify/ExternalNonSpringAopSelfTest.java   |   17 +-
 modules/apache-license-gen/pom.xml              |   13 +-
 modules/aws/pom.xml                             |    3 +-
 .../spi/checkpoint/s3/S3CheckpointData.java     |    9 +-
 .../spi/checkpoint/s3/S3CheckpointSpi.java      |   60 +-
 .../spi/checkpoint/s3/S3CheckpointSpiMBean.java |    6 +-
 .../ignite/spi/checkpoint/s3/S3TimeData.java    |    4 +-
 .../tcp/ipfinder/s3/TcpDiscoveryS3IpFinder.java |   46 +-
 .../s3/S3CheckpointManagerSelfTest.java         |   11 +-
 .../s3/S3CheckpointSpiConfigSelfTest.java       |    5 +-
 .../checkpoint/s3/S3CheckpointSpiSelfTest.java  |   32 +-
 .../s3/S3CheckpointSpiStartStopSelfTest.java    |   11 +-
 .../s3/S3SessionCheckpointSelfTest.java         |   12 +-
 .../s3/TcpDiscoveryS3IpFinderSelfTest.java      |   16 +-
 .../ignite/testsuites/IgniteS3TestSuite.java    |   12 +-
 modules/clients/pom.xml                         |   10 +-
 modules/clients/src/test/config/jdbc-config.xml |   55 +
 .../internal/TaskEventSubjectIdSelfTest.java    |   53 +-
 .../ClientAbstractMultiThreadedSelfTest.java    |   62 +-
 .../client/ClientDefaultCacheSelfTest.java      |   35 +-
 .../internal/client/ClientGetAffinityTask.java  |   19 +-
 .../ignite/internal/client/ClientHttpTask.java  |   19 +-
 .../internal/client/ClientNodeStartup.java      |    9 +-
 .../client/ClientReconnectionSelfTest.java      |   16 +-
 .../internal/client/ClientSslNodeStartup.java   |    9 +-
 .../internal/client/ClientStartNodeTask.java    |   38 +-
 .../internal/client/ClientStopNodeTask.java     |   26 +-
 .../internal/client/ClientStringLengthTask.java |   16 +-
 .../client/ClientTcpMultiThreadedSelfTest.java  |    4 +-
 .../ClientTcpSslAuthenticationSelfTest.java     |   30 +-
 .../ClientTcpSslMultiThreadedSelfTest.java      |    6 +-
 .../ignite/internal/client/ClientTcpTask.java   |   16 +-
 ...skExecutionAfterTopologyRestartSelfTest.java |   10 +-
 .../internal/client/ClientTestRestServer.java   |   43 +-
 .../ignite/internal/client/HashMapStore.java    |   12 +-
 .../ignite/internal/client/SleepTestTask.java   |   16 +-
 .../client/TaskSingleJobSplitAdapter.java       |   14 +-
 .../client/impl/ClientCacheFlagsCodecTest.java  |   19 +-
 .../client/impl/ClientComputeImplSelfTest.java  |   16 +-
 .../client/impl/ClientDataImplSelfTest.java     |   11 +-
 .../impl/ClientFutureAdapterSelfTest.java       |   14 +-
 .../ClientPropertiesConfigurationSelfTest.java  |   41 +-
 .../ClientAbstractConnectivitySelfTest.java     |   35 +-
 .../ClientAbstractMultiNodeSelfTest.java        |  100 +-
 .../integration/ClientAbstractSelfTest.java     |   86 +-
 .../integration/ClientPreferDirectSelfTest.java |   43 +-
 .../ClientTcpConnectivitySelfTest.java          |   15 +-
 .../ClientTcpDirectMultiNodeSelfTest.java       |   11 +-
 .../integration/ClientTcpDirectSelfTest.java    |   11 +-
 .../integration/ClientTcpMultiNodeSelfTest.java |    4 +-
 .../client/integration/ClientTcpSelfTest.java   |    6 +-
 .../ClientTcpSslDirectMultiNodeSelfTest.java    |   15 +-
 .../integration/ClientTcpSslDirectSelfTest.java |   13 +-
 .../ClientTcpSslMultiNodeSelfTest.java          |    8 +-
 .../integration/ClientTcpSslSelfTest.java       |    8 +-
 .../ClientTcpUnreachableMultiNodeSelfTest.java  |   22 +-
 .../client/router/ClientFailedInitSelfTest.java |   50 +-
 .../client/router/RouterFactorySelfTest.java    |   23 +-
 .../router/TcpRouterAbstractSelfTest.java       |   22 +-
 .../router/TcpRouterMultiNodeSelfTest.java      |   23 +-
 .../client/router/TcpRouterSelfTest.java        |    4 +-
 .../client/router/TcpSslRouterSelfTest.java     |    8 +-
 .../client/suite/IgniteClientTestSuite.java     |   50 +-
 .../client/util/ClientByteUtilsTest.java        |   24 +-
 .../util/ClientConsistentHashSelfTest.java      |   19 +-
 .../client/util/ClientJavaHasherSelfTest.java   |    9 +-
 .../jdbc2/JdbcComplexQuerySelfTest.java         |  316 ++
 .../internal/jdbc2/JdbcConnectionSelfTest.java  |  268 ++
 .../internal/jdbc2/JdbcEmptyCacheSelfTest.java  |  140 +
 .../internal/jdbc2/JdbcLocalCachesSelfTest.java |  156 +
 .../internal/jdbc2/JdbcMetadataSelfTest.java    |  334 ++
 .../jdbc2/JdbcPreparedStatementSelfTest.java    |  730 ++++
 .../internal/jdbc2/JdbcResultSetSelfTest.java   |  751 ++++
 .../internal/jdbc2/JdbcStatementSelfTest.java   |  292 ++
 .../rest/AbstractRestProcessorSelfTest.java     |   16 +-
 .../rest/ClientMemcachedProtocolSelfTest.java   |   27 +-
 .../JettyRestProcessorAbstractSelfTest.java     |   44 +-
 .../rest/JettyRestProcessorSignedSelfTest.java  |   16 +-
 .../JettyRestProcessorUnsignedSelfTest.java     |    2 +-
 .../rest/MemcacheRestProcessorTest.java         |   15 +-
 .../rest/RestBinaryProtocolSelfTest.java        |   52 +-
 .../rest/RestMemcacheProtocolSelfTest.java      |   29 +-
 .../rest/RestProcessorMultiStartSelfTest.java   |    7 +-
 .../rest/RestProcessorStartSelfTest.java        |   34 +-
 .../processors/rest/RestProcessorTest.java      |   30 +-
 .../rest/TaskCommandHandlerSelfTest.java        |   59 +-
 .../processors/rest/TestBinaryClient.java       |   64 +-
 .../processors/rest/TestMemcacheClient.java     |   34 +-
 .../internal/processors/rest/TestTask1.java     |   16 +-
 .../internal/processors/rest/TestTask2.java     |   16 +-
 .../rest/protocols/tcp/MockNioSession.java      |   15 +-
 .../protocols/tcp/TcpRestParserSelfTest.java    |   43 +-
 .../ignite/jdbc/JdbcComplexQuerySelfTest.java   |   37 +-
 .../ignite/jdbc/JdbcConnectionSelfTest.java     |   25 +-
 .../ignite/jdbc/JdbcEmptyCacheSelfTest.java     |   25 +-
 .../ignite/jdbc/JdbcLocalCachesSelfTest.java    |   31 +-
 .../ignite/jdbc/JdbcMetadataSelfTest.java       |   45 +-
 .../jdbc/JdbcPreparedStatementSelfTest.java     |   53 +-
 .../ignite/jdbc/JdbcResultSetSelfTest.java      |   47 +-
 .../ignite/jdbc/JdbcStatementSelfTest.java      |   34 +-
 .../jdbc/suite/IgniteJdbcDriverTestSuite.java   |   24 +-
 .../loadtests/client/ClientCacheBenchmark.java  |   28 +-
 .../client/ClientMarshallerBenchmarkTest.java   |   27 +-
 .../loadtests/client/ClientTcpSslLoadTest.java  |    8 +-
 modules/cloud/pom.xml                           |    3 +-
 .../cloud/TcpDiscoveryCloudIpFinder.java        |   59 +-
 .../TcpDiscoveryCloudIpFinderSelfTest.java      |   15 +-
 .../ignite/testsuites/IgniteCloudTestSuite.java |    9 +-
 modules/codegen/pom.xml                         |    3 +-
 .../ignite/codegen/MessageCodeGenerator.java    |   79 +-
 modules/core/pom.xml                            |    9 +-
 modules/core/src/main/java/META-INF/LICENSE     |    2 +-
 modules/core/src/main/java/META-INF/NOTICE      |    2 +-
 .../src/main/java/org/apache/ignite/Ignite.java |   28 +-
 .../org/apache/ignite/IgniteAtomicLong.java     |    4 +-
 .../apache/ignite/IgniteAtomicReference.java    |    4 +-
 .../org/apache/ignite/IgniteAtomicSequence.java |    4 +-
 .../org/apache/ignite/IgniteAtomicStamped.java  |    7 +-
 .../ignite/IgniteAuthenticationException.java   |    2 +-
 .../java/org/apache/ignite/IgniteCache.java     |   58 +-
 .../apache/ignite/IgniteCheckedException.java   |    6 +-
 .../IgniteClientDisconnectedException.java      |    6 +-
 .../java/org/apache/ignite/IgniteCluster.java   |   23 +-
 .../java/org/apache/ignite/IgniteCompute.java   |   33 +-
 .../org/apache/ignite/IgniteCountDownLatch.java |    6 +-
 .../org/apache/ignite/IgniteDataStreamer.java   |   15 +-
 .../ignite/IgniteDeploymentException.java       |    4 +-
 .../java/org/apache/ignite/IgniteEvents.java    |   18 +-
 .../java/org/apache/ignite/IgniteException.java |    6 +-
 .../org/apache/ignite/IgniteFileSystem.java     |   25 +-
 .../ignite/IgniteIllegalStateException.java     |    6 +-
 .../ignite/IgniteInterruptedException.java      |    2 +-
 .../org/apache/ignite/IgniteJdbcDriver.java     |  295 +-
 .../java/org/apache/ignite/IgniteLogger.java    |    6 +-
 .../java/org/apache/ignite/IgniteMessaging.java |   17 +-
 .../java/org/apache/ignite/IgniteQueue.java     |   10 +-
 .../java/org/apache/ignite/IgniteScheduler.java |   13 +-
 .../java/org/apache/ignite/IgniteServices.java  |   20 +-
 .../main/java/org/apache/ignite/IgniteSet.java  |    8 +-
 .../java/org/apache/ignite/IgniteState.java     |    6 +-
 .../apache/ignite/IgniteSystemProperties.java   |   15 +-
 .../org/apache/ignite/IgniteTransactions.java   |   12 +-
 .../main/java/org/apache/ignite/Ignition.java   |   22 +-
 .../org/apache/ignite/IgnitionListener.java     |    7 +-
 .../CacheAtomicUpdateTimeoutException.java      |    4 +-
 .../ignite/cache/CacheAtomicWriteOrderMode.java |    4 +-
 .../apache/ignite/cache/CacheAtomicityMode.java |    6 +-
 .../org/apache/ignite/cache/CacheEntry.java     |   93 +
 .../CacheEntryEventSerializableFilter.java      |    6 +-
 .../ignite/cache/CacheEntryProcessor.java       |    6 +-
 .../ignite/cache/CacheExistsException.java      |    7 +-
 .../apache/ignite/cache/CacheInterceptor.java   |   15 +-
 .../ignite/cache/CacheInterceptorAdapter.java   |    9 +-
 .../org/apache/ignite/cache/CacheManager.java   |   55 +-
 .../apache/ignite/cache/CacheMemoryMode.java    |    4 +-
 .../org/apache/ignite/cache/CacheMetrics.java   |   10 +-
 .../java/org/apache/ignite/cache/CacheMode.java |    8 +-
 .../cache/CachePartialUpdateException.java      |    9 +-
 .../org/apache/ignite/cache/CachePeekMode.java  |    6 +-
 .../apache/ignite/cache/CacheRebalanceMode.java |    6 +-
 .../cache/CacheServerNotFoundException.java     |    4 +-
 .../ignite/cache/CacheTypeFieldMetadata.java    |    7 +-
 .../apache/ignite/cache/CacheTypeMetadata.java  |   19 +-
 .../cache/CacheWriteSynchronizationMode.java    |    6 +-
 .../apache/ignite/cache/CachingProvider.java    |   30 +-
 .../apache/ignite/cache/affinity/Affinity.java  |   10 +-
 .../affinity/AffinityCentralizedFunction.java   |    7 +-
 .../ignite/cache/affinity/AffinityFunction.java |   10 +-
 .../cache/affinity/AffinityFunctionContext.java |   13 +-
 .../ignite/cache/affinity/AffinityKey.java      |   15 +-
 .../cache/affinity/AffinityKeyMapped.java       |   10 +-
 .../cache/affinity/AffinityKeyMapper.java       |    4 +-
 .../AffinityNodeAddressHashResolver.java        |   11 +-
 .../affinity/AffinityNodeHashResolver.java      |    9 +-
 .../affinity/AffinityNodeIdHashResolver.java    |   11 +-
 .../ignite/cache/affinity/AffinityUuid.java     |    6 +-
 .../affinity/fair/FairAffinityFunction.java     |   34 +-
 .../rendezvous/RendezvousAffinityFunction.java  |   49 +-
 .../ignite/cache/eviction/EvictableEntry.java   |    7 +-
 .../ignite/cache/eviction/EvictionFilter.java   |    7 +-
 .../ignite/cache/eviction/EvictionPolicy.java   |   10 +-
 .../cache/eviction/fifo/FifoEvictionPolicy.java |   27 +-
 .../eviction/fifo/FifoEvictionPolicyMBean.java  |    4 +-
 .../cache/eviction/igfs/IgfsEvictionFilter.java |    9 +-
 .../igfs/IgfsPerBlockLruEvictionPolicy.java     |   35 +-
 .../IgfsPerBlockLruEvictionPolicyMXBean.java    |    9 +-
 .../cache/eviction/lru/LruEvictionPolicy.java   |   27 +-
 .../eviction/lru/LruEvictionPolicyMBean.java    |    4 +-
 .../eviction/random/RandomEvictionPolicy.java   |   24 +-
 .../random/RandomEvictionPolicyMBean.java       |    4 +-
 .../eviction/sorted/SortedEvictionPolicy.java   |   36 +-
 .../sorted/SortedEvictionPolicyMBean.java       |    4 +-
 .../ignite/cache/query/ContinuousQuery.java     |   11 +-
 .../org/apache/ignite/cache/query/Query.java    |    9 +-
 .../apache/ignite/cache/query/QueryCursor.java  |    4 +-
 .../apache/ignite/cache/query/QueryMetrics.java |    4 +-
 .../apache/ignite/cache/query/ScanQuery.java    |   13 +-
 .../org/apache/ignite/cache/query/SpiQuery.java |   13 +-
 .../ignite/cache/query/SqlFieldsQuery.java      |   12 +-
 .../org/apache/ignite/cache/query/SqlQuery.java |   14 +-
 .../apache/ignite/cache/query/TextQuery.java    |   12 +-
 .../query/annotations/QueryGroupIndex.java      |    7 +-
 .../cache/query/annotations/QuerySqlField.java  |   11 +-
 .../query/annotations/QuerySqlFunction.java     |    8 +-
 .../cache/query/annotations/QueryTextField.java |   11 +-
 .../cache/store/CacheLoadOnlyStoreAdapter.java  |   36 +-
 .../apache/ignite/cache/store/CacheStore.java   |   20 +-
 .../ignite/cache/store/CacheStoreAdapter.java   |   12 +-
 .../ignite/cache/store/CacheStoreSession.java   |   11 +-
 .../cache/store/CacheStoreSessionListener.java  |   12 +-
 .../store/jdbc/CacheAbstractJdbcStore.java      |  128 +-
 .../cache/store/jdbc/CacheJdbcBlobStore.java    |   57 +-
 .../store/jdbc/CacheJdbcBlobStoreFactory.java   |   24 +-
 .../cache/store/jdbc/CacheJdbcPojoStore.java    |   63 +-
 .../store/jdbc/CacheJdbcPojoStoreFactory.java   |   27 +-
 .../jdbc/CacheJdbcStoreSessionListener.java     |   22 +-
 .../store/jdbc/dialect/BasicJdbcDialect.java    |   13 +-
 .../cache/store/jdbc/dialect/DB2Dialect.java    |   11 +-
 .../cache/store/jdbc/dialect/H2Dialect.java     |   10 +-
 .../cache/store/jdbc/dialect/JdbcDialect.java   |    7 +-
 .../cache/store/jdbc/dialect/MySQLDialect.java  |   11 +-
 .../cache/store/jdbc/dialect/OracleDialect.java |   11 +-
 .../store/jdbc/dialect/SQLServerDialect.java    |   11 +-
 .../org/apache/ignite/cluster/ClusterGroup.java |   17 +-
 .../cluster/ClusterGroupEmptyException.java     |    4 +-
 .../apache/ignite/cluster/ClusterMetrics.java   |    6 +-
 .../org/apache/ignite/cluster/ClusterNode.java  |   20 +-
 .../ignite/cluster/ClusterStartNodeResult.java  |    4 +-
 .../cluster/ClusterTopologyException.java       |   24 +-
 .../ComputeExecutionRejectedException.java      |    6 +-
 .../org/apache/ignite/compute/ComputeJob.java   |   10 +-
 .../ignite/compute/ComputeJobAdapter.java       |    9 +-
 .../ignite/compute/ComputeJobAfterSend.java     |    8 +-
 .../compute/ComputeJobBeforeFailover.java       |    8 +-
 .../ignite/compute/ComputeJobContext.java       |   10 +-
 .../ignite/compute/ComputeJobContinuation.java  |    4 +-
 .../compute/ComputeJobContinuationAdapter.java  |    6 +-
 .../compute/ComputeJobFailoverException.java    |   12 +-
 .../compute/ComputeJobMasterLeaveAware.java     |    4 +-
 .../apache/ignite/compute/ComputeJobResult.java |   10 +-
 .../ignite/compute/ComputeJobResultPolicy.java  |    7 +-
 .../ignite/compute/ComputeJobSibling.java       |    9 +-
 .../ignite/compute/ComputeLoadBalancer.java     |   12 +-
 .../org/apache/ignite/compute/ComputeTask.java  |   14 +-
 .../ignite/compute/ComputeTaskAdapter.java      |    9 +-
 .../compute/ComputeTaskCancelledException.java  |    6 +-
 .../compute/ComputeTaskContinuousMapper.java    |   11 +-
 .../ignite/compute/ComputeTaskFuture.java       |   10 +-
 .../ignite/compute/ComputeTaskMapAsync.java     |   10 +-
 .../apache/ignite/compute/ComputeTaskName.java  |    8 +-
 .../compute/ComputeTaskNoResultCache.java       |   10 +-
 .../ignite/compute/ComputeTaskSession.java      |   15 +-
 .../ComputeTaskSessionAttributeListener.java    |    4 +-
 .../compute/ComputeTaskSessionFullSupport.java  |   10 +-
 .../ignite/compute/ComputeTaskSessionScope.java |    4 +-
 .../apache/ignite/compute/ComputeTaskSpis.java  |    8 +-
 .../ignite/compute/ComputeTaskSplitAdapter.java |   17 +-
 .../compute/ComputeTaskTimeoutException.java    |    6 +-
 .../compute/ComputeUserUndeclaredException.java |    6 +-
 .../apache/ignite/compute/gridify/Gridify.java  |   15 +-
 .../ignite/compute/gridify/GridifyArgument.java |    4 +-
 .../ignite/compute/gridify/GridifyInput.java    |    8 +-
 .../compute/gridify/GridifyInterceptor.java     |    7 +-
 .../compute/gridify/GridifyNodeFilter.java      |    8 +-
 .../gridify/GridifyRuntimeException.java        |    4 +-
 .../ignite/compute/gridify/GridifySetToSet.java |   10 +-
 .../compute/gridify/GridifySetToValue.java      |   10 +-
 .../compute/gridify/GridifyTaskAdapter.java     |    7 +-
 .../gridify/GridifyTaskSplitAdapter.java        |    7 +-
 .../gridify/aop/GridifyArgumentAdapter.java     |    7 +-
 .../gridify/aop/GridifyDefaultRangeTask.java    |   46 +-
 .../compute/gridify/aop/GridifyDefaultTask.java |   27 +-
 .../aop/GridifySetToSetAbstractAspect.java      |   27 +-
 .../aop/GridifySetToValueAbstractAspect.java    |   29 +-
 .../ignite/configuration/AddressResolver.java   |    9 +-
 .../configuration/AtomicConfiguration.java      |   10 +-
 .../configuration/CacheConfiguration.java       |   55 +-
 .../configuration/CollectionConfiguration.java  |   23 +-
 .../configuration/ConnectorConfiguration.java   |   18 +-
 .../ConnectorMessageInterceptor.java            |    4 +-
 .../ignite/configuration/DeploymentMode.java    |    4 +-
 .../configuration/FileSystemConfiguration.java  |   17 +-
 .../configuration/HadoopConfiguration.java      |    6 +-
 .../configuration/IgniteConfiguration.java      |  109 +-
 .../configuration/IgniteReflectionFactory.java  |   18 +-
 .../configuration/NearCacheConfiguration.java   |   27 +-
 .../ignite/configuration/TopologyValidator.java |   37 +-
 .../configuration/TransactionConfiguration.java |    9 +-
 .../org/apache/ignite/events/CacheEvent.java    |   18 +-
 .../ignite/events/CacheQueryExecutedEvent.java  |   18 +-
 .../ignite/events/CacheQueryReadEvent.java      |   18 +-
 .../ignite/events/CacheRebalancingEvent.java    |    8 +-
 .../apache/ignite/events/CheckpointEvent.java   |    7 +-
 .../apache/ignite/events/DeploymentEvent.java   |    7 +-
 .../apache/ignite/events/DiscoveryEvent.java    |   12 +-
 .../java/org/apache/ignite/events/Event.java    |   11 +-
 .../org/apache/ignite/events/EventAdapter.java  |   12 +-
 .../apache/ignite/events/EventLocalOrder.java   |    4 +-
 .../org/apache/ignite/events/EventType.java     |    9 +-
 .../org/apache/ignite/events/IgfsEvent.java     |   19 +-
 .../java/org/apache/ignite/events/JobEvent.java |   12 +-
 .../apache/ignite/events/SwapSpaceEvent.java    |    9 +-
 .../org/apache/ignite/events/TaskEvent.java     |   14 +-
 .../apache/ignite/igfs/IgfsBlockLocation.java   |    5 +-
 .../IgfsConcurrentModificationException.java    |    4 +-
 .../ignite/igfs/IgfsCorruptedFileException.java |    4 +-
 .../igfs/IgfsDirectoryNotEmptyException.java    |    4 +-
 .../org/apache/ignite/igfs/IgfsException.java   |    6 +-
 .../java/org/apache/ignite/igfs/IgfsFile.java   |    7 +-
 .../igfs/IgfsGroupDataBlocksKeyMapper.java      |    9 +-
 .../org/apache/ignite/igfs/IgfsInputStream.java |    5 +-
 .../igfs/IgfsInvalidHdfsVersionException.java   |    4 +-
 .../ignite/igfs/IgfsInvalidPathException.java   |    4 +-
 .../igfs/IgfsIpcEndpointConfiguration.java      |   10 +-
 .../apache/ignite/igfs/IgfsIpcEndpointType.java |    2 +-
 .../org/apache/ignite/igfs/IgfsMetrics.java     |    2 +-
 .../java/org/apache/ignite/igfs/IgfsMode.java   |    4 +-
 .../ignite/igfs/IgfsOutOfSpaceException.java    |    4 +-
 .../apache/ignite/igfs/IgfsOutputStream.java    |    6 +-
 .../java/org/apache/ignite/igfs/IgfsPath.java   |   23 +-
 .../igfs/IgfsPathAlreadyExistsException.java    |    5 +-
 .../igfs/IgfsPathIsDirectoryException.java      |    4 +-
 .../igfs/IgfsPathIsNotDirectoryException.java   |    4 +-
 .../ignite/igfs/IgfsPathNotFoundException.java  |    4 +-
 .../org/apache/ignite/igfs/IgfsPathSummary.java |   10 +-
 .../org/apache/ignite/igfs/IgfsUserContext.java |   11 +-
 .../ignite/igfs/mapreduce/IgfsFileRange.java    |    6 +-
 .../mapreduce/IgfsInputStreamJobAdapter.java    |   12 +-
 .../apache/ignite/igfs/mapreduce/IgfsJob.java   |   10 +-
 .../ignite/igfs/mapreduce/IgfsJobAdapter.java   |    2 +-
 .../igfs/mapreduce/IgfsRangeInputStream.java    |   12 +-
 .../igfs/mapreduce/IgfsRecordResolver.java      |   13 +-
 .../apache/ignite/igfs/mapreduce/IgfsTask.java  |   32 +-
 .../ignite/igfs/mapreduce/IgfsTaskArgs.java     |    7 +-
 .../igfs/mapreduce/IgfsTaskNoReduceAdapter.java |    7 +-
 .../IgfsByteDelimiterRecordResolver.java        |   31 +-
 .../records/IgfsFixedLengthRecordResolver.java  |   18 +-
 .../records/IgfsNewLineRecordResolver.java      |    7 +-
 .../IgfsStringDelimiterRecordResolver.java      |   11 +-
 .../igfs/secondary/IgfsSecondaryFileSystem.java |   15 +-
 ...fsSecondaryFileSystemPositionedReadable.java |    5 +-
 .../ignite/internal/AsyncSupportAdapter.java    |    9 +-
 .../ClusterLocalNodeMetricsMXBeanImpl.java      |    8 +-
 .../ignite/internal/ClusterMetricsSnapshot.java |   24 +-
 .../internal/ComputeTaskInternalFuture.java     |   35 +-
 .../ignite/internal/GridCachePluginContext.java |   13 +-
 .../ignite/internal/GridClosureCallMode.java    |    4 +-
 .../apache/ignite/internal/GridComponent.java   |   17 +-
 .../apache/ignite/internal/GridDiagnostic.java  |   20 +-
 .../ignite/internal/GridDirectCollection.java   |   10 +-
 .../apache/ignite/internal/GridDirectMap.java   |    7 +-
 .../ignite/internal/GridDirectTransient.java    |    7 +-
 .../internal/GridEventConsumeHandler.java       |   65 +-
 .../ignite/internal/GridInternalException.java  |    9 +-
 .../ignite/internal/GridInternalWrapper.java    |    2 +-
 .../ignite/internal/GridJobCancelRequest.java   |   19 +-
 .../ignite/internal/GridJobContextImpl.java     |   35 +-
 .../ignite/internal/GridJobExecuteRequest.java  |   34 +-
 .../ignite/internal/GridJobExecuteResponse.java |   26 +-
 .../ignite/internal/GridJobResultImpl.java      |   19 +-
 .../ignite/internal/GridJobSessionImpl.java     |   24 +-
 .../ignite/internal/GridJobSiblingImpl.java     |   30 +-
 .../ignite/internal/GridJobSiblingsRequest.java |   17 +-
 .../internal/GridJobSiblingsResponse.java       |   25 +-
 .../ignite/internal/GridKernalContext.java      |  103 +-
 .../ignite/internal/GridKernalContextImpl.java  |  150 +-
 .../ignite/internal/GridKernalGateway.java      |    7 +-
 .../ignite/internal/GridKernalGatewayImpl.java  |   23 +-
 .../apache/ignite/internal/GridKernalState.java |    4 +-
 .../apache/ignite/internal/GridLoggerProxy.java |   33 +-
 .../internal/GridMessageListenHandler.java      |   27 +-
 .../internal/GridNodeOrderComparator.java       |    9 +-
 .../internal/GridPerformanceSuggestions.java    |   18 +-
 .../ignite/internal/GridPluginComponent.java    |   20 +-
 .../ignite/internal/GridPluginContext.java      |   18 +-
 .../ignite/internal/GridProxyListener.java      |    4 +-
 .../ignite/internal/GridTaskCancelRequest.java  |   17 +-
 .../ignite/internal/GridTaskNameHashKey.java    |   10 +-
 .../ignite/internal/GridTaskSessionImpl.java    |   36 +-
 .../internal/GridTaskSessionInternal.java       |   11 +-
 .../ignite/internal/GridTaskSessionRequest.java |   19 +-
 .../org/apache/ignite/internal/GridTopic.java   |   20 +-
 .../ignite/internal/GridUpdateNotifier.java     |  126 +-
 ...gniteClientDisconnectedCheckedException.java |    6 +-
 .../internal/IgniteCodeGeneratingFail.java      |    7 +-
 .../ignite/internal/IgniteComponentType.java    |   13 +-
 .../ignite/internal/IgniteComputeImpl.java      |   52 +-
 .../IgniteDeploymentCheckedException.java       |    6 +-
 .../ignite/internal/IgniteEventsImpl.java       |   32 +-
 .../org/apache/ignite/internal/IgniteEx.java    |   30 +-
 .../IgniteFutureCancelledCheckedException.java  |    6 +-
 .../IgniteFutureTimeoutCheckedException.java    |    6 +-
 .../ignite/internal/IgniteInternalFuture.java   |   21 +-
 .../IgniteInterruptedCheckedException.java      |    4 +-
 .../apache/ignite/internal/IgniteKernal.java    |  401 ++-
 .../ignite/internal/IgniteMessagingImpl.java    |   31 +-
 .../ignite/internal/IgniteProperties.java       |    7 +-
 .../ignite/internal/IgniteSchedulerImpl.java    |   21 +-
 .../ignite/internal/IgniteServicesImpl.java     |   28 +-
 .../ignite/internal/IgniteTransactionsEx.java   |   11 +-
 .../ignite/internal/IgniteVersionUtils.java     |    9 +-
 .../org/apache/ignite/internal/IgnitionEx.java  |  194 +-
 .../ignite/internal/IgnitionMXBeanAdapter.java  |    7 +-
 .../internal/MarshallerContextAdapter.java      |   34 +-
 .../ignite/internal/MarshallerContextImpl.java  |   58 +-
 .../org/apache/ignite/internal/SkipDaemon.java  |    7 +-
 .../internal/ThreadPoolMXBeanAdapter.java       |   13 +-
 .../ignite/internal/client/GridClient.java      |    5 +-
 .../GridClientAuthenticationException.java      |    2 +-
 .../internal/client/GridClientCacheFlag.java    |    4 +-
 .../internal/client/GridClientCacheMode.java    |    2 +-
 .../client/GridClientClosedException.java       |    2 +-
 .../internal/client/GridClientCompute.java      |   11 +-
 .../client/GridClientConfiguration.java         |   35 +-
 .../ignite/internal/client/GridClientData.java  |    7 +-
 .../internal/client/GridClientDataAffinity.java |    4 +-
 .../client/GridClientDataConfiguration.java     |    5 +-
 .../internal/client/GridClientDataMetrics.java  |    4 +-
 .../client/GridClientDisconnectedException.java |    2 +-
 .../internal/client/GridClientException.java    |    6 +-
 .../internal/client/GridClientFactory.java      |   13 +-
 .../internal/client/GridClientFuture.java       |    4 +-
 .../client/GridClientFutureListener.java        |    2 +-
 .../GridClientFutureTimeoutException.java       |    2 +-
 .../client/GridClientHandshakeException.java    |    2 +-
 .../ignite/internal/client/GridClientNode.java  |   12 +-
 .../internal/client/GridClientNodeMetrics.java  |    4 +-
 .../client/GridClientPartitionAffinity.java     |   15 +-
 .../internal/client/GridClientPredicate.java    |    2 +-
 .../internal/client/GridClientProtocol.java     |    2 +-
 .../client/GridClientTopologyListener.java      |    2 +-
 .../client/GridServerUnreachableException.java  |    2 +-
 .../balancer/GridClientBalancerAdapter.java     |   11 +-
 .../client/balancer/GridClientLoadBalancer.java |    8 +-
 .../balancer/GridClientRandomBalancer.java      |   10 +-
 .../balancer/GridClientRoundRobinBalancer.java  |   18 +-
 .../impl/GridClientAbstractProjection.java      |   39 +-
 .../client/impl/GridClientAndPredicate.java     |    4 +-
 .../client/impl/GridClientComputeImpl.java      |   30 +-
 .../client/impl/GridClientDataImpl.java         |   32 +-
 .../impl/GridClientDataMetricsAdapter.java      |    4 +-
 .../client/impl/GridClientFutureAdapter.java    |   20 +-
 .../client/impl/GridClientFutureCallback.java   |    5 +-
 .../internal/client/impl/GridClientImpl.java    |   66 +-
 .../client/impl/GridClientNodeImpl.java         |   23 +-
 .../impl/GridClientNodeMetricsAdapter.java      |    4 +-
 .../client/impl/GridClientThreadFactory.java    |    9 +-
 .../impl/connection/GridClientConnection.java   |   29 +-
 .../GridClientConnectionCloseReason.java        |    2 +-
 .../connection/GridClientConnectionManager.java |   15 +-
 .../GridClientConnectionManagerAdapter.java     |   79 +-
 .../GridClientConnectionManagerOsImpl.java      |   13 +-
 .../GridClientConnectionResetException.java     |    4 +-
 .../connection/GridClientNioTcpConnection.java  |  105 +-
 .../impl/connection/GridClientTopology.java     |   45 +-
 .../GridConnectionIdleClosedException.java      |    2 +-
 .../client/marshaller/GridClientMarshaller.java |    6 +-
 .../marshaller/jdk/GridClientJdkMarshaller.java |   16 +-
 .../GridClientOptimizedMarshaller.java          |   24 +-
 .../client/router/GridRouterFactory.java        |   17 +-
 .../internal/client/router/GridTcpRouter.java   |    4 +-
 .../router/GridTcpRouterConfiguration.java      |   22 +-
 .../client/router/GridTcpRouterMBean.java       |    7 +-
 .../router/impl/GridRouterClientImpl.java       |   40 +-
 .../impl/GridRouterCommandLineStartup.java      |   32 +-
 .../client/router/impl/GridTcpRouterImpl.java   |   52 +-
 .../impl/GridTcpRouterNioListenerAdapter.java   |   41 +-
 .../impl/GridTcpRouterNioListenerOsImpl.java    |    4 +-
 .../router/impl/GridTcpRouterNioParser.java     |   28 +-
 .../client/ssl/GridSslBasicContextFactory.java  |   27 +-
 .../client/ssl/GridSslContextFactory.java       |    7 +-
 .../client/util/GridClientConsistentHash.java   |   25 +-
 .../client/util/GridClientStripedLock.java      |    5 +-
 .../internal/client/util/GridClientUtils.java   |   22 +-
 .../client/util/GridConcurrentHashSet.java      |    9 +-
 .../internal/cluster/ClusterGroupAdapter.java   |   60 +-
 .../ClusterGroupEmptyCheckedException.java      |    4 +-
 .../ignite/internal/cluster/ClusterGroupEx.java |   10 +-
 .../cluster/ClusterNodeLocalMapImpl.java        |   21 +-
 .../cluster/ClusterStartNodeResultImpl.java     |    4 +-
 .../ClusterTopologyCheckedException.java        |   24 +-
 .../ClusterTopologyServerNotFoundException.java |    2 +-
 .../cluster/IgniteClusterAsyncImpl.java         |   34 +-
 .../internal/cluster/IgniteClusterEx.java       |    4 +-
 .../internal/cluster/IgniteClusterImpl.java     |   70 +-
 .../ignite/internal/cluster/IgniteKillTask.java |   24 +-
 .../ComputeTaskCancelledCheckedException.java   |    6 +-
 .../ComputeTaskTimeoutCheckedException.java     |    6 +-
 .../internal/direct/DirectByteBufferStream.java |   34 +-
 .../internal/direct/DirectMessageReader.java    |   26 +-
 .../internal/direct/DirectMessageWriter.java    |   18 +-
 .../direct/DirectMessageWriterState.java        |    4 +-
 .../internal/events/DiscoveryCustomEvent.java   |   11 +-
 .../internal/executor/GridExecutorService.java  |   47 +-
 .../igfs/common/IgfsControlResponse.java        |   42 +-
 .../igfs/common/IgfsDataInputStream.java        |    7 +-
 .../igfs/common/IgfsDataOutputStream.java       |    8 +-
 .../igfs/common/IgfsHandshakeRequest.java       |    6 +-
 .../internal/igfs/common/IgfsIpcCommand.java    |    5 +-
 .../ignite/internal/igfs/common/IgfsLogger.java |   34 +-
 .../internal/igfs/common/IgfsMarshaller.java    |   26 +-
 .../internal/igfs/common/IgfsMessage.java       |    2 +-
 .../igfs/common/IgfsPathControlRequest.java     |   13 +-
 .../internal/igfs/common/IgfsStatusRequest.java |    4 +-
 .../igfs/common/IgfsStreamControlRequest.java   |    6 +-
 .../interop/InteropAwareEventFilter.java        |   37 -
 .../internal/interop/InteropBootstrap.java      |   35 -
 .../interop/InteropBootstrapFactory.java        |   39 -
 .../internal/interop/InteropException.java      |   71 -
 .../internal/interop/InteropIgnition.java       |  241 --
 .../interop/InteropLocalEventListener.java      |   28 -
 .../interop/InteropNoCallbackException.java     |   50 -
 .../internal/interop/InteropProcessor.java      |   51 -
 .../ignite/internal/jdbc/JdbcConnection.java    |   51 +-
 .../internal/jdbc/JdbcConnectionInfo.java       |   91 -
 .../internal/jdbc/JdbcDatabaseMetadata.java     |   34 +-
 .../internal/jdbc/JdbcPreparedStatement.java    |   31 +-
 .../ignite/internal/jdbc/JdbcResultSet.java     |   42 +-
 .../internal/jdbc/JdbcResultSetMetadata.java    |   11 +-
 .../ignite/internal/jdbc/JdbcStatement.java     |   29 +-
 .../apache/ignite/internal/jdbc/JdbcUtils.java  |   41 +-
 .../ignite/internal/jdbc2/JdbcConnection.java   |  777 +++++
 .../internal/jdbc2/JdbcDatabaseMetadata.java    | 1401 ++++++++
 .../internal/jdbc2/JdbcPreparedStatement.java   |  411 +++
 .../ignite/internal/jdbc2/JdbcQueryTask.java    |  361 ++
 .../ignite/internal/jdbc2/JdbcResultSet.java    | 1520 ++++++++
 .../internal/jdbc2/JdbcResultSetMetadata.java   |  171 +
 .../ignite/internal/jdbc2/JdbcStatement.java    |  456 +++
 .../apache/ignite/internal/jdbc2/JdbcUtils.java |  155 +
 .../ignite/internal/managers/GridManager.java   |    6 +-
 .../internal/managers/GridManagerAdapter.java   |   69 +-
 .../checkpoint/GridCheckpointManager.java       |   65 +-
 .../checkpoint/GridCheckpointRequest.java       |   17 +-
 .../GridCollisionJobContextAdapter.java         |   11 +-
 .../collision/GridCollisionManager.java         |   23 +-
 .../communication/GridDisconnectListener.java   |    4 +-
 .../managers/communication/GridIoManager.java   |  130 +-
 .../managers/communication/GridIoMessage.java   |   25 +-
 .../communication/GridIoMessageFactory.java     |  146 +-
 .../managers/communication/GridIoPolicy.java    |    2 +-
 .../communication/GridIoUserMessage.java        |   33 +-
 .../GridLifecycleAwareMessageFilter.java        |   38 -
 .../communication/GridMessageListener.java      |    5 +-
 .../managers/deployment/GridDeployment.java     |   50 +-
 .../deployment/GridDeploymentClassLoader.java   |   40 +-
 .../deployment/GridDeploymentCommunication.java |   50 +-
 .../managers/deployment/GridDeploymentInfo.java |   10 +-
 .../deployment/GridDeploymentInfoBean.java      |   31 +-
 .../deployment/GridDeploymentLocalStore.java    |   59 +-
 .../deployment/GridDeploymentManager.java       |   46 +-
 .../deployment/GridDeploymentMetadata.java      |   17 +-
 .../GridDeploymentPerLoaderStore.java           |   49 +-
 .../GridDeploymentPerVersionStore.java          |   65 +-
 .../deployment/GridDeploymentRequest.java       |   26 +-
 .../deployment/GridDeploymentResponse.java      |   17 +-
 .../deployment/GridDeploymentStore.java         |   13 +-
 .../deployment/GridDeploymentStoreAdapter.java  |   20 +-
 .../protocol/gg/GridProtocolHandler.java        |   11 +-
 .../protocol/gg/GridUrlConnection.java          |   14 +-
 .../managers/discovery/CustomEventListener.java |    4 +-
 .../discovery/CustomMessageWrapper.java         |    6 +-
 .../discovery/DiscoveryCustomMessage.java       |   11 +-
 .../discovery/GridDiscoveryManager.java         |  207 +-
 .../GridDiscoveryTopologySnapshot.java          |   11 +-
 .../managers/discovery/GridLocalMetrics.java    |    4 +-
 .../eventstorage/GridEventStorageManager.java   |   98 +-
 .../eventstorage/GridEventStorageMessage.java   |   32 +-
 .../eventstorage/GridLocalEventListener.java    |    7 +-
 .../failover/GridFailoverContextImpl.java       |   22 +-
 .../managers/failover/GridFailoverManager.java  |   20 +-
 .../managers/indexing/GridIndexingManager.java  |   22 +-
 .../loadbalancer/GridLoadBalancerAdapter.java   |   12 +-
 .../loadbalancer/GridLoadBalancerManager.java   |   27 +-
 .../swapspace/GridSwapSpaceManager.java         |   46 +-
 .../internal/mxbean/IgniteStandardMXBean.java   |   21 +-
 .../portable/GridPortableMarshaller.java        |  304 ++
 .../portable/PortableClassDescriptor.java       | 1365 ++++++++
 .../internal/portable/PortableContext.java      | 1143 ++++++
 .../portable/PortableMetaDataCollector.java     |  258 ++
 .../portable/PortableMetaDataHandler.java       |   44 +
 .../internal/portable/PortableMetaDataImpl.java |  150 +
 .../internal/portable/PortableObjectEx.java     |  214 ++
 .../internal/portable/PortableObjectImpl.java   |  391 +++
 .../portable/PortableObjectOffheapImpl.java     |  243 ++
 .../internal/portable/PortablePrimitives.java   |  773 +++++
 .../internal/portable/PortableRawReaderEx.java  |   33 +
 .../internal/portable/PortableRawWriterEx.java  |   60 +
 .../portable/PortableReaderContext.java         |   82 +
 .../internal/portable/PortableReaderExImpl.java | 3157 +++++++++++++++++
 .../PortableThreadLocalMemoryAllocator.java     |  162 +
 .../ignite/internal/portable/PortableUtils.java |  469 +++
 .../internal/portable/PortableWriterExImpl.java | 1850 ++++++++++
 .../internal/portable/api/IgnitePortables.java  |  362 ++
 .../internal/portable/api/PortableBuilder.java  |  136 +
 .../portable/api/PortableException.java         |   57 +
 .../internal/portable/api/PortableIdMapper.java |   54 +
 .../api/PortableInvalidClassException.java      |   58 +
 .../portable/api/PortableMarshalAware.java      |   48 +
 .../portable/api/PortableMarshaller.java        |  358 ++
 .../internal/portable/api/PortableMetadata.java |   60 +
 .../internal/portable/api/PortableObject.java   |  152 +
 .../portable/api/PortableProtocolVersion.java   |   41 +
 .../portable/api/PortableRawReader.java         |  234 ++
 .../portable/api/PortableRawWriter.java         |  219 ++
 .../internal/portable/api/PortableReader.java   |  284 ++
 .../portable/api/PortableSerializer.java        |   47 +
 .../portable/api/PortableTypeConfiguration.java |  195 ++
 .../internal/portable/api/PortableWriter.java   |  266 ++
 .../builder/PortableAbstractLazyValue.java      |   57 +
 .../portable/builder/PortableBuilderEnum.java   |  116 +
 .../portable/builder/PortableBuilderImpl.java   |  533 +++
 .../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 +
 .../ignite/internal/portable/package-info.java  |   22 +
 .../streams/PortableAbstractInputStream.java    |  343 ++
 .../streams/PortableAbstractOutputStream.java   |  323 ++
 .../streams/PortableAbstractStream.java         |   80 +
 .../streams/PortableHeapInputStream.java        |  134 +
 .../streams/PortableHeapOutputStream.java       |  156 +
 .../portable/streams/PortableInputStream.java   |  168 +
 .../streams/PortableMemoryAllocator.java        |   76 +
 .../streams/PortableOffheapInputStream.java     |  129 +
 .../streams/PortableOffheapOutputStream.java    |  169 +
 .../portable/streams/PortableOutputStream.java  |  165 +
 .../streams/PortableSimpleMemoryAllocator.java  |   66 +
 .../portable/streams/PortableStream.java        |   53 +
 .../internal/portable/streams/package-info.java |   22 +
 .../internal/processors/GridProcessor.java      |    6 +-
 .../processors/GridProcessorAdapter.java        |   23 +-
 .../affinity/AffinityTopologyVersion.java       |   18 +-
 .../affinity/GridAffinityAssignment.java        |   17 +-
 .../affinity/GridAffinityAssignmentCache.java   |   47 +-
 .../GridAffinityFunctionContextImpl.java        |   14 +-
 .../affinity/GridAffinityMessage.java           |   21 +-
 .../affinity/GridAffinityProcessor.java         |   92 +-
 .../processors/affinity/GridAffinityUtils.java  |   41 +-
 ...acheAtomicUpdateTimeoutCheckedException.java |    4 +-
 .../processors/cache/CacheEntryImpl.java        |   28 +-
 .../processors/cache/CacheEntryImpl0.java       |   10 +-
 .../processors/cache/CacheEntryImplEx.java      |   84 +
 .../cache/CacheEntryInfoCollection.java         |   17 +-
 .../processors/cache/CacheEntryPredicate.java   |    8 +-
 .../cache/CacheEntryPredicateAdapter.java       |   15 +-
 .../cache/CacheEntryPredicateContainsValue.java |   19 +-
 .../cache/CacheEntryPredicateHasValue.java      |    2 +-
 .../cache/CacheEntryPredicateNoValue.java       |    2 +-
 .../cache/CacheEntrySerializablePredicate.java  |   16 +-
 .../cache/CacheEvictableEntryImpl.java          |   23 +-
 .../processors/cache/CacheEvictionEntry.java    |   17 +-
 .../cache/CacheInvokeDirectResult.java          |   24 +-
 .../processors/cache/CacheInvokeEntry.java      |   37 +-
 .../processors/cache/CacheInvokeResult.java     |   17 +-
 .../cache/CacheIteratorConverter.java           |    2 +-
 .../processors/cache/CacheLazyEntry.java        |   12 +-
 .../processors/cache/CacheLockImpl.java         |   21 +-
 .../processors/cache/CacheMetricsImpl.java      |   21 +-
 .../cache/CacheMetricsMXBeanImpl.java           |    6 +-
 .../processors/cache/CacheMetricsSnapshot.java  |   14 +-
 .../internal/processors/cache/CacheObject.java  |    8 +-
 .../processors/cache/CacheObjectAdapter.java    |   21 +-
 .../cache/CacheObjectByteArrayImpl.java         |   24 +-
 .../processors/cache/CacheObjectContext.java    |   11 +-
 .../processors/cache/CacheObjectImpl.java       |    7 +-
 .../processors/cache/CacheOperationContext.java |   15 +-
 .../cache/CacheOsConflictResolutionManager.java |    8 +-
 .../CachePartialUpdateCheckedException.java     |   43 +-
 .../processors/cache/CachePeekModes.java        |    4 +-
 .../cache/CacheStoreBalancingWrapper.java       |   30 +-
 .../cache/CacheStorePartialUpdateException.java |    7 +-
 .../internal/processors/cache/CacheType.java    |    5 +-
 .../cache/CacheVersionedEntryImpl.java          |   80 -
 .../cache/CacheWeakQueryIteratorsHolder.java    |   21 +-
 .../cache/DynamicCacheChangeBatch.java          |   17 +-
 .../cache/DynamicCacheChangeRequest.java        |   31 +-
 .../cache/DynamicCacheDescriptor.java           |   32 +-
 .../processors/cache/GridCacheAdapter.java      |  497 ++-
 .../cache/GridCacheAffinityManager.java         |   31 +-
 .../processors/cache/GridCacheAtomicFuture.java |   14 +-
 .../cache/GridCacheAtomicVersionComparator.java |    4 +-
 .../processors/cache/GridCacheAttributes.java   |   30 +-
 .../cache/GridCacheBatchSwapEntry.java          |   11 +-
 .../cache/GridCacheClearAllRunnable.java        |   18 +-
 .../cache/GridCacheConcurrentMap.java           |   60 +-
 .../processors/cache/GridCacheContext.java      |  142 +-
 .../GridCacheDefaultAffinityKeyMapper.java      |   29 +-
 .../cache/GridCacheDeploymentManager.java       |   68 +-
 .../processors/cache/GridCacheEntryEx.java      |   61 +-
 .../processors/cache/GridCacheEntryInfo.java    |   22 +-
 .../GridCacheEntryInfoCollectSwapListener.java  |   10 +-
 .../processors/cache/GridCacheEntrySet.java     |   22 +-
 .../processors/cache/GridCacheEventManager.java |   30 +-
 .../cache/GridCacheEvictionManager.java         |  143 +-
 .../cache/GridCacheEvictionRequest.java         |   30 +-
 .../cache/GridCacheEvictionResponse.java        |   24 +-
 .../cache/GridCacheExplicitLockSpan.java        |   28 +-
 .../cache/GridCacheFilterFailedException.java   |    4 +-
 .../processors/cache/GridCacheFuture.java       |   14 +-
 .../processors/cache/GridCacheGateway.java      |   29 +-
 .../cache/GridCacheIndexUpdateException.java    |    6 +-
 .../processors/cache/GridCacheInternal.java     |    2 +-
 .../processors/cache/GridCacheIoManager.java    |   95 +-
 .../processors/cache/GridCacheIterator.java     |   14 +-
 .../processors/cache/GridCacheKeySet.java       |   18 +-
 .../cache/GridCacheLoaderWriterStore.java       |   22 +-
 .../GridCacheLoaderWriterStoreFactory.java      |   28 +-
 .../processors/cache/GridCacheLogger.java       |   23 +-
 .../processors/cache/GridCacheManager.java      |    6 +-
 .../cache/GridCacheManagerAdapter.java          |   12 +-
 .../processors/cache/GridCacheMapEntry.java     |  180 +-
 .../cache/GridCacheMapEntryFactory.java         |    6 +-
 .../processors/cache/GridCacheMessage.java      |   39 +-
 .../cache/GridCacheMultiTxFuture.java           |   25 +-
 .../processors/cache/GridCacheMvcc.java         |   33 +-
 .../processors/cache/GridCacheMvccCallback.java |    4 +-
 .../cache/GridCacheMvccCandidate.java           |   53 +-
 .../processors/cache/GridCacheMvccFuture.java   |    2 +-
 .../processors/cache/GridCacheMvccManager.java  |  140 +-
 .../cache/GridCacheOffheapSwapEntry.java        |   15 +-
 .../GridCachePartitionExchangeManager.java      |  105 +-
 .../processors/cache/GridCachePreloader.java    |   18 +-
 .../cache/GridCachePreloaderAdapter.java        |   23 +-
 .../processors/cache/GridCacheProcessor.java    |  497 +--
 .../processors/cache/GridCacheProxyImpl.java    |   54 +-
 .../processors/cache/GridCacheReturn.java       |   37 +-
 .../cache/GridCacheSharedContext.java           |   98 +-
 .../cache/GridCacheSharedManager.java           |    6 +-
 .../cache/GridCacheSharedManagerAdapter.java    |   12 +-
 .../processors/cache/GridCacheSwapEntry.java    |    8 +-
 .../cache/GridCacheSwapEntryImpl.java           |   44 +-
 .../processors/cache/GridCacheSwapListener.java |    4 +-
 .../processors/cache/GridCacheSwapManager.java  |  347 +-
 .../cache/GridCacheTryPutFailedException.java   |    4 +-
 .../processors/cache/GridCacheTtlManager.java   |   24 +-
 .../cache/GridCacheUpdateAtomicResult.java      |   16 +-
 .../cache/GridCacheUpdateTxResult.java          |    8 +-
 .../processors/cache/GridCacheUtilityKey.java   |    4 +-
 .../processors/cache/GridCacheUtils.java        |  174 +-
 .../cache/GridCacheValueCollection.java         |   22 +-
 .../cache/GridLoadCacheCloseablePredicate.java  |   30 -
 .../cache/IgniteCacheExpiryPolicy.java          |   15 +-
 .../processors/cache/IgniteCacheFutureImpl.java |   17 +-
 .../processors/cache/IgniteCacheProxy.java      |  123 +-
 .../processors/cache/IgniteInternalCache.java   |   53 +-
 .../processors/cache/KeyCacheObject.java        |    2 +-
 .../processors/cache/KeyCacheObjectImpl.java    |    6 +-
 .../processors/cache/QueryCursorImpl.java       |   14 +-
 .../cache/affinity/GridCacheAffinityImpl.java   |   28 +-
 .../cache/affinity/GridCacheAffinityProxy.java  |   22 +-
 .../CacheDataStructuresManager.java             |   78 +-
 .../distributed/GridCacheCommittedTxInfo.java   |   22 +-
 .../distributed/GridCacheMappedVersion.java     |    6 +-
 .../distributed/GridCacheTtlUpdateRequest.java  |   33 +-
 .../distributed/GridCacheTxFinishSync.java      |   27 +-
 .../distributed/GridCacheTxRecoveryFuture.java  |   47 +-
 .../distributed/GridCacheTxRecoveryRequest.java |   20 +-
 .../GridCacheTxRecoveryResponse.java            |   18 +-
 .../distributed/GridDistributedBaseMessage.java |   34 +-
 .../GridDistributedCacheAdapter.java            |   72 +-
 .../distributed/GridDistributedCacheEntry.java  |   46 +-
 .../distributed/GridDistributedLockRequest.java |   42 +-
 .../GridDistributedLockResponse.java            |   38 +-
 .../GridDistributedTxFinishRequest.java         |   41 +-
 .../GridDistributedTxFinishResponse.java        |   20 +-
 .../distributed/GridDistributedTxMapping.java   |   34 +-
 .../GridDistributedTxPrepareRequest.java        |   50 +-
 .../GridDistributedTxPrepareResponse.java       |   23 +-
 .../GridDistributedTxRemoteAdapter.java         |   99 +-
 .../GridDistributedUnlockRequest.java           |   28 +-
 .../GridFutureRemapTimeoutObject.java           |   13 +-
 .../IgniteExternalizableExpiryPolicy.java       |   18 +-
 .../dht/GridClientPartitionTopology.java        |  172 +-
 .../dht/GridDhtAffinityAssignmentRequest.java   |   18 +-
 .../dht/GridDhtAffinityAssignmentResponse.java  |   31 +-
 .../dht/GridDhtAssignmentFetchFuture.java       |   37 +-
 .../cache/distributed/dht/GridDhtCache.java     |   15 +-
 .../distributed/dht/GridDhtCacheAdapter.java    |  121 +-
 .../distributed/dht/GridDhtCacheEntry.java      |   49 +-
 .../distributed/dht/GridDhtEmbeddedFuture.java  |   16 +-
 .../distributed/dht/GridDhtFinishedFuture.java  |   10 +-
 .../cache/distributed/dht/GridDhtFuture.java    |    7 +-
 .../cache/distributed/dht/GridDhtGetFuture.java |   64 +-
 .../dht/GridDhtInvalidPartitionException.java   |    2 +-
 .../distributed/dht/GridDhtLocalPartition.java  |  153 +-
 .../distributed/dht/GridDhtLockFuture.java      |   90 +-
 .../distributed/dht/GridDhtLockRequest.java     |   53 +-
 .../distributed/dht/GridDhtLockResponse.java    |   40 +-
 .../dht/GridDhtOffHeapCacheEntry.java           |    9 +-
 .../distributed/dht/GridDhtPartitionState.java  |    4 +-
 .../dht/GridDhtPartitionTopology.java           |   33 +-
 .../dht/GridDhtPartitionTopologyImpl.java       |   95 +-
 .../dht/GridDhtPartitionsReservation.java       |   16 +-
 .../distributed/dht/GridDhtTopologyFuture.java  |    8 +-
 .../dht/GridDhtTransactionalCacheAdapter.java   |  602 ++--
 .../distributed/dht/GridDhtTxFinishFuture.java  |   63 +-
 .../distributed/dht/GridDhtTxFinishRequest.java |  116 +-
 .../dht/GridDhtTxFinishResponse.java            |  109 +-
 .../cache/distributed/dht/GridDhtTxLocal.java   |   69 +-
 .../distributed/dht/GridDhtTxLocalAdapter.java  |  164 +-
 .../cache/distributed/dht/GridDhtTxMapping.java |   21 +-
 .../distributed/dht/GridDhtTxPrepareFuture.java |  327 +-
 .../dht/GridDhtTxPrepareRequest.java            |   47 +-
 .../dht/GridDhtTxPrepareResponse.java           |   84 +-
 .../cache/distributed/dht/GridDhtTxRemote.java  |   85 +-
 .../distributed/dht/GridDhtUnlockRequest.java   |   27 +-
 .../GridDhtUnreservedPartitionException.java    |   66 +
 .../distributed/dht/GridNoStorageCacheMap.java  |   20 +-
 .../dht/GridPartitionedGetFuture.java           |  209 +-
 .../cache/distributed/dht/GridReservable.java   |    2 +-
 .../dht/atomic/GridDhtAtomicCache.java          |  173 +-
 .../dht/atomic/GridDhtAtomicCacheEntry.java     |   14 +-
 .../GridDhtAtomicDeferredUpdateResponse.java    |   24 +-
 .../atomic/GridDhtAtomicOffHeapCacheEntry.java  |    9 +-
 .../dht/atomic/GridDhtAtomicUpdateFuture.java   |   66 +-
 .../dht/atomic/GridDhtAtomicUpdateRequest.java  |   48 +-
 .../dht/atomic/GridDhtAtomicUpdateResponse.java |   34 +-
 .../dht/atomic/GridNearAtomicUpdateFuture.java  | 1286 +++----
 .../dht/atomic/GridNearAtomicUpdateRequest.java |   60 +-
 .../atomic/GridNearAtomicUpdateResponse.java    |   43 +-
 .../dht/colocated/GridDhtColocatedCache.java    |   97 +-
 .../colocated/GridDhtColocatedCacheEntry.java   |   13 +-
 .../colocated/GridDhtColocatedLockFuture.java   |  110 +-
 .../GridDhtColocatedOffHeapCacheEntry.java      |    9 +-
 .../colocated/GridDhtDetachedCacheEntry.java    |   17 +-
 .../dht/preloader/GridDhtForceKeysFuture.java   |   70 +-
 .../dht/preloader/GridDhtForceKeysRequest.java  |   36 +-
 .../dht/preloader/GridDhtForceKeysResponse.java |   38 +-
 .../GridDhtPartitionDemandMessage.java          |   32 +-
 .../preloader/GridDhtPartitionDemandPool.java   |   93 +-
 .../preloader/GridDhtPartitionExchangeId.java   |   33 +-
 .../dht/preloader/GridDhtPartitionFullMap.java  |   16 +-
 .../dht/preloader/GridDhtPartitionMap.java      |   22 +-
 .../GridDhtPartitionSupplyMessage.java          |   37 +-
 .../preloader/GridDhtPartitionSupplyPool.java   |   56 +-
 .../GridDhtPartitionsAbstractMessage.java       |   20 +-
 .../GridDhtPartitionsExchangeFuture.java        |  225 +-
 .../preloader/GridDhtPartitionsFullMessage.java |   32 +-
 .../GridDhtPartitionsSingleMessage.java         |   29 +-
 .../GridDhtPartitionsSingleRequest.java         |   14 +-
 .../dht/preloader/GridDhtPreloader.java         |   92 +-
 .../preloader/GridDhtPreloaderAssignments.java  |   12 +-
 .../distributed/near/CacheVersionedValue.java   |   23 +-
 .../distributed/near/GridNearAtomicCache.java   |   81 +-
 .../distributed/near/GridNearCacheAdapter.java  |   87 +-
 .../near/GridNearCacheClearAllRunnable.java     |    9 +-
 .../distributed/near/GridNearCacheEntry.java    |   50 +-
 .../distributed/near/GridNearGetFuture.java     |  232 +-
 .../distributed/near/GridNearGetRequest.java    |   45 +-
 .../distributed/near/GridNearGetResponse.java   |   42 +-
 .../distributed/near/GridNearLockFuture.java    |  111 +-
 .../distributed/near/GridNearLockMapping.java   |   17 +-
 .../distributed/near/GridNearLockRequest.java   |   56 +-
 .../distributed/near/GridNearLockResponse.java  |   35 +-
 .../near/GridNearOffHeapCacheEntry.java         |    7 +-
 .../near/GridNearOptimisticTxPrepareFuture.java |  230 +-
 .../GridNearPessimisticTxPrepareFuture.java     |   70 +-
 .../near/GridNearTransactionalCache.java        |   73 +-
 .../near/GridNearTxFinishFuture.java            |  401 ++-
 .../near/GridNearTxFinishRequest.java           |   44 +-
 .../near/GridNearTxFinishResponse.java          |   28 +-
 .../cache/distributed/near/GridNearTxLocal.java |  157 +-
 .../near/GridNearTxPrepareFutureAdapter.java    |   47 +-
 .../near/GridNearTxPrepareRequest.java          |   39 +-
 .../near/GridNearTxPrepareResponse.java         |   50 +-
 .../distributed/near/GridNearTxRemote.java      |   79 +-
 .../distributed/near/GridNearUnlockRequest.java |   16 +-
 .../cache/dr/GridCacheDrExpirationInfo.java     |    8 +-
 .../processors/cache/dr/GridCacheDrInfo.java    |   15 +-
 .../processors/cache/dr/GridCacheDrManager.java |   16 +-
 .../cache/dr/GridOsCacheDrManager.java          |   18 +-
 .../extras/GridCacheAttributesEntryExtras.java  |    8 +-
 .../GridCacheAttributesMvccEntryExtras.java     |   10 +-
 ...dCacheAttributesMvccObsoleteEntryExtras.java |   10 +-
 ...cheAttributesMvccObsoleteTtlEntryExtras.java |   10 +-
 .../GridCacheAttributesMvccTtlEntryExtras.java  |   10 +-
 .../GridCacheAttributesObsoleteEntryExtras.java |    8 +-
 ...idCacheAttributesObsoleteTtlEntryExtras.java |    8 +-
 .../GridCacheAttributesTtlEntryExtras.java      |    8 +-
 .../cache/extras/GridCacheEntryExtras.java      |    8 +-
 .../extras/GridCacheEntryExtrasAdapter.java     |    6 +-
 .../cache/extras/GridCacheMvccEntryExtras.java  |    8 +-
 .../GridCacheMvccObsoleteEntryExtras.java       |    8 +-
 .../GridCacheMvccObsoleteTtlEntryExtras.java    |    8 +-
 .../extras/GridCacheMvccTtlEntryExtras.java     |    7 +-
 .../extras/GridCacheObsoleteEntryExtras.java    |    8 +-
 .../extras/GridCacheObsoleteTtlEntryExtras.java |    8 +-
 .../cache/extras/GridCacheTtlEntryExtras.java   |    8 +-
 .../cache/jta/CacheJtaManagerAdapter.java       |   11 +-
 .../cache/jta/CacheNoopJtaManager.java          |    8 +-
 .../processors/cache/local/GridLocalCache.java  |   43 +-
 .../cache/local/GridLocalCacheEntry.java        |   25 +-
 .../cache/local/GridLocalLockCallback.java      |    4 +-
 .../cache/local/GridLocalLockFuture.java        |   42 +-
 .../local/atomic/GridLocalAtomicCache.java      |  111 +-
 .../CacheDefaultPortableAffinityKeyMapper.java  |   51 +
 .../portable/CacheObjectPortableContext.java    |  186 +
 .../portable/CacheObjectPortableProcessor.java  |  103 +
 .../CacheObjectPortableProcessorImpl.java       | 1025 ++++++
 .../cache/portable/IgnitePortablesImpl.java     |  177 +
 .../cache/portable/PortableMetaDataKey.java     |   82 +
 .../processors/cache/portable/package-info.java |   22 +
 .../processors/cache/query/CacheQuery.java      |   18 +-
 .../CacheQueryCloseableScanBiPredicate.java     |   30 -
 .../cache/query/CacheQueryFuture.java           |   11 +-
 .../processors/cache/query/CacheQueryType.java  |    2 +-
 .../GridCacheDistributedFieldsQueryFuture.java  |   20 +-
 .../query/GridCacheDistributedQueryFuture.java  |   57 +-
 .../query/GridCacheDistributedQueryManager.java |   67 +-
 .../query/GridCacheLocalFieldsQueryFuture.java  |   17 +-
 .../cache/query/GridCacheLocalQueryFuture.java  |   28 +-
 .../cache/query/GridCacheLocalQueryManager.java |   17 +-
 .../cache/query/GridCacheQueryAdapter.java      |  270 +-
 .../cache/query/GridCacheQueryBean.java         |    9 +-
 .../cache/query/GridCacheQueryErrorFuture.java  |   13 +-
 .../query/GridCacheQueryFutureAdapter.java      |   70 +-
 .../cache/query/GridCacheQueryInfo.java         |   12 +-
 .../cache/query/GridCacheQueryManager.java      |  434 ++-
 .../query/GridCacheQueryMetadataAware.java      |    9 +-
 .../query/GridCacheQueryMetricsAdapter.java     |  137 +-
 .../cache/query/GridCacheQueryMetricsKey.java   |   14 +-
 .../cache/query/GridCacheQueryRequest.java      |   98 +-
 .../cache/query/GridCacheQueryResponse.java     |   36 +-
 .../query/GridCacheQueryResponseEntry.java      |   12 +-
 .../cache/query/GridCacheQueryType.java         |    4 +-
 .../cache/query/GridCacheSqlIndexMetadata.java  |    6 +-
 .../cache/query/GridCacheSqlMetadata.java       |   12 +-
 .../cache/query/GridCacheSqlQuery.java          |   33 +-
 .../cache/query/GridCacheTwoStepQuery.java      |   11 +-
 .../processors/cache/query/QueryCursorEx.java   |   11 +-
 .../continuous/CacheContinuousQueryEntry.java   |   31 +-
 .../continuous/CacheContinuousQueryEvent.java   |   14 +-
 .../CacheContinuousQueryFilterEx.java           |   31 -
 .../continuous/CacheContinuousQueryHandler.java |   60 +-
 .../CacheContinuousQueryListener.java           |    2 +-
 .../continuous/CacheContinuousQueryManager.java |   73 +-
 .../jdbc/GridCacheQueryJdbcMetadataTask.java    |   46 +-
 .../query/jdbc/GridCacheQueryJdbcTask.java      |   62 +-
 .../jdbc/GridCacheQueryJdbcValidationTask.java  |   25 +-
 .../processors/cache/store/CacheLocalStore.java |    8 +-
 .../cache/store/CacheOsStoreManager.java        |   37 +-
 .../cache/store/CacheStoreManager.java          |   34 +-
 .../store/GridCacheStoreManagerAdapter.java     |   74 +-
 .../cache/store/GridCacheWriteBehindStore.java  |   55 +-
 .../cache/transactions/IgniteInternalTx.java    |   42 +-
 .../transactions/IgniteTransactionsImpl.java    |   24 +-
 .../cache/transactions/IgniteTxAdapter.java     |  150 +-
 .../cache/transactions/IgniteTxEntry.java       |   97 +-
 .../cache/transactions/IgniteTxHandler.java     |  373 +-
 .../cache/transactions/IgniteTxKey.java         |   22 +-
 .../transactions/IgniteTxLocalAdapter.java      |  271 +-
 .../cache/transactions/IgniteTxLocalEx.java     |   26 +-
 .../cache/transactions/IgniteTxManager.java     |  163 +-
 .../cache/transactions/IgniteTxMap.java         |   23 +-
 .../cache/transactions/IgniteTxRemoteEx.java    |    7 +-
 .../transactions/TransactionMetricsAdapter.java |   13 +-
 .../cache/transactions/TransactionProxy.java    |    4 +-
 .../transactions/TransactionProxyImpl.java      |   38 +-
 .../cache/transactions/TxEntryValueHolder.java  |   33 +-
 .../version/CacheVersionConflictResolver.java   |    4 +-
 .../version/GridCachePlainVersionedEntry.java   |    6 +-
 .../version/GridCacheRawVersionedEntry.java     |   31 +-
 .../cache/version/GridCacheVersion.java         |   24 +-
 .../cache/version/GridCacheVersionAware.java    |   30 +
 .../GridCacheVersionConflictContext.java        |   10 +-
 .../cache/version/GridCacheVersionEx.java       |   15 +-
 .../cache/version/GridCacheVersionManager.java  |   31 +-
 .../cache/version/GridCacheVersionable.java     |    2 +-
 .../cache/version/GridCacheVersionedEntry.java  |    4 +-
 .../version/GridCacheVersionedEntryEx.java      |    2 +-
 .../cacheobject/IgniteCacheObjectProcessor.java |   20 +-
 .../IgniteCacheObjectProcessorImpl.java         |   48 +-
 .../clock/GridClockDeltaSnapshot.java           |   20 +-
 .../clock/GridClockDeltaSnapshotMessage.java    |   23 +-
 .../processors/clock/GridClockDeltaVersion.java |   18 +-
 .../processors/clock/GridClockMessage.java      |   10 +-
 .../processors/clock/GridClockServer.java       |   24 +-
 .../processors/clock/GridClockSource.java       |    2 +-
 .../clock/GridClockSyncProcessor.java           |   54 +-
 .../processors/clock/GridJvmClockSource.java    |    2 +-
 .../processors/closure/AffinityTask.java        |    4 +-
 .../processors/closure/GridClosurePolicy.java   |    4 +-
 .../closure/GridClosureProcessor.java           |   80 +-
 .../closure/GridPeerDeployAwareTaskAdapter.java |   10 +-
 .../processors/cluster/ClusterProcessor.java    |   12 +-
 .../continuous/AbstractContinuousMessage.java   |    9 +-
 .../continuous/GridContinuousHandler.java       |   14 +-
 .../continuous/GridContinuousMessage.java       |   32 +-
 .../continuous/GridContinuousMessageType.java   |    4 +-
 .../continuous/GridContinuousProcessor.java     |  174 +-
 .../processors/continuous/StartRequestData.java |   23 +-
 .../StartRoutineAckDiscoveryMessage.java        |   15 +-
 .../StartRoutineDiscoveryMessage.java           |   13 +-
 .../StopRoutineAckDiscoveryMessage.java         |   11 +-
 .../continuous/StopRoutineDiscoveryMessage.java |   11 +-
 .../datastreamer/DataStreamProcessor.java       |   48 +-
 .../datastreamer/DataStreamerCacheUpdaters.java |   20 +-
 .../datastreamer/DataStreamerEntry.java         |   21 +-
 .../datastreamer/DataStreamerFuture.java        |   10 +-
 .../datastreamer/DataStreamerImpl.java          |  125 +-
 .../datastreamer/DataStreamerRequest.java       |   32 +-
 .../datastreamer/DataStreamerResponse.java      |   13 +-
 .../datastreamer/DataStreamerUpdateJob.java     |   25 +-
 .../CacheDataStructuresCacheKey.java            |   10 +-
 .../CacheDataStructuresConfigurationKey.java    |   10 +-
 .../datastructures/DataStructuresProcessor.java |  201 +-
 .../GridAtomicCacheQueueImpl.java               |   20 +-
 .../GridCacheAnnotationHelper.java              |   23 +-
 .../datastructures/GridCacheAtomicLongEx.java   |    4 +-
 .../datastructures/GridCacheAtomicLongImpl.java |   39 +-
 .../GridCacheAtomicLongValue.java               |   12 +-
 .../GridCacheAtomicReferenceEx.java             |    4 +-
 .../GridCacheAtomicReferenceImpl.java           |   38 +-
 .../GridCacheAtomicReferenceValue.java          |   14 +-
 .../GridCacheAtomicSequenceEx.java              |    4 +-
 .../GridCacheAtomicSequenceImpl.java            |   52 +-
 .../GridCacheAtomicSequenceValue.java           |   12 +-
 .../GridCacheAtomicStampedEx.java               |    4 +-
 .../GridCacheAtomicStampedImpl.java             |   44 +-
 .../GridCacheAtomicStampedValue.java            |   18 +-
 .../GridCacheCountDownLatchEx.java              |    4 +-
 .../GridCacheCountDownLatchImpl.java            |   44 +-
 .../GridCacheCountDownLatchValue.java           |   12 +-
 .../datastructures/GridCacheInternalKey.java    |    4 +-
 .../GridCacheInternalKeyImpl.java               |   15 +-
 .../datastructures/GridCacheQueueAdapter.java   |   49 +-
 .../datastructures/GridCacheQueueHeader.java    |   24 +-
 .../datastructures/GridCacheQueueHeaderKey.java |   13 +-
 .../datastructures/GridCacheQueueItemKey.java   |   15 +-
 .../datastructures/GridCacheQueueProxy.java     |   33 +-
 .../datastructures/GridCacheRemovable.java      |    2 +-
 .../datastructures/GridCacheSetHeader.java      |   15 +-
 .../datastructures/GridCacheSetHeaderKey.java   |   13 +-
 .../datastructures/GridCacheSetImpl.java        |   57 +-
 .../datastructures/GridCacheSetItemKey.java     |   17 +-
 .../datastructures/GridCacheSetProxy.java       |   34 +-
 .../datastructures/GridSetQueryPredicate.java   |   19 +-
 .../GridTransactionalCacheQueueImpl.java        |   43 +-
 .../internal/processors/dr/GridDrType.java      |    2 +-
 .../dr/IgniteDrDataStreamerCacheUpdater.java    |   40 +-
 .../internal/processors/hadoop/Hadoop.java      |   12 +-
 .../processors/hadoop/HadoopFileBlock.java      |   16 +-
 .../processors/hadoop/HadoopInputSplit.java     |    4 +-
 .../internal/processors/hadoop/HadoopJob.java   |    8 +-
 .../internal/processors/hadoop/HadoopJobId.java |   14 +-
 .../processors/hadoop/HadoopJobInfo.java        |   10 +-
 .../processors/hadoop/HadoopJobPhase.java       |    2 +-
 .../processors/hadoop/HadoopJobProperty.java    |    4 +-
 .../processors/hadoop/HadoopJobStatus.java      |   11 +-
 .../processors/hadoop/HadoopMapReducePlan.java  |   10 +-
 .../hadoop/HadoopMapReducePlanner.java          |   11 +-
 .../processors/hadoop/HadoopNoopProcessor.java  |   13 +-
 .../processors/hadoop/HadoopPartitioner.java    |    2 +-
 .../hadoop/HadoopProcessorAdapter.java          |   13 +-
 .../processors/hadoop/HadoopSerialization.java  |   10 +-
 .../internal/processors/hadoop/HadoopTask.java  |    7 +-
 .../processors/hadoop/HadoopTaskContext.java    |   12 +-
 .../processors/hadoop/HadoopTaskInfo.java       |   12 +-
 .../processors/hadoop/HadoopTaskInput.java      |    7 +-
 .../processors/hadoop/HadoopTaskOutput.java     |    4 +-
 .../processors/hadoop/HadoopTaskType.java       |    4 +-
 .../hadoop/counter/HadoopCounter.java           |    2 +-
 .../hadoop/counter/HadoopCounterWriter.java     |    6 +-
 .../hadoop/counter/HadoopCounters.java          |    4 +-
 .../processors/igfs/IgfsAckMessage.java         |   22 +-
 .../internal/processors/igfs/IgfsAsyncImpl.java |   34 +-
 .../processors/igfs/IgfsAttributes.java         |   16 +-
 .../internal/processors/igfs/IgfsBlockKey.java  |   28 +-
 .../processors/igfs/IgfsBlockLocationImpl.java  |   28 +-
 .../processors/igfs/IgfsBlocksMessage.java      |   19 +-
 .../processors/igfs/IgfsClientSession.java      |   13 +-
 .../igfs/IgfsCommunicationMessage.java          |   17 +-
 .../internal/processors/igfs/IgfsContext.java   |   23 +-
 .../processors/igfs/IgfsDataManager.java        |  298 +-
 .../processors/igfs/IgfsDeleteMessage.java      |   24 +-
 .../processors/igfs/IgfsDeleteWorker.java       |   46 +-
 .../ignite/internal/processors/igfs/IgfsEx.java |   19 +-
 .../processors/igfs/IgfsFileAffinityRange.java  |   36 +-
 .../internal/processors/igfs/IgfsFileImpl.java  |   22 +-
 .../internal/processors/igfs/IgfsFileInfo.java  |   29 +-
 .../internal/processors/igfs/IgfsFileMap.java   |   28 +-
 .../processors/igfs/IgfsFileWorkerBatch.java    |   33 +-
 .../igfs/IgfsFragmentizerManager.java           |   75 +-
 .../igfs/IgfsFragmentizerRequest.java           |   23 +-
 .../igfs/IgfsFragmentizerResponse.java          |   14 +-
 .../processors/igfs/IgfsHandshakeResponse.java  |   10 +-
 .../internal/processors/igfs/IgfsHelper.java    |    6 +-
 .../processors/igfs/IgfsHelperImpl.java         |   14 +-
 .../internal/processors/igfs/IgfsImpl.java      |  134 +-
 .../processors/igfs/IgfsInputStreamAdapter.java |    9 +-
 .../igfs/IgfsInputStreamDescriptor.java         |    7 +-
 .../processors/igfs/IgfsInputStreamImpl.java    |   39 +-
 .../igfs/IgfsInvalidRangeException.java         |    4 +-
 .../processors/igfs/IgfsIpcHandler.java         |   45 +-
 .../internal/processors/igfs/IgfsJobImpl.java   |   24 +-
 .../processors/igfs/IgfsListingEntry.java       |   15 +-
 .../processors/igfs/IgfsLocalMetrics.java       |    8 +-
 .../internal/processors/igfs/IgfsManager.java   |    8 +-
 .../processors/igfs/IgfsMetaManager.java        |  142 +-
 .../processors/igfs/IgfsMetricsAdapter.java     |   12 +-
 .../processors/igfs/IgfsModeResolver.java       |   20 +-
 .../processors/igfs/IgfsNoopHelper.java         |    6 +-
 .../processors/igfs/IgfsNoopProcessor.java      |   23 +-
 .../igfs/IgfsOutputStreamAdapter.java           |   18 +-
 .../processors/igfs/IgfsOutputStreamImpl.java   |   42 +-
 .../internal/processors/igfs/IgfsPaths.java     |   21 +-
 .../internal/processors/igfs/IgfsProcessor.java |   75 +-
 .../processors/igfs/IgfsProcessorAdapter.java   |   22 +-
 .../processors/igfs/IgfsSamplingKey.java        |   15 +-
 .../igfs/IgfsSecondaryFileSystemImpl.java       |   19 +-
 .../IgfsSecondaryInputStreamDescriptor.java     |    4 +-
 .../IgfsSecondaryOutputStreamDescriptor.java    |    7 +-
 .../internal/processors/igfs/IgfsServer.java    |   47 +-
 .../processors/igfs/IgfsServerHandler.java      |   13 +-
 .../processors/igfs/IgfsServerManager.java      |   36 +-
 .../internal/processors/igfs/IgfsStatus.java    |    7 +-
 .../processors/igfs/IgfsSyncMessage.java        |   14 +-
 .../processors/igfs/IgfsTaskArgsImpl.java       |   18 +-
 .../internal/processors/igfs/IgfsThread.java    |    4 +-
 .../internal/processors/igfs/IgfsUtils.java     |   70 +-
 .../processors/job/GridJobEventListener.java    |    4 +-
 .../processors/job/GridJobHoldListener.java     |    4 +-
 .../processors/job/GridJobProcessor.java        |  112 +-
 .../internal/processors/job/GridJobWorker.java  |   75 +-
 .../processors/jobmetrics/GridJobMetrics.java   |    4 +-
 .../jobmetrics/GridJobMetricsProcessor.java     |   21 +-
 .../jobmetrics/GridJobMetricsSnapshot.java      |    5 +-
 .../DiscoveryNodeValidationProcessor.java       |    4 +-
 .../OsDiscoveryNodeValidationProcessor.java     |   19 +-
 .../offheap/GridOffHeapProcessor.java           |   59 +-
 .../platform/PlatformConfiguration.java         |   25 +
 .../processors/platform/PlatformContext.java    |  275 ++
 .../platform/PlatformEventFilterListener.java   |   39 +
 .../processors/platform/PlatformException.java  |   71 +
 .../platform/PlatformExtendedException.java     |   57 +
 .../platform/PlatformNativeException.java       |   77 +
 .../platform/PlatformNoCallbackException.java   |   50 +
 .../platform/PlatformNoopProcessor.java         |  125 +
 .../processors/platform/PlatformProcessor.java  |  172 +
 .../processors/platform/PlatformTarget.java     |  116 +
 .../cache/PlatformCacheEntryFilter.java         |   30 +
 .../cache/PlatformCacheEntryProcessor.java      |   27 +
 .../cache/query/PlatformContinuousQuery.java    |   57 +
 .../query/PlatformContinuousQueryFilter.java    |   30 +
 .../cache/store/PlatformCacheStore.java         |   25 +
 .../callback/PlatformCallbackGateway.java       |  943 +++++
 .../callback/PlatformCallbackUtils.java         |  490 +++
 .../cluster/PlatformClusterNodeFilter.java      |   28 +
 .../platform/compute/PlatformJob.java           |   39 +
 .../datastreamer/PlatformStreamReceiver.java    |   27 +
 .../dotnet/PlatformDotNetConfiguration.java     |  119 +
 .../PlatformDotNetPortableConfiguration.java    |  228 ++
 ...PlatformDotNetPortableTypeConfiguration.java |  248 ++
 .../platform/memory/PlatformInputStream.java    |   30 +
 .../platform/memory/PlatformMemory.java         |   77 +
 .../platform/memory/PlatformMemoryManager.java  |   46 +
 .../platform/memory/PlatformOutputStream.java   |   30 +
 .../platform/message/PlatformMessageFilter.java |   40 +
 .../platform/services/PlatformService.java      |   44 +
 .../processors/plugin/CachePluginManager.java   |   33 +-
 .../plugin/IgnitePluginProcessor.java           |   37 +-
 .../processors/port/GridPortListener.java       |    4 +-
 .../processors/port/GridPortProcessor.java      |   21 +-
 .../processors/port/GridPortRecord.java         |    6 +-
 .../portable/GridPortableInputStream.java       |  168 -
 .../portable/GridPortableOutputStream.java      |  165 -
 .../processors/portable/GridPortableStream.java |   53 -
 .../processors/portable/package-info.java       |   22 -
 .../query/GridQueryCacheObjectsIterator.java    |    9 +-
 .../query/GridQueryFieldMetadata.java           |    4 +-
 .../processors/query/GridQueryFieldsResult.java |    7 +-
 .../query/GridQueryFieldsResultAdapter.java     |   10 +-
 .../query/GridQueryIndexDescriptor.java         |    4 +-
 .../processors/query/GridQueryIndexType.java    |    2 +-
 .../processors/query/GridQueryIndexing.java     |   36 +-
 .../processors/query/GridQueryProcessor.java    |  149 +-
 .../query/GridQueryTypeDescriptor.java          |    7 +-
 .../messages/GridQueryCancelRequest.java        |   13 +-
 .../twostep/messages/GridQueryFailResponse.java |   13 +-
 .../messages/GridQueryNextPageRequest.java      |   13 +-
 .../messages/GridQueryNextPageResponse.java     |   25 +-
 .../h2/twostep/messages/GridQueryRequest.java   |   31 +-
 .../resource/GridNoImplicitInjection.java       |    2 +-
 .../resource/GridResourceBasicInjector.java     |    8 +-
 .../processors/resource/GridResourceField.java  |   13 +-
 .../resource/GridResourceInjector.java          |    6 +-
 .../processors/resource/GridResourceIoc.java    |   34 +-
 .../GridResourceJobContextInjector.java         |    9 +-
 .../resource/GridResourceLoggerInjector.java    |    9 +-
 .../processors/resource/GridResourceMethod.java |   11 +-
 .../resource/GridResourceProcessor.java         |   54 +-
 .../resource/GridResourceServiceInjector.java   |   14 +-
 .../processors/resource/GridResourceUtils.java  |   13 +-
 .../resource/GridSpringResourceContext.java     |    5 +-
 .../processors/rest/GridRestCommand.java        |    8 +-
 .../processors/rest/GridRestProcessor.java      |   84 +-
 .../processors/rest/GridRestProtocol.java       |    9 +-
 .../rest/GridRestProtocolHandler.java           |    8 +-
 .../processors/rest/GridRestResponse.java       |   15 +-
 .../message/GridClientAbstractMessage.java      |   13 +-
 .../GridClientAuthenticationRequest.java        |    9 +-
 .../client/message/GridClientCacheRequest.java  |   15 +-
 .../message/GridClientHandshakeRequest.java     |    7 +-
 .../message/GridClientHandshakeResponse.java    |    4 +-
 .../rest/client/message/GridClientMessage.java  |    6 +-
 .../rest/client/message/GridClientNodeBean.java |   15 +-
 .../message/GridClientNodeMetricsBean.java      |    7 +-
 .../client/message/GridClientPingPacket.java    |    4 +-
 .../message/GridClientPortableMetaData.java     |    7 +-
 .../rest/client/message/GridClientResponse.java |    9 +-
 .../client/message/GridClientTaskRequest.java   |    9 +-
 .../message/GridClientTaskResultBean.java       |   10 +-
 .../message/GridClientTopologyRequest.java      |   11 +-
 .../rest/client/message/GridRouterRequest.java  |    9 +-
 .../rest/client/message/GridRouterResponse.java |    9 +-
 .../rest/handlers/GridRestCommandHandler.java   |   12 +-
 .../handlers/GridRestCommandHandlerAdapter.java |    6 +-
 .../handlers/cache/GridCacheCommandHandler.java |  103 +-
 .../handlers/cache/GridCacheRestMetrics.java    |    7 +-
 .../handlers/cache/GridCacheRestResponse.java   |   12 +-
 .../DataStructuresCommandHandler.java           |   32 +-
 .../query/CacheQueryFieldsMetaResult.java       |   12 +-
 .../rest/handlers/query/CacheQueryResult.java   |   13 +-
 .../handlers/query/QueryCommandHandler.java     |   48 +-
 .../handlers/task/GridTaskCommandHandler.java   |  102 +-
 .../handlers/task/GridTaskResultRequest.java    |   17 +-
 .../handlers/task/GridTaskResultResponse.java   |   15 +-
 .../top/GridTopologyCommandHandler.java         |   61 +-
 .../version/GridVersionCommandHandler.java      |   24 +-
 .../rest/protocols/GridRestProtocolAdapter.java |   36 +-
 .../protocols/tcp/GridClientPacketType.java     |    2 +-
 .../protocols/tcp/GridMemcachedMessage.java     |   12 +-
 .../tcp/GridTcpMemcachedNioListener.java        |   65 +-
 .../protocols/tcp/GridTcpRestNioListener.java   |   83 +-
 .../rest/protocols/tcp/GridTcpRestParser.java   |   62 +-
 .../rest/protocols/tcp/GridTcpRestProtocol.java |   63 +-
 .../rest/request/DataStructuresRequest.java     |    2 +-
 .../rest/request/GridRestCacheRequest.java      |    7 +-
 .../rest/request/GridRestLogRequest.java        |    4 +-
 .../rest/request/GridRestRequest.java           |   13 +-
 .../rest/request/GridRestTaskRequest.java       |    7 +-
 .../rest/request/GridRestTopologyRequest.java   |    7 +-
 .../rest/request/RestSqlQueryRequest.java       |    2 +-
 .../schedule/GridScheduleStatistics.java        |    5 +-
 .../schedule/IgniteNoopScheduleProcessor.java   |   11 +-
 .../IgniteScheduleProcessorAdapter.java         |   11 +-
 .../security/GridSecurityProcessor.java         |   19 +-
 .../processors/security/SecurityContext.java    |    5 +-
 .../security/os/GridOsSecurityProcessor.java    |   25 +-
 .../segmentation/GridSegmentationProcessor.java |    4 +-
 .../os/GridOsSegmentationProcessor.java         |    8 +-
 .../service/GridServiceAssignments.java         |   22 +-
 .../service/GridServiceAssignmentsKey.java      |    6 +-
 .../service/GridServiceDeployment.java          |   13 +-
 .../service/GridServiceDeploymentFuture.java    |    8 +-
 .../service/GridServiceDeploymentKey.java       |    6 +-
 .../GridServiceMethodNotFoundException.java     |    7 +-
 .../service/GridServiceMethodReflectKey.java    |    4 +-
 .../service/GridServiceNotFoundException.java   |    4 +-
 .../service/GridServiceProcessor.java           |  110 +-
 .../processors/service/GridServiceProxy.java    |   54 +-
 .../processors/service/ServiceContextImpl.java  |   20 +-
 .../service/ServiceDescriptorImpl.java          |   15 +-
 .../session/GridTaskSessionProcessor.java       |   28 +-
 .../internal/processors/task/GridInternal.java  |    8 +-
 .../processors/task/GridTaskEventListener.java  |    8 +-
 .../processors/task/GridTaskProcessor.java      |   99 +-
 .../task/GridTaskThreadContextKey.java          |    2 +-
 .../processors/task/GridTaskWorker.java         |  113 +-
 .../timeout/GridSpiTimeoutObject.java           |   10 +-
 .../processors/timeout/GridTimeoutObject.java   |    4 +-
 .../timeout/GridTimeoutObjectAdapter.java       |    6 +-
 .../timeout/GridTimeoutProcessor.java           |   29 +-
 .../internal/tck/TCKMBeanServerBuilder.java     |   14 +-
 .../IgniteTxHeuristicCheckedException.java      |    4 +-
 .../IgniteTxOptimisticCheckedException.java     |    4 +-
 .../IgniteTxRollbackCheckedException.java       |    4 +-
 .../IgniteTxTimeoutCheckedException.java        |    4 +-
 .../org/apache/ignite/internal/util/F0.java     |   29 +-
 .../internal/util/GridAnnotationsCache.java     |   17 +-
 .../ignite/internal/util/GridArgumentCheck.java |   24 +-
 .../internal/util/GridAtomicInitializer.java    |   12 +-
 .../ignite/internal/util/GridAtomicInteger.java |    7 +-
 .../ignite/internal/util/GridAtomicLong.java    |    7 +-
 .../GridBoundedConcurrentLinkedHashMap.java     |    8 +-
 .../GridBoundedConcurrentLinkedHashSet.java     |   18 +-
 .../util/GridBoundedConcurrentOrderedMap.java   |   18 +-
 .../util/GridBoundedConcurrentOrderedSet.java   |    9 +-
 .../internal/util/GridBoundedLinkedHashMap.java |    5 +-
 .../internal/util/GridBoundedLinkedHashSet.java |    7 +-
 .../ignite/internal/util/GridBusyLock.java      |    7 +-
 .../ignite/internal/util/GridByteArrayList.java |   28 +-
 .../internal/util/GridCircularBuffer.java       |   23 +-
 .../internal/util/GridClassLoaderCache.java     |   13 +-
 .../internal/util/GridClientByteUtils.java      |    7 +-
 .../util/GridCloseableIteratorAdapter.java      |   10 +-
 .../util/GridCloseableIteratorAdapterEx.java    |   12 +-
 .../ignite/internal/util/GridCollections.java   |   14 +-
 .../internal/util/GridConcurrentFactory.java    |   11 +-
 .../internal/util/GridConcurrentHashSet.java    |   13 +-
 .../util/GridConcurrentLinkedHashSet.java       |   16 +-
 .../util/GridConcurrentPhantomHashSet.java      |   25 +-
 .../util/GridConcurrentSkipListSet.java         |   22 +-
 .../util/GridConcurrentWeakHashSet.java         |   25 +-
 .../internal/util/GridConfigurationFinder.java  |   19 +-
 .../internal/util/GridConsistentHash.java       |   32 +-
 .../apache/ignite/internal/util/GridDebug.java  |   33 +-
 .../util/GridEmptyCloseableIterator.java        |    6 +-
 .../ignite/internal/util/GridEmptyIterator.java |    9 +-
 .../ignite/internal/util/GridEnumCache.java     |    7 +-
 .../internal/util/GridExecutionStatistics.java  |   19 +-
 .../internal/util/GridFixedSizeInputStream.java |    5 +-
 .../ignite/internal/util/GridHandleTable.java   |    7 +-
 .../ignite/internal/util/GridJavaProcess.java   |   25 +-
 .../ignite/internal/util/GridKeyLock.java       |   11 +-
 .../internal/util/GridLeanIdentitySet.java      |    7 +-
 .../ignite/internal/util/GridLeanMap.java       |   19 +-
 .../ignite/internal/util/GridLeanSet.java       |   10 +-
 .../ignite/internal/util/GridListSet.java       |   19 +-
 .../ignite/internal/util/GridLogThrottle.java   |   17 +-
 .../ignite/internal/util/GridLongList.java      |   28 +-
 .../apache/ignite/internal/util/GridMutex.java  |   10 +-
 .../apache/ignite/internal/util/GridQueue.java  |   16 +-
 .../apache/ignite/internal/util/GridRandom.java |    6 +-
 .../internal/util/GridReflectionCache.java      |   29 +-
 .../util/GridSerializableCollection.java        |    6 +-
 .../internal/util/GridSerializableIterable.java |    4 +-
 .../internal/util/GridSerializableIterator.java |    6 +-
 .../internal/util/GridSerializableList.java     |    6 +-
 .../internal/util/GridSerializableMap.java      |    6 +-
 .../internal/util/GridSerializableSet.java      |    7 +-
 .../ignite/internal/util/GridSetWrapper.java    |   12 +-
 .../ignite/internal/util/GridSnapshotLock.java  |    7 +-
 .../util/GridSpiCloseableIteratorWrapper.java   |   13 +-
 .../ignite/internal/util/GridSpinBusyLock.java  |    4 +-
 .../internal/util/GridSpinReadWriteLock.java    |   12 +-
 .../ignite/internal/util/GridStringBuilder.java |    7 +-
 .../internal/util/GridStringBuilderFactory.java |    4 +-
 .../ignite/internal/util/GridStripedLock.java   |   11 +-
 .../internal/util/GridStripedReadWriteLock.java |   14 +-
 .../internal/util/GridStripedSpinBusyLock.java  |  127 +
 .../internal/util/GridSynchronizedMap.java      |   12 +-
 .../apache/ignite/internal/util/GridTimer.java  |   10 +-
 .../ignite/internal/util/GridTimerTask.java     |    4 +-
 .../apache/ignite/internal/util/GridUnsafe.java |   11 +-
 .../ignite/internal/util/GridWeakIterator.java  |   10 +-
 .../internal/util/IgniteExceptionRegistry.java  |   27 +-
 .../ignite/internal/util/IgniteUtils.java       |  306 +-
 .../ignite/internal/util/IgniteUuidCache.java   |    6 +-
 .../util/future/GridCompoundFuture.java         |   37 +-
 .../util/future/GridCompoundIdentityFuture.java |   10 +-
 .../util/future/GridEmbeddedFuture.java         |   71 +-
 .../util/future/GridFinishedFuture.java         |   22 +-
 .../internal/util/future/GridFutureAdapter.java |   85 +-
 .../util/future/GridFutureChainListener.java    |    9 +-
 .../util/future/IgniteFinishedFutureImpl.java   |    4 +-
 .../internal/util/future/IgniteFutureImpl.java  |   21 +-
 .../util/gridify/GridifyArgumentBuilder.java    |   20 +-
 .../util/gridify/GridifyJobAdapter.java         |   13 +-
 .../util/gridify/GridifyRangeArgument.java      |    9 +-
 .../internal/util/gridify/GridifyUtils.java     |   35 +-
 .../util/io/GridByteArrayInputStream.java       |    9 +-
 .../util/io/GridByteArrayOutputStream.java      |   13 +-
 .../ignite/internal/util/io/GridDataInput.java  |    6 +-
 .../ignite/internal/util/io/GridDataOutput.java |    6 +-
 .../internal/util/io/GridFilenameUtils.java     |   10 +-
 .../util/io/GridReversedLinesFileReader.java    |   12 +-
 .../internal/util/io/GridUnsafeDataInput.java   |   22 +-
 .../internal/util/io/GridUnsafeDataOutput.java  |   15 +-
 .../ignite/internal/util/ipc/IpcEndpoint.java   |    9 +-
 .../util/ipc/IpcEndpointBindException.java      |    4 +-
 .../internal/util/ipc/IpcEndpointFactory.java   |   13 +-
 .../internal/util/ipc/IpcEndpointType.java      |    2 +-
 .../internal/util/ipc/IpcServerEndpoint.java    |    9 +-
 .../internal/util/ipc/IpcToNioAdapter.java      |   31 +-
 .../util/ipc/loopback/IpcClientTcpEndpoint.java |   16 +-
 .../util/ipc/loopback/IpcServerTcpEndpoint.java |   23 +-
 .../shmem/IpcOutOfSystemResourcesException.java |    6 +-
 .../shmem/IpcSharedMemoryClientEndpoint.java    |   21 +-
 .../ipc/shmem/IpcSharedMemoryInitRequest.java   |    7 +-
 .../ipc/shmem/IpcSharedMemoryInitResponse.java  |   12 +-
 .../ipc/shmem/IpcSharedMemoryInputStream.java   |   10 +-
 .../ipc/shmem/IpcSharedMemoryNativeLoader.java  |   35 +-
 ...cSharedMemoryOperationTimedoutException.java |    6 +-
 .../ipc/shmem/IpcSharedMemoryOutputStream.java  |   10 +-
 .../shmem/IpcSharedMemoryServerEndpoint.java    |   58 +-
 .../util/ipc/shmem/IpcSharedMemorySpace.java    |   19 +-
 .../util/ipc/shmem/IpcSharedMemoryUtils.java    |   18 +-
 .../internal/util/lang/GridAbsClosure.java      |    4 +-
 .../internal/util/lang/GridAbsClosureX.java     |    6 +-
 .../internal/util/lang/GridAbsPredicate.java    |    4 +-
 .../internal/util/lang/GridAbsPredicateX.java   |    6 +-
 .../util/lang/GridCloseableIterator.java        |    6 +-
 .../ignite/internal/util/lang/GridClosure3.java |    4 +-
 .../internal/util/lang/GridClosure3X.java       |    7 +-
 .../util/lang/GridClosureException.java         |    5 +-
 .../util/lang/GridFilteredIterator.java         |    5 +-
 .../ignite/internal/util/lang/GridFunc.java     |   91 +-
 .../internal/util/lang/GridInClosure3.java      |    5 +-
 .../ignite/internal/util/lang/GridIterable.java |    4 +-
 .../internal/util/lang/GridIterableAdapter.java |   10 +-
 .../ignite/internal/util/lang/GridIterator.java |   10 +-
 .../internal/util/lang/GridIteratorAdapter.java |    8 +-
 .../ignite/internal/util/lang/GridMapEntry.java |   13 +-
 .../util/lang/GridMetadataAwareAdapter.java     |   18 +-
 .../internal/util/lang/GridNodePredicate.java   |   24 +-
 .../internal/util/lang/GridPeerDeployAware.java |    4 +-
 .../util/lang/GridPeerDeployAwareAdapter.java   |    4 +-
 .../internal/util/lang/GridPlainAbsClosure.java |    4 +-
 .../internal/util/lang/GridPlainCallable.java   |    4 +-
 .../internal/util/lang/GridPlainClosure.java    |    4 +-
 .../internal/util/lang/GridPlainClosure2.java   |    4 +-
 .../internal/util/lang/GridPlainInClosure.java  |    4 +-
 .../internal/util/lang/GridPlainOutClosure.java |    4 +-
 .../internal/util/lang/GridPlainRunnable.java   |    2 +-
 .../internal/util/lang/GridPredicate3.java      |    4 +-
 .../internal/util/lang/GridPredicate3X.java     |    7 +-
 .../ignite/internal/util/lang/GridTriple.java   |    7 +-
 .../ignite/internal/util/lang/GridTuple.java    |   17 +-
 .../ignite/internal/util/lang/GridTuple3.java   |   19 +-
 .../ignite/internal/util/lang/GridTuple4.java   |   19 +-
 .../ignite/internal/util/lang/GridTuple5.java   |   19 +-
 .../ignite/internal/util/lang/GridTuple6.java   |   19 +-
 .../ignite/internal/util/lang/GridTupleV.java   |   18 +-
 .../internal/util/lang/IgniteClosure2X.java     |    9 +-
 .../internal/util/lang/IgniteClosureX.java      |    9 +-
 .../internal/util/lang/IgniteInClosure2X.java   |    9 +-
 .../internal/util/lang/IgniteInClosureX.java    |    9 +-
 .../internal/util/lang/IgniteOutClosureX.java   |    8 +-
 .../ignite/internal/util/lang/IgnitePair.java   |    9 +-
 .../internal/util/lang/IgnitePredicate2X.java   |    9 +-
 .../internal/util/lang/IgnitePredicateX.java    |    9 +-
 .../internal/util/lang/IgniteReducer2.java      |    6 +-
 .../internal/util/lang/IgniteReducer2X.java     |    7 +-
 .../internal/util/lang/IgniteReducer3.java      |    6 +-
 .../internal/util/lang/IgniteReducer3X.java     |    7 +-
 .../internal/util/lang/IgniteReducerX.java      |    9 +-
 .../nio/GridAbstractCommunicationClient.java    |   10 +-
 .../internal/util/nio/GridBufferedParser.java   |   10 +-
 .../util/nio/GridCommunicationClient.java       |   21 +-
 .../nio/GridConnectionBytesVerifyFilter.java    |   11 +-
 .../internal/util/nio/GridDelimitedParser.java  |    9 +-
 .../internal/util/nio/GridDirectParser.java     |   19 +-
 .../util/nio/GridNioAsyncNotifyFilter.java      |   13 +-
 .../util/nio/GridNioBackPressureControl.java    |    4 +-
 .../internal/util/nio/GridNioCodecFilter.java   |   15 +-
 .../util/nio/GridNioDelimitedBuffer.java        |    9 +-
 .../util/nio/GridNioEmbeddedFuture.java         |   12 +-
 .../internal/util/nio/GridNioException.java     |    6 +-
 .../ignite/internal/util/nio/GridNioFilter.java |    4 +-
 .../internal/util/nio/GridNioFilterAdapter.java |    4 +-
 .../internal/util/nio/GridNioFilterChain.java   |    7 +-
 .../util/nio/GridNioFinishedFuture.java         |   10 +-
 .../ignite/internal/util/nio/GridNioFuture.java |    8 +-
 .../internal/util/nio/GridNioFutureImpl.java    |   10 +-
 .../internal/util/nio/GridNioMessageReader.java |   12 +-
 .../util/nio/GridNioMessageTracker.java         |   12 +-
 .../internal/util/nio/GridNioMessageWriter.java |   14 +-
 .../util/nio/GridNioMetricsListener.java        |    2 +-
 .../ignite/internal/util/nio/GridNioParser.java |   11 +-
 .../util/nio/GridNioRecoveryDescriptor.java     |   20 +-
 .../ignite/internal/util/nio/GridNioServer.java |   86 +-
 .../internal/util/nio/GridNioServerBuffer.java  |    9 +-
 .../util/nio/GridNioServerListener.java         |    4 +-
 .../util/nio/GridNioServerListenerAdapter.java  |    2 +-
 .../internal/util/nio/GridNioSession.java       |    7 +-
 .../internal/util/nio/GridNioSessionImpl.java   |   16 +-
 .../util/nio/GridNioSessionMetaKey.java         |    4 +-
 .../util/nio/GridSelectorNioSessionImpl.java    |   26 +-
 .../util/nio/GridShmemCommunicationClient.java  |   30 +-
 .../util/nio/GridTcpNioCommunicationClient.java |   30 +-
 .../util/nio/ssl/BlockingSslHandler.java        |   37 +-
 .../internal/util/nio/ssl/GridNioSslFilter.java |   30 +-
 .../util/nio/ssl/GridNioSslHandler.java         |   44 +-
 .../util/nodestart/IgniteNodeStartUtils.java    |   30 +-
 .../IgniteRemoteStartSpecification.java         |   11 +-
 .../util/nodestart/IgniteSshHelper.java         |    4 +-
 .../util/nodestart/StartNodeCallable.java       |    7 +-
 .../internal/util/offheap/GridOffHeapEvent.java |    4 +-
 .../util/offheap/GridOffHeapEventListener.java  |    2 +-
 .../util/offheap/GridOffHeapEvictListener.java  |    7 +-
 .../internal/util/offheap/GridOffHeapMap.java   |   24 +-
 .../util/offheap/GridOffHeapMapFactory.java     |   35 +-
 .../GridOffHeapOutOfMemoryException.java        |    2 +-
 .../util/offheap/GridOffHeapPartitionedMap.java |   26 +-
 .../offheap/unsafe/GridOffHeapSmartPointer.java |    2 +-
 .../unsafe/GridOffHeapSmartPointerFactory.java  |    2 +-
 .../offheap/unsafe/GridOffHeapSnapTreeMap.java  |   33 +-
 .../unsafe/GridUnsafeCompoundMemory.java        |    2 +-
 .../util/offheap/unsafe/GridUnsafeGuard.java    |   12 +-
 .../util/offheap/unsafe/GridUnsafeLru.java      |   15 +-
 .../offheap/unsafe/GridUnsafeLruPoller.java     |    2 +-
 .../util/offheap/unsafe/GridUnsafeMap.java      |  164 +-
 .../util/offheap/unsafe/GridUnsafeMemory.java   |   27 +-
 .../unsafe/GridUnsafePartitionedMap.java        |   35 +-
 .../util/portscanner/GridJmxPortFinder.java     |   15 +-
 .../util/snaptree/CopyOnWriteManager.java       |    6 +-
 .../ignite/internal/util/snaptree/Epoch.java    |    2 +-
 .../internal/util/snaptree/EpochNode.java       |    5 +-
 .../internal/util/snaptree/SnapTreeMap.java     |   22 +-
 .../util/spring/IgniteSpringHelper.java         |   19 +-
 .../internal/util/test/GridTestPrintStream.java |   11 +-
 .../util/test/GridTestPrintStreamFactory.java   |    4 +-
 .../util/tostring/GridToStringBuilder.java      |   33 +-
 .../tostring/GridToStringClassDescriptor.java   |    7 +-
 .../util/tostring/GridToStringExclude.java      |    8 +-
 .../tostring/GridToStringFieldDescriptor.java   |    2 +-
 .../util/tostring/GridToStringInclude.java      |    8 +-
 .../util/tostring/GridToStringOrder.java        |    8 +-
 .../util/tostring/GridToStringThreadLocal.java  |    4 +-
 .../apache/ignite/internal/util/typedef/C1.java |    6 +-
 .../apache/ignite/internal/util/typedef/C2.java |    6 +-
 .../apache/ignite/internal/util/typedef/C3.java |    5 +-
 .../apache/ignite/internal/util/typedef/CA.java |    5 +-
 .../ignite/internal/util/typedef/CAX.java       |    5 +-
 .../ignite/internal/util/typedef/CI1.java       |    6 +-
 .../ignite/internal/util/typedef/CI2.java       |    6 +-
 .../ignite/internal/util/typedef/CIX1.java      |    5 +-
 .../ignite/internal/util/typedef/CIX2.java      |    5 +-
 .../ignite/internal/util/typedef/CIX3.java      |    5 +-
 .../apache/ignite/internal/util/typedef/CO.java |    6 +-
 .../ignite/internal/util/typedef/COX.java       |    5 +-
 .../ignite/internal/util/typedef/CX1.java       |    5 +-
 .../ignite/internal/util/typedef/CX2.java       |    5 +-
 .../ignite/internal/util/typedef/CX3.java       |    5 +-
 .../apache/ignite/internal/util/typedef/F.java  |    4 +-
 .../apache/ignite/internal/util/typedef/G.java  |    4 +-
 .../apache/ignite/internal/util/typedef/P1.java |    6 +-
 .../apache/ignite/internal/util/typedef/P2.java |    6 +-
 .../apache/ignite/internal/util/typedef/P3.java |    5 +-
 .../apache/ignite/internal/util/typedef/PA.java |    5 +-
 .../ignite/internal/util/typedef/PAX.java       |    6 +-
 .../ignite/internal/util/typedef/PCE.java       |    9 +-
 .../apache/ignite/internal/util/typedef/PE.java |    8 +-
 .../ignite/internal/util/typedef/PKV.java       |    6 +-
 .../apache/ignite/internal/util/typedef/PN.java |    8 +-
 .../ignite/internal/util/typedef/PX1.java       |    5 +-
 .../ignite/internal/util/typedef/PX2.java       |    5 +-
 .../ignite/internal/util/typedef/PX3.java       |    5 +-
 .../apache/ignite/internal/util/typedef/R1.java |    6 +-
 .../apache/ignite/internal/util/typedef/R2.java |    5 +-
 .../apache/ignite/internal/util/typedef/R3.java |    5 +-
 .../ignite/internal/util/typedef/RX1.java       |    5 +-
 .../ignite/internal/util/typedef/RX2.java       |    5 +-
 .../ignite/internal/util/typedef/RX3.java       |    5 +-
 .../apache/ignite/internal/util/typedef/T1.java |    8 +-
 .../apache/ignite/internal/util/typedef/T2.java |   10 +-
 .../apache/ignite/internal/util/typedef/T3.java |    9 +-
 .../apache/ignite/internal/util/typedef/T4.java |    9 +-
 .../apache/ignite/internal/util/typedef/T5.java |    9 +-
 .../apache/ignite/internal/util/typedef/T6.java |    9 +-
 .../apache/ignite/internal/util/typedef/X.java  |   38 +-
 .../internal/util/typedef/internal/A.java       |    4 +-
 .../internal/util/typedef/internal/CU.java      |    4 +-
 .../internal/util/typedef/internal/D.java       |    4 +-
 .../internal/util/typedef/internal/GPC.java     |    4 +-
 .../internal/util/typedef/internal/GPR.java     |    4 +-
 .../internal/util/typedef/internal/LT.java      |    4 +-
 .../internal/util/typedef/internal/S.java       |    4 +-
 .../internal/util/typedef/internal/SB.java      |    4 +-
 .../internal/util/typedef/internal/U.java       |    4 +-
 .../ignite/internal/util/worker/GridWorker.java |   18 +-
 .../internal/util/worker/GridWorkerFuture.java  |    6 +-
 .../util/worker/GridWorkerListener.java         |    4 +-
 .../util/worker/GridWorkerListenerAdapter.java  |    2 +-
 .../internal/util/worker/GridWorkerPool.java    |   17 +-
 .../apache/ignite/internal/visor/VisorJob.java  |   17 +-
 .../internal/visor/VisorMultiNodeTask.java      |   31 +-
 .../ignite/internal/visor/VisorOneNodeTask.java |   11 +-
 .../internal/visor/VisorTaskArgument.java       |    8 +-
 .../ignite/internal/visor/cache/VisorCache.java |   44 +-
 .../cache/VisorCacheAffinityConfiguration.java  |   17 +-
 .../cache/VisorCacheAggregatedMetrics.java      |   14 +-
 .../visor/cache/VisorCacheClearTask.java        |   34 +-
 .../visor/cache/VisorCacheConfiguration.java    |   25 +-
 .../VisorCacheConfigurationCollectorJob.java    |   17 +-
 .../VisorCacheConfigurationCollectorTask.java   |   12 +-
 .../cache/VisorCacheDefaultConfiguration.java   |    9 +-
 .../cache/VisorCacheEvictionConfiguration.java  |   16 +-
 .../visor/cache/VisorCacheLoadTask.java         |   26 +-
 .../visor/cache/VisorCacheMetadataTask.java     |   21 +-
 .../internal/visor/cache/VisorCacheMetrics.java |   16 +-
 .../cache/VisorCacheMetricsCollectorTask.java   |   25 +-
 .../cache/VisorCacheNearConfiguration.java      |   17 +-
 .../visor/cache/VisorCacheNodesTask.java        |   16 +-
 .../cache/VisorCacheQueryConfiguration.java     |    9 +-
 .../visor/cache/VisorCacheQueryMetrics.java     |    9 +-
 .../cache/VisorCacheRebalanceConfiguration.java |   11 +-
 .../visor/cache/VisorCacheRebalanceTask.java    |   21 +-
 .../visor/cache/VisorCacheResetMetricsTask.java |   11 +-
 .../visor/cache/VisorCacheStartTask.java        |   35 +-
 .../visor/cache/VisorCacheStopTask.java         |   11 +-
 .../cache/VisorCacheStoreConfiguration.java     |   21 +-
 .../visor/cache/VisorCacheSwapBackupsTask.java  |   28 +-
 .../cache/VisorCacheTypeFieldMetadata.java      |    9 +-
 .../visor/cache/VisorCacheTypeMetadata.java     |   20 +-
 .../compute/VisorComputeCancelSessionsTask.java |   24 +-
 .../compute/VisorComputeMonitoringHolder.java   |   14 +-
 .../compute/VisorComputeResetMetricsTask.java   |    9 +-
 .../VisorComputeToggleMonitoringTask.java       |   29 +-
 .../visor/debug/VisorThreadDumpTask.java        |   16 +-
 .../internal/visor/debug/VisorThreadInfo.java   |    6 +-
 .../visor/debug/VisorThreadLockInfo.java        |    6 +-
 .../visor/debug/VisorThreadMonitorInfo.java     |    4 +-
 .../visor/event/VisorGridDeploymentEvent.java   |   11 +-
 .../visor/event/VisorGridDiscoveryEvent.java    |   11 +-
 .../internal/visor/event/VisorGridEvent.java    |   13 +-
 .../visor/event/VisorGridEventsLost.java        |    9 +-
 .../internal/visor/event/VisorGridJobEvent.java |   11 +-
 .../visor/event/VisorGridTaskEvent.java         |   11 +-
 .../internal/visor/file/VisorFileBlock.java     |    7 +-
 .../internal/visor/file/VisorFileBlockTask.java |   26 +-
 .../visor/file/VisorLatestTextFilesTask.java    |   30 +-
 .../ignite/internal/visor/igfs/VisorIgfs.java   |   11 +-
 .../internal/visor/igfs/VisorIgfsEndpoint.java  |    9 +-
 .../visor/igfs/VisorIgfsFormatTask.java         |   11 +-
 .../internal/visor/igfs/VisorIgfsMetrics.java   |    9 +-
 .../internal/visor/igfs/VisorIgfsProfiler.java  |   10 +-
 .../visor/igfs/VisorIgfsProfilerClearTask.java  |   30 +-
 .../visor/igfs/VisorIgfsProfilerEntry.java      |   11 +-
 .../visor/igfs/VisorIgfsProfilerTask.java       |   51 +-
 .../VisorIgfsProfilerUniformityCounters.java    |   12 +-
 .../visor/igfs/VisorIgfsResetMetricsTask.java   |   14 +-
 .../visor/igfs/VisorIgfsSamplingStateTask.java  |   17 +-
 .../ignite/internal/visor/log/VisorLogFile.java |    8 +-
 .../visor/log/VisorLogSearchResult.java         |    9 +-
 .../internal/visor/log/VisorLogSearchTask.java  |   45 +-
 .../internal/visor/misc/VisorAckTask.java       |   16 +-
 .../visor/misc/VisorLatestVersionTask.java      |    9 +-
 .../internal/visor/misc/VisorNopTask.java       |   21 +-
 .../visor/misc/VisorResolveHostNameTask.java    |   23 +-
 .../visor/node/VisorAtomicConfiguration.java    |   11 +-
 .../visor/node/VisorBasicConfiguration.java     |   34 +-
 .../node/VisorExecutorServiceConfiguration.java |   10 +-
 .../visor/node/VisorGridConfiguration.java      |   19 +-
 .../visor/node/VisorIgfsConfiguration.java      |   29 +-
 .../visor/node/VisorLifecycleConfiguration.java |   13 +-
 .../visor/node/VisorMetricsConfiguration.java   |    9 +-
 .../VisorNodeConfigurationCollectorJob.java     |    6 +-
 .../VisorNodeConfigurationCollectorTask.java    |    6 +-
 .../visor/node/VisorNodeDataCollectorJob.java   |   41 +-
 .../node/VisorNodeDataCollectorJobResult.java   |   15 +-
 .../visor/node/VisorNodeDataCollectorTask.java  |   30 +-
 .../node/VisorNodeDataCollectorTaskArg.java     |    4 +-
 .../node/VisorNodeDataCollectorTaskResult.java  |   21 +-
 .../node/VisorNodeEventsCollectorTask.java      |   46 +-
 .../internal/visor/node/VisorNodeGcTask.java    |   24 +-
 .../internal/visor/node/VisorNodePingTask.java  |   21 +-
 .../visor/node/VisorNodeRestartTask.java        |   18 +-
 .../internal/visor/node/VisorNodeStopTask.java  |   18 +-
 .../node/VisorNodeSuppressedErrorsTask.java     |   26 +-
 .../node/VisorPeerToPeerConfiguration.java      |   13 +-
 .../visor/node/VisorRestConfiguration.java      |   22 +-
 .../node/VisorSegmentationConfiguration.java    |   15 +-
 .../visor/node/VisorSpisConfiguration.java      |   27 +-
 .../node/VisorTransactionConfiguration.java     |   12 +-
 .../internal/visor/query/VisorQueryArg.java     |    4 +-
 .../visor/query/VisorQueryCleanupTask.java      |   32 +-
 .../internal/visor/query/VisorQueryCursor.java  |   14 +-
 .../internal/visor/query/VisorQueryField.java   |    9 +-
 .../internal/visor/query/VisorQueryJob.java     |   43 +-
 .../visor/query/VisorQueryNextPageTask.java     |   21 +-
 .../internal/visor/query/VisorQueryResult.java  |    9 +-
 .../visor/query/VisorQueryResultEx.java         |    9 +-
 .../internal/visor/query/VisorQueryTask.java    |   10 +-
 .../internal/visor/query/VisorQueryUtils.java   |   17 +-
 .../util/VisorClusterGroupEmptyException.java   |    4 +-
 .../internal/visor/util/VisorEventMapper.java   |   24 +-
 .../visor/util/VisorExceptionWrapper.java       |    2 +-
 .../internal/visor/util/VisorMimeTypes.java     |   13 +-
 .../internal/visor/util/VisorTaskUtils.java     |   97 +-
 .../apache/ignite/lang/IgniteAsyncSupport.java  |    2 +-
 .../ignite/lang/IgniteAsyncSupported.java       |    8 +-
 .../org/apache/ignite/lang/IgniteBiClosure.java |    4 +-
 .../apache/ignite/lang/IgniteBiInClosure.java   |    4 +-
 .../apache/ignite/lang/IgniteBiPredicate.java   |    4 +-
 .../org/apache/ignite/lang/IgniteBiTuple.java   |   24 +-
 .../org/apache/ignite/lang/IgniteCallable.java  |    6 +-
 .../org/apache/ignite/lang/IgniteClosure.java   |    4 +-
 .../org/apache/ignite/lang/IgniteFuture.java    |    8 +-
 .../lang/IgniteFutureCancelledException.java    |    6 +-
 .../lang/IgniteFutureTimeoutException.java      |    6 +-
 .../org/apache/ignite/lang/IgniteInClosure.java |    4 +-
 .../ignite/lang/IgniteNotPeerDeployable.java    |    8 +-
 .../apache/ignite/lang/IgniteOutClosure.java    |    4 +-
 .../org/apache/ignite/lang/IgnitePredicate.java |    4 +-
 .../ignite/lang/IgniteProductVersion.java       |   21 +-
 .../org/apache/ignite/lang/IgniteReducer.java   |    9 +-
 .../org/apache/ignite/lang/IgniteRunnable.java  |    4 +-
 .../java/org/apache/ignite/lang/IgniteUuid.java |   20 +-
 .../apache/ignite/lifecycle/LifecycleAware.java |    4 +-
 .../apache/ignite/lifecycle/LifecycleBean.java  |    4 +-
 .../ignite/lifecycle/LifecycleEventType.java    |    4 +-
 .../apache/ignite/logger/LoggerNodeIdAware.java |    4 +-
 .../org/apache/ignite/logger/NullLogger.java    |    6 +-
 .../apache/ignite/logger/java/JavaLogger.java   |   40 +-
 .../logger/java/JavaLoggerFileHandler.java      |   22 +-
 .../ignite/logger/java/JavaLoggerFormatter.java |   14 +-
 .../ignite/marshaller/AbstractMarshaller.java   |   13 +-
 .../apache/ignite/marshaller/Marshaller.java    |   12 +-
 .../ignite/marshaller/MarshallerContext.java    |    4 +-
 .../ignite/marshaller/MarshallerExclusions.java |   27 +-
 .../ignite/marshaller/jdk/JdkMarshaller.java    |   20 +-
 .../jdk/JdkMarshallerDummySerializable.java     |    4 +-
 .../jdk/JdkMarshallerInputStreamWrapper.java    |    5 +-
 .../jdk/JdkMarshallerObjectInputStream.java     |    9 +-
 .../jdk/JdkMarshallerObjectOutputStream.java    |   10 +-
 .../jdk/JdkMarshallerOutputStreamWrapper.java   |    5 +-
 .../optimized/OptimizedClassDescriptor.java     |   83 +-
 .../optimized/OptimizedFieldType.java           |    2 +-
 .../optimized/OptimizedMarshaller.java          |   28 +-
 .../optimized/OptimizedMarshallerIdMapper.java  |    2 +-
 .../optimized/OptimizedMarshallerUtils.java     |   32 +-
 .../optimized/OptimizedObjectInputStream.java   |   91 +-
 .../optimized/OptimizedObjectOutputStream.java  |   58 +-
 .../OptimizedObjectStreamRegistry.java          |   17 +-
 .../ignite/messaging/MessagingListenActor.java  |   22 +-
 .../ignite/mxbean/CacheMetricsMXBean.java       |   10 +-
 .../mxbean/ClusterLocalNodeMetricsMXBean.java   |    4 +-
 .../org/apache/ignite/mxbean/IgniteMXBean.java  |    7 +-
 .../apache/ignite/mxbean/IgnitionMXBean.java    |    2 +-
 .../apache/ignite/mxbean/MXBeanDescription.java |    8 +-
 .../mxbean/MXBeanParametersDescriptions.java    |    8 +-
 .../ignite/mxbean/MXBeanParametersNames.java    |    8 +-
 .../apache/ignite/mxbean/ThreadPoolMXBean.java  |    2 +-
 .../ignite/plugin/CachePluginConfiguration.java |    5 +-
 .../ignite/plugin/CachePluginContext.java       |   10 +-
 .../ignite/plugin/CachePluginProvider.java      |   10 +-
 .../org/apache/ignite/plugin/Extension.java     |    2 +-
 .../apache/ignite/plugin/ExtensionRegistry.java |    2 +-
 .../org/apache/ignite/plugin/IgnitePlugin.java  |    2 +-
 .../ignite/plugin/PluginConfiguration.java      |    2 +-
 .../org/apache/ignite/plugin/PluginContext.java |   16 +-
 .../ignite/plugin/PluginNotFoundException.java  |    4 +-
 .../apache/ignite/plugin/PluginProvider.java    |   16 +-
 .../plugin/PluginValidationException.java       |    7 +-
 .../plugin/extensions/communication/IoPool.java |    7 +-
 .../extensions/communication/Message.java       |    6 +-
 .../MessageCollectionItemType.java              |    4 +-
 .../communication/MessageFactory.java           |    6 +-
 .../communication/MessageFormatter.java         |    7 +-
 .../extensions/communication/MessageReader.java |   26 +-
 .../extensions/communication/MessageWriter.java |   12 +-
 .../plugin/security/AuthenticationContext.java  |    6 +-
 .../plugin/security/SecurityCredentials.java    |   16 +-
 .../SecurityCredentialsBasicProvider.java       |    4 +-
 .../security/SecurityCredentialsProvider.java   |    4 +-
 .../plugin/security/SecurityException.java      |    6 +-
 .../plugin/security/SecurityPermission.java     |    4 +-
 .../plugin/security/SecurityPermissionSet.java  |   10 +-
 .../ignite/plugin/security/SecuritySubject.java |    8 +-
 .../plugin/security/SecuritySubjectType.java    |    6 +-
 .../plugin/segmentation/SegmentationPolicy.java |    2 +-
 .../segmentation/SegmentationResolver.java      |    9 +-
 .../ignite/resources/CacheNameResource.java     |    8 +-
 .../resources/CacheStoreSessionResource.java    |    8 +-
 .../resources/IgniteInstanceResource.java       |    8 +-
 .../ignite/resources/JobContextResource.java    |    8 +-
 .../ignite/resources/LoadBalancerResource.java  |    8 +-
 .../apache/ignite/resources/LoggerResource.java |    8 +-
 .../ignite/resources/MBeanServerResource.java   |   10 +-
 .../ignite/resources/ServiceResource.java       |    8 +-
 .../SpringApplicationContextResource.java       |   10 +-
 .../apache/ignite/resources/SpringResource.java |   10 +-
 .../resources/TaskContinuousMapperResource.java |    8 +-
 .../ignite/resources/TaskSessionResource.java   |    8 +-
 .../apache/ignite/resources/package-info.java   |    2 +-
 .../ignite/scheduler/SchedulerFuture.java       |   14 +-
 .../org/apache/ignite/services/Service.java     |    4 +-
 .../ignite/services/ServiceConfiguration.java   |   13 +-
 .../apache/ignite/services/ServiceContext.java  |    9 +-
 .../ignite/services/ServiceDescriptor.java      |   10 +-
 .../ignite/spi/IgniteNodeValidationResult.java  |    4 +-
 .../apache/ignite/spi/IgnitePortProtocol.java   |    4 +-
 .../java/org/apache/ignite/spi/IgniteSpi.java   |    9 +-
 .../org/apache/ignite/spi/IgniteSpiAdapter.java |   70 +-
 .../ignite/spi/IgniteSpiCloseableIterator.java  |    9 +-
 .../ignite/spi/IgniteSpiConfiguration.java      |    8 +-
 .../ignite/spi/IgniteSpiConsistencyChecked.java |    8 +-
 .../org/apache/ignite/spi/IgniteSpiContext.java |   27 +-
 .../apache/ignite/spi/IgniteSpiException.java   |    4 +-
 .../ignite/spi/IgniteSpiManagementMBean.java    |    7 +-
 .../ignite/spi/IgniteSpiMultiException.java     |   11 +-
 .../spi/IgniteSpiMultipleInstancesSupport.java  |    9 +-
 .../org/apache/ignite/spi/IgniteSpiNoop.java    |    8 +-
 .../spi/IgniteSpiOperationTimeoutException.java |   10 +-
 .../spi/IgniteSpiOperationTimeoutHelper.java    |   10 +-
 .../org/apache/ignite/spi/IgniteSpiThread.java  |   10 +-
 .../ignite/spi/IgniteSpiThreadFactory.java      |    7 +-
 .../ignite/spi/IgniteSpiTimeoutObject.java      |    4 +-
 .../spi/IgniteSpiVersionCheckException.java     |    2 +-
 .../spi/checkpoint/CheckpointListener.java      |    2 +-
 .../ignite/spi/checkpoint/CheckpointSpi.java    |    7 +-
 .../checkpoint/cache/CacheCheckpointSpi.java    |   37 +-
 .../cache/CacheCheckpointSpiMBean.java          |    6 +-
 .../spi/checkpoint/jdbc/JdbcCheckpointSpi.java  |   32 +-
 .../checkpoint/jdbc/JdbcCheckpointSpiMBean.java |    6 +-
 .../spi/checkpoint/noop/NoopCheckpointSpi.java  |   19 +-
 .../sharedfs/SharedFsCheckpointData.java        |    7 +-
 .../sharedfs/SharedFsCheckpointSpi.java         |   42 +-
 .../sharedfs/SharedFsCheckpointSpiMBean.java    |    9 +-
 .../checkpoint/sharedfs/SharedFsTimeData.java   |    4 +-
 .../sharedfs/SharedFsTimeoutTask.java           |   23 +-
 .../spi/checkpoint/sharedfs/SharedFsUtils.java  |   17 +-
 .../ignite/spi/collision/CollisionContext.java  |    4 +-
 .../collision/CollisionExternalListener.java    |    4 +-
 .../spi/collision/CollisionJobContext.java      |    6 +-
 .../ignite/spi/collision/CollisionSpi.java      |    6 +-
 .../fifoqueue/FifoQueueCollisionSpi.java        |   23 +-
 .../fifoqueue/FifoQueueCollisionSpiMBean.java   |    6 +-
 .../jobstealing/JobStealingCollisionSpi.java    |   69 +-
 .../JobStealingCollisionSpiMBean.java           |   11 +-
 .../jobstealing/JobStealingDisabled.java        |    8 +-
 .../jobstealing/JobStealingRequest.java         |   15 +-
 .../spi/collision/noop/NoopCollisionSpi.java    |   15 +-
 .../PriorityQueueCollisionSpi.java              |   39 +-
 .../PriorityQueueCollisionSpiMBean.java         |    6 +-
 .../communication/CommunicationListener.java    |    9 +-
 .../spi/communication/CommunicationSpi.java     |   14 +-
 .../communication/tcp/TcpCommunicationSpi.java  |  170 +-
 .../tcp/TcpCommunicationSpiMBean.java           |    6 +-
 .../spi/deployment/DeploymentListener.java      |    2 +-
 .../spi/deployment/DeploymentResource.java      |    2 +-
 .../deployment/DeploymentResourceAdapter.java   |    4 +-
 .../ignite/spi/deployment/DeploymentSpi.java    |    7 +-
 .../IgnoreIfPeerClassLoadingDisabled.java       |    9 +-
 .../deployment/local/LocalDeploymentSpi.java    |   44 +-
 .../local/LocalDeploymentSpiMBean.java          |    6 +-
 .../spi/discovery/DiscoveryMetricsProvider.java |   11 +-
 .../ignite/spi/discovery/DiscoverySpi.java      |   20 +-
 .../discovery/DiscoverySpiCustomMessage.java    |    7 +-
 .../spi/discovery/DiscoverySpiDataExchange.java |    7 +-
 .../discovery/DiscoverySpiHistorySupport.java   |    9 +-
 .../spi/discovery/DiscoverySpiListener.java     |   12 +-
 .../DiscoverySpiNodeAuthenticator.java          |   10 +-
 .../spi/discovery/DiscoverySpiOrderSupport.java |    9 +-
 .../ignite/spi/discovery/tcp/ClientImpl.java    |  123 +-
 .../ignite/spi/discovery/tcp/ServerImpl.java    |  336 +-
 .../spi/discovery/tcp/TcpDiscoveryImpl.java     |   40 +-
 .../spi/discovery/tcp/TcpDiscoverySpi.java      |  170 +-
 .../spi/discovery/tcp/TcpDiscoverySpiMBean.java |   12 +-
 .../tcp/internal/TcpDiscoveryNode.java          |   68 +-
 .../tcp/internal/TcpDiscoveryNodesRing.java     |   56 +-
 .../tcp/internal/TcpDiscoverySpiState.java      |    2 +-
 .../tcp/internal/TcpDiscoveryStatistics.java    |   26 +-
 .../tcp/ipfinder/TcpDiscoveryIpFinder.java      |   10 +-
 .../ipfinder/TcpDiscoveryIpFinderAdapter.java   |   23 +-
 .../ipfinder/jdbc/TcpDiscoveryJdbcIpFinder.java |   42 +-
 .../TcpDiscoveryMulticastIpFinder.java          |   94 +-
 .../sharedfs/TcpDiscoverySharedFsIpFinder.java  |   37 +-
 .../tcp/ipfinder/vm/TcpDiscoveryVmIpFinder.java |   31 +-
 .../messages/TcpDiscoveryAbstractMessage.java   |   14 +-
 .../messages/TcpDiscoveryAuthFailedMessage.java |   14 +-
 .../TcpDiscoveryCheckFailedMessage.java         |    7 +-
 .../messages/TcpDiscoveryClientAckResponse.java |   12 +-
 .../TcpDiscoveryClientHeartbeatMessage.java     |   11 +-
 .../messages/TcpDiscoveryClientPingRequest.java |    9 +-
 .../TcpDiscoveryClientPingResponse.java         |    9 +-
 .../TcpDiscoveryClientReconnectMessage.java     |   12 +-
 .../TcpDiscoveryConnectionCheckMessage.java     |   12 +-
 .../TcpDiscoveryCustomEventMessage.java         |   15 +-
 .../messages/TcpDiscoveryDiscardMessage.java    |    9 +-
 .../TcpDiscoveryDuplicateIdMessage.java         |    9 +-
 .../messages/TcpDiscoveryEnsureDelivery.java    |    8 +-
 .../messages/TcpDiscoveryHandshakeRequest.java  |    7 +-
 .../messages/TcpDiscoveryHandshakeResponse.java |    7 +-
 .../messages/TcpDiscoveryHeartbeatMessage.java  |   30 +-
 .../TcpDiscoveryJoinRequestMessage.java         |    9 +-
 .../TcpDiscoveryLoopbackProblemMessage.java     |    8 +-
 .../TcpDiscoveryNodeAddFinishedMessage.java     |   12 +-
 .../messages/TcpDiscoveryNodeAddedMessage.java  |   20 +-
 .../messages/TcpDiscoveryNodeFailedMessage.java |    9 +-
 .../messages/TcpDiscoveryNodeLeftMessage.java   |    7 +-
 .../tcp/messages/TcpDiscoveryPingRequest.java   |    9 +-
 .../tcp/messages/TcpDiscoveryPingResponse.java  |    7 +-
 .../messages/TcpDiscoveryRedirectToClient.java  |    8 +-
 .../TcpDiscoveryStatusCheckMessage.java         |    9 +-
 .../spi/eventstorage/EventStorageSpi.java       |   12 +-
 .../memory/MemoryEventStorageSpi.java           |   34 +-
 .../memory/MemoryEventStorageSpiMBean.java      |    6 +-
 .../ignite/spi/failover/FailoverContext.java    |   18 +-
 .../apache/ignite/spi/failover/FailoverSpi.java |    9 +-
 .../spi/failover/always/AlwaysFailoverSpi.java  |   33 +-
 .../failover/always/AlwaysFailoverSpiMBean.java |    6 +-
 .../jobstealing/JobStealingFailoverSpi.java     |   37 +-
 .../JobStealingFailoverSpiMBean.java            |    6 +-
 .../spi/failover/never/NeverFailoverSpi.java    |   23 +-
 .../failover/never/NeverFailoverSpiMBean.java   |    6 +-
 .../spi/indexing/IndexingQueryFilter.java       |    6 +-
 .../apache/ignite/spi/indexing/IndexingSpi.java |   15 +-
 .../spi/indexing/noop/NoopIndexingSpi.java      |   17 +-
 .../spi/loadbalancing/LoadBalancingSpi.java     |   14 +-
 .../adaptive/AdaptiveCpuLoadProbe.java          |    8 +-
 .../adaptive/AdaptiveJobCountLoadProbe.java     |    7 +-
 .../adaptive/AdaptiveLoadBalancingSpi.java      |   69 +-
 .../adaptive/AdaptiveLoadBalancingSpiMBean.java |    6 +-
 .../adaptive/AdaptiveLoadProbe.java             |    4 +-
 .../AdaptiveProcessingTimeLoadProbe.java        |    7 +-
 .../RoundRobinGlobalLoadBalancer.java           |   39 +-
 .../roundrobin/RoundRobinLoadBalancingSpi.java  |   50 +-
 .../RoundRobinLoadBalancingSpiMBean.java        |    6 +-
 .../RoundRobinPerTaskLoadBalancer.java          |   10 +-
 .../WeightedRandomLoadBalancingSpi.java         |   57 +-
 .../WeightedRandomLoadBalancingSpiMBean.java    |    6 +-
 .../ignite/spi/swapspace/SwapContext.java       |    4 +-
 .../apache/ignite/spi/swapspace/SwapKey.java    |    9 +-
 .../ignite/spi/swapspace/SwapSpaceSpi.java      |   16 +-
 .../spi/swapspace/SwapSpaceSpiListener.java     |    4 +-
 .../spi/swapspace/file/FileSwapArray.java       |    7 +-
 .../spi/swapspace/file/FileSwapSpaceSpi.java    |   83 +-
 .../swapspace/file/FileSwapSpaceSpiMBean.java   |    6 +-
 .../spi/swapspace/noop/NoopSwapSpaceSpi.java    |   33 +-
 .../apache/ignite/ssl/SslContextFactory.java    |   29 +-
 .../ignite/startup/BasicWarmupClosure.java      |   45 +-
 .../ignite/startup/cmdline/AboutDialog.java     |   41 +-
 .../CommandLineRandomNumberGenerator.java       |    4 +-
 .../startup/cmdline/CommandLineStartup.java     |   53 +-
 .../startup/cmdline/CommandLineTransformer.java |    7 +-
 .../org/apache/ignite/startup/package-info.java |    2 +-
 .../org/apache/ignite/stream/StreamAdapter.java |    6 +-
 .../apache/ignite/stream/StreamReceiver.java    |   12 +-
 .../apache/ignite/stream/StreamTransformer.java |   15 +-
 .../ignite/stream/StreamTupleExtractor.java     |    4 +-
 .../org/apache/ignite/stream/StreamVisitor.java |   11 +-
 .../org/apache/ignite/stream/package-info.java  |    2 +-
 .../stream/socket/SocketMessageConverter.java   |    2 +-
 .../ignite/stream/socket/SocketStreamer.java    |   35 +-
 .../thread/IgniteStripedThreadPoolExecutor.java |   18 +-
 .../org/apache/ignite/thread/IgniteThread.java  |    9 +-
 .../ignite/thread/IgniteThreadFactory.java      |    7 +-
 .../ignite/thread/IgniteThreadPoolExecutor.java |   13 +-
 .../apache/ignite/transactions/Transaction.java |   12 +-
 .../transactions/TransactionConcurrency.java    |    4 +-
 .../TransactionHeuristicException.java          |    4 +-
 .../transactions/TransactionIsolation.java      |    4 +-
 .../ignite/transactions/TransactionMetrics.java |    4 +-
 .../TransactionOptimisticException.java         |    4 +-
 .../TransactionRollbackException.java           |    4 +-
 .../ignite/transactions/TransactionState.java   |    4 +-
 .../TransactionTimeoutException.java            |    4 +-
 .../java/org/jetbrains/annotations/NotNull.java |    8 +-
 .../java/org/jsr166/ConcurrentHashMap8.java     |   20 +-
 .../java/org/jsr166/ConcurrentLinkedDeque8.java |   18 +-
 .../org/jsr166/ConcurrentLinkedHashMap.java     |   28 +-
 .../src/main/java/org/jsr166/LongAdder8.java    |    8 +-
 .../src/main/java/org/jsr166/Striped64_8.java   |    4 +-
 .../resources/META-INF/classnames.properties    |  298 +-
 .../core/src/main/resources/ignite.properties   |    2 +-
 .../config/store/jdbc/ignite-type-metadata.xml  |    8 +
 .../GridCacheAffinityBackupsSelfTest.java       |   20 +-
 .../ignite/GridSuppressedExceptionSelfTest.java |   31 +-
 .../java/org/apache/ignite/GridTestIoUtils.java |   28 +-
 .../java/org/apache/ignite/GridTestJob.java     |    6 +-
 .../org/apache/ignite/GridTestJobContext.java   |   12 +-
 .../org/apache/ignite/GridTestJobResult.java    |   11 +-
 .../org/apache/ignite/GridTestNodeStartup.java  |    7 +-
 .../apache/ignite/GridTestStoreNodeStartup.java |    8 +-
 .../java/org/apache/ignite/GridTestTask.java    |   13 +-
 .../org/apache/ignite/GridTestTaskSession.java  |   20 +-
 .../ignite/IgniteCacheAffinitySelfTest.java     |   30 +-
 .../IgniteExternalizableAbstractTest.java       |   16 +-
 .../cache/IgniteWarmupClosureSelfTest.java      |   15 +-
 .../affinity/IgniteClientNodeAffinityTest.java  |   34 +-
 .../GridFairAffinityFunctionNodesSelfTest.java  |   25 +-
 .../fair/GridFairAffinityFunctionSelfTest.java  |   26 +-
 .../IgniteFairAffinityDynamicCacheSelfTest.java |   25 +-
 ...cheStoreSessionListenerAbstractSelfTest.java |   36 +-
 ...heStoreSessionListenerLifecycleSelfTest.java |   40 +-
 .../store/GridCacheBalancingStoreSelfTest.java  |   34 +-
 .../GridCacheLoadOnlyStoreAdapterSelfTest.java  |   18 +-
 .../cache/store/GridGeneratingTestStore.java    |   19 +-
 .../IgniteCacheExpiryStoreLoadSelfTest.java     |   39 +-
 .../store/StoreResourceInjectionSelfTest.java   |  104 +
 .../CacheJdbcPojoStoreMultitreadedSelfTest.java |    4 +-
 .../store/jdbc/CacheJdbcPojoStoreTest.java      |   90 +-
 ...eJdbcStoreAbstractMultithreadedSelfTest.java |   83 +-
 .../CacheJdbcStoreSessionListenerSelfTest.java  |   28 +-
 ...CacheJdbcBlobStoreMultithreadedSelfTest.java |   55 +-
 .../jdbc/GridCacheJdbcBlobStoreSelfTest.java    |    9 +-
 .../cache/store/jdbc/model/Organization.java    |    3 +-
 .../cache/store/jdbc/model/OrganizationKey.java |    3 +-
 .../ignite/cache/store/jdbc/model/Person.java   |   29 +-
 .../store/jdbc/model/PersonComplexKey.java      |    4 +-
 .../cache/store/jdbc/model/PersonKey.java       |    3 +-
 .../ignite/igfs/IgfsEventsAbstractSelfTest.java |   64 +-
 .../igfs/IgfsFragmentizerAbstractSelfTest.java  |   30 +-
 .../ignite/igfs/IgfsFragmentizerSelfTest.java   |   23 +-
 .../igfs/IgfsFragmentizerTopologySelfTest.java  |    4 +-
 .../apache/ignite/igfs/IgfsPathSelfTest.java    |   26 +-
 .../ignite/igfs/IgfsTestInputGenerator.java     |    7 +-
 .../internal/ClusterGroupAbstractTest.java      |   64 +-
 .../internal/ClusterGroupHostsSelfTest.java     |   24 +-
 .../ignite/internal/ClusterGroupSelfTest.java   |   21 +-
 .../ignite/internal/ClusterMetricsSelfTest.java |   34 +-
 .../internal/ClusterNodeMetricsSelfTest.java    |   41 +-
 .../ignite/internal/GridAffinityMappedTest.java |   33 +-
 .../internal/GridAffinityP2PSelfTest.java       |   41 +-
 .../ignite/internal/GridAffinitySelfTest.java   |   34 +-
 .../GridAlwaysFailoverSpiFailSelfTest.java      |   32 +-
 .../internal/GridCancelOnGridStopSelfTest.java  |   27 +-
 .../internal/GridCancelUnusedJobSelfTest.java   |   39 +-
 .../GridCancelledJobsMetricsSelfTest.java       |   47 +-
 .../GridCollisionJobsContextSelfTest.java       |   28 +-
 .../internal/GridCommunicationSelfTest.java     |   19 +-
 .../GridContinuousJobAnnotationSelfTest.java    |   36 +-
 .../GridContinuousJobSiblingsSelfTest.java      |   29 +-
 .../internal/GridContinuousTaskSelfTest.java    |   44 +-
 .../GridDeploymentMultiThreadedSelfTest.java    |   29 +-
 .../ignite/internal/GridDeploymentSelfTest.java |   43 +-
 .../internal/GridDiscoveryEventSelfTest.java    |   48 +-
 .../ignite/internal/GridDiscoverySelfTest.java  |   60 +-
 .../GridEventStorageCheckAllEventsSelfTest.java |   67 +-
 ...ventStorageRuntimeConfigurationSelfTest.java |   38 +-
 .../internal/GridEventStorageSelfTest.java      |   38 +-
 .../internal/GridFactoryVmShutdownTest.java     |   21 +-
 ...ridFailFastNodeFailureDetectionSelfTest.java |   37 +-
 .../GridFailedInputParametersSelfTest.java      |   14 +-
 .../GridFailoverCustomTopologySelfTest.java     |   38 +-
 .../ignite/internal/GridFailoverSelfTest.java   |   36 +-
 .../GridFailoverTaskWithPredicateSelfTest.java  |   41 +-
 .../internal/GridFailoverTopologySelfTest.java  |   34 +-
 .../ignite/internal/GridHomePathSelfTest.java   |   13 +-
 .../GridJobCheckpointCleanupSelfTest.java       |   35 +-
 .../GridJobCollisionCancelSelfTest.java         |   41 +-
 .../ignite/internal/GridJobContextSelfTest.java |   26 +-
 .../GridJobMasterLeaveAwareSelfTest.java        |   71 +-
 .../internal/GridJobStealingSelfTest.java       |   51 +-
 .../GridJobStealingZeroActiveJobsSelfTest.java  |   36 +-
 .../internal/GridJobSubjectIdSelfTest.java      |   32 +-
 .../GridKernalConcurrentAccessStopSelfTest.java |   12 +-
 .../ignite/internal/GridKernalTestUtils.java    |    4 +-
 .../internal/GridLifecycleAwareSelfTest.java    |   28 +-
 .../internal/GridLifecycleBeanSelfTest.java     |   37 +-
 .../internal/GridListenActorSelfTest.java       |   21 +-
 .../GridLocalEventListenerSelfTest.java         |   18 +-
 .../internal/GridMultipleJobsSelfTest.java      |   41 +-
 .../internal/GridMultipleSpisSelfTest.java      |   40 +-
 .../GridMultipleVersionsDeploymentSelfTest.java |   48 +-
 .../GridMultithreadedJobStealingSelfTest.java   |   41 +-
 .../ignite/internal/GridNodeFilterSelfTest.java |   13 +-
 .../ignite/internal/GridNodeLocalSelfTest.java  |   17 +-
 .../GridNodeVisorAttributesSelfTest.java        |   11 +-
 .../internal/GridNonHistoryMetricsSelfTest.java |   37 +-
 .../GridProjectionForCachesSelfTest.java        |   36 +-
 ...ectionLocalJobMultipleArgumentsSelfTest.java |   33 +-
 .../ignite/internal/GridReduceSelfTest.java     |   20 +-
 .../internal/GridReleaseTypeSelfTest.java       |   22 +-
 .../internal/GridRuntimeExceptionSelfTest.java  |   40 +-
 .../internal/GridSameVmStartupSelfTest.java     |   29 +-
 .../apache/ignite/internal/GridSelfTest.java    |   24 +-
 .../internal/GridSpiExceptionSelfTest.java      |   33 +-
 .../ignite/internal/GridStartStopSelfTest.java  |   44 +-
 .../apache/ignite/internal/GridStartupMain.java |    9 +-
 .../apache/ignite/internal/GridStartupTest.java |   12 +-
 .../internal/GridStopWithCancelSelfTest.java    |   28 +-
 .../internal/GridStopWithWaitSelfTest.java      |   46 +-
 .../GridTaskCancelSingleNodeSelfTest.java       |   41 +-
 .../GridTaskContinuousMapperSelfTest.java       |   31 +-
 .../GridTaskExecutionContextSelfTest.java       |   34 +-
 .../internal/GridTaskExecutionSelfTest.java     |   11 +-
 .../GridTaskFailoverAffinityRunTest.java        |   38 +-
 .../internal/GridTaskFailoverSelfTest.java      |   25 +-
 .../GridTaskFutureImplStopGridSelfTest.java     |   34 +-
 .../GridTaskInstanceExecutionSelfTest.java      |   23 +-
 .../internal/GridTaskInstantiationSelfTest.java |   26 +-
 .../internal/GridTaskJobRejectSelfTest.java     |   46 +-
 .../internal/GridTaskListenerSelfTest.java      |   28 +-
 .../internal/GridTaskMapAsyncSelfTest.java      |   33 +-
 .../GridTaskNameAnnotationSelfTest.java         |   30 +-
 .../internal/GridTaskResultCacheSelfTest.java   |   23 +-
 .../internal/GridTaskTimeoutSelfTest.java       |   42 +-
 .../GridTopicExternalizableSelfTest.java        |   14 +-
 .../internal/GridUpdateNotifierSelfTest.java    |   38 +-
 .../ignite/internal/GridVersionSelfTest.java    |   28 +-
 .../IgniteClientReconnectAbstractTest.java      |   66 +-
 .../IgniteClientReconnectApiExceptionTest.java  |   57 +-
 .../IgniteClientReconnectAtomicsTest.java       |   18 +-
 .../IgniteClientReconnectCacheTest.java         |  102 +-
 .../IgniteClientReconnectCollectionsTest.java   |   26 +-
 .../IgniteClientReconnectComputeTest.java       |   17 +-
 ...eClientReconnectContinuousProcessorTest.java |   29 +-
 ...IgniteClientReconnectDiscoveryStateTest.java |   24 +-
 ...niteClientReconnectFailoverAbstractTest.java |   38 +-
 .../IgniteClientReconnectFailoverTest.java      |   34 +-
 .../IgniteClientReconnectServicesTest.java      |   23 +-
 .../internal/IgniteClientReconnectStopTest.java |   21 +-
 .../IgniteClientReconnectStreamerTest.java      |   28 +-
 .../IgniteComputeEmptyClusterGroupTest.java     |   35 +-
 .../IgniteComputeTopologyExceptionTest.java     |   25 +-
 .../internal/IgniteExecutorServiceTest.java     |   25 +-
 ...gniteExplicitImplicitDeploymentSelfTest.java |   43 +-
 .../internal/IgniteInternalCacheRemoveTest.java |    6 +-
 .../IgniteSlowClientDetectionSelfTest.java      |   47 +-
 .../managers/GridManagerStopSelfTest.java       |   64 +-
 .../managers/GridNoopManagerSelfTest.java       |   19 +-
 .../GridCheckpointManagerAbstractSelfTest.java  |   77 +-
 .../GridCheckpointManagerSelfTest.java          |    4 +-
 .../checkpoint/GridCheckpointTaskSelfTest.java  |   49 +-
 ...idCommunicationManagerListenersSelfTest.java |   26 +-
 .../GridCommunicationSendMessageSelfTest.java   |   35 +-
 .../communication/GridIoManagerSelfTest.java    |   45 +-
 .../GridDeploymentManagerStopSelfTest.java      |   25 +-
 .../GridDeploymentMessageCountSelfTest.java     |   45 +-
 .../managers/deployment/GridTestDeployment.java |    6 +-
 .../GridDiscoveryManagerAliveCacheSelfTest.java |   49 +-
 .../GridDiscoveryManagerAttributesSelfTest.java |   24 +-
 .../discovery/GridDiscoveryManagerSelfTest.java |   29 +-
 .../IgniteTopologyPrintFormatSelfTest.java      |   32 +-
 .../events/GridEventStorageManagerSelfTest.java |   24 +-
 .../GridSwapSpaceCustomLocalValue.java          |    7 +-
 .../swapspace/GridSwapSpaceManagerSelfTest.java |   38 +-
 .../GridCacheTxLoadFromStoreOnLockSelfTest.java |   36 +-
 .../GridAffinityProcessorAbstractSelfTest.java  |   32 +-
 ...GridAffinityProcessorRendezvousSelfTest.java |    6 +-
 .../cache/CacheAffinityCallSelfTest.java        |  131 +-
 .../cache/CacheClientStoreSelfTest.java         |   33 +-
 ...cheDhtLocalPartitionAfterRemoveSelfTest.java |   44 +-
 .../cache/CacheFutureExceptionSelfTest.java     |   32 +-
 .../processors/cache/CacheGetFromJobTest.java   |   19 +-
 .../CacheMetricsForClusterGroupSelfTest.java    |   31 +-
 ...cheNearUpdateTopologyChangeAbstractTest.java |   27 +-
 .../cache/CacheOffheapMapEntrySelfTest.java     |   36 +-
 .../CacheReadThroughAtomicRestartSelfTest.java  |    6 +-
 ...heReadThroughLocalAtomicRestartSelfTest.java |    6 +-
 .../CacheReadThroughLocalRestartSelfTest.java   |    6 +-
 ...dThroughReplicatedAtomicRestartSelfTest.java |    6 +-
 ...cheReadThroughReplicatedRestartSelfTest.java |    6 +-
 .../cache/CacheReadThroughRestartSelfTest.java  |   28 +-
 .../cache/CacheRemoveAllSelfTest.java           |   17 +-
 .../cache/CacheStopAndDestroySelfTest.java      |   50 +-
 .../CacheStoreUsageMultinodeAbstractTest.java   |   76 +-
 ...eUsageMultinodeDynamicStartAbstractTest.java |    8 +-
 ...oreUsageMultinodeDynamicStartAtomicTest.java |    6 +-
 ...heStoreUsageMultinodeDynamicStartTxTest.java |    6 +-
 ...reUsageMultinodeStaticStartAbstractTest.java |    9 +-
 ...toreUsageMultinodeStaticStartAtomicTest.java |    6 +-
 ...cheStoreUsageMultinodeStaticStartTxTest.java |    6 +-
 .../cache/CacheSwapUnswapGetTest.java           |  313 ++
 .../cache/CrossCacheTxRandomOperationsTest.java |  547 +++
 ...ridCacheAbstractByteArrayValuesSelfTest.java |   12 +-
 .../GridCacheAbstractFailoverSelfTest.java      |   42 +-
 .../GridCacheAbstractFailoverTxSelfTest.java    |    9 +-
 ...cheAbstractFullApiMultithreadedSelfTest.java |  102 +-
 .../cache/GridCacheAbstractFullApiSelfTest.java |  186 +-
 .../GridCacheAbstractIteratorsSelfTest.java     |   14 +-
 .../GridCacheAbstractLocalStoreSelfTest.java    |   73 +-
 .../cache/GridCacheAbstractMetricsSelfTest.java |   42 +-
 .../GridCacheAbstractRemoveFailureTest.java     |  236 +-
 .../cache/GridCacheAbstractSelfTest.java        |   70 +-
 .../cache/GridCacheAbstractTxReadTest.java      |   14 +-
 ...acheAbstractUsersAffinityMapperSelfTest.java |   37 +-
 .../cache/GridCacheAffinityApiSelfTest.java     |   34 +-
 .../cache/GridCacheAffinityMapperSelfTest.java  |   14 +-
 .../cache/GridCacheAffinityRoutingSelfTest.java |   45 +-
 .../cache/GridCacheAlwaysEvictionPolicy.java    |   11 +-
 .../GridCacheAsyncOperationsLimitSelfTest.java  |   15 +-
 .../GridCacheAtomicMessageCountSelfTest.java    |   49 +-
 ...dCacheAtomicUsersAffinityMapperSelfTest.java |    7 +-
 .../cache/GridCacheBasicApiAbstractTest.java    |   50 +-
 .../cache/GridCacheBasicStoreAbstractTest.java  |   47 +-
 ...acheBasicStoreMultithreadedAbstractTest.java |   29 +-
 .../cache/GridCacheClearAllSelfTest.java        |   31 +-
 .../cache/GridCacheClearLocallySelfTest.java    |   42 +-
 ...dCacheColocatedTxStoreExceptionSelfTest.java |    8 +-
 .../cache/GridCacheConcurrentMapSelfTest.java   |   35 +-
 .../cache/GridCacheConcurrentMapTest.java       |   28 +-
 .../GridCacheConcurrentTxMultiNodeTest.java     |   97 +-
 ...idCacheConfigurationConsistencySelfTest.java |   85 +-
 ...ridCacheConfigurationValidationSelfTest.java |   27 +-
 .../GridCacheDaemonNodeAbstractSelfTest.java    |   44 +-
 .../GridCacheDeploymentOffHeapSelfTest.java     |    9 +-
 .../cache/GridCacheDeploymentSelfTest.java      |   42 +-
 .../cache/GridCacheEntryMemorySizeSelfTest.java |   47 +-
 .../cache/GridCacheEntryVersionSelfTest.java    |   37 +-
 .../GridCacheEvictionEventAbstractTest.java     |   39 +-
 .../GridCacheFinishPartitionsSelfTest.java      |   42 +-
 ...CacheFullTextQueryMultithreadedSelfTest.java |   29 +-
 .../cache/GridCacheGenericTestStore.java        |   29 +-
 ...idCacheGetAndTransformStoreAbstractTest.java |   40 +-
 .../cache/GridCacheIncrementTransformTest.java  |   48 +-
 .../GridCacheInterceptorAbstractSelfTest.java   |   57 +-
 ...cheInterceptorAtomicNearEnabledSelfTest.java |    4 +-
 ...erceptorAtomicPrimaryWriteOrderSelfTest.java |   14 +-
 ...omicReplicatedPrimaryWriteOrderSelfTest.java |   14 +-
 ...acheInterceptorAtomicReplicatedSelfTest.java |   14 +-
 .../GridCacheInterceptorAtomicSelfTest.java     |   14 +-
 ...ceptorAtomicWithStoreReplicatedSelfTest.java |    2 +-
 ...CacheInterceptorAtomicWithStoreSelfTest.java |    2 +-
 ...GridCacheInterceptorLocalAtomicSelfTest.java |   14 +-
 ...InterceptorLocalAtomicWithStoreSelfTest.java |    2 +-
 .../GridCacheInterceptorLocalSelfTest.java      |    9 +-
 ...dCacheInterceptorLocalWithStoreSelfTest.java |    2 +-
 ...GridCacheInterceptorNearEnabledSelfTest.java |    6 +-
 .../GridCacheInterceptorReplicatedSelfTest.java |    9 +-
 ...eInterceptorReplicatedWithStoreSelfTest.java |    2 +-
 .../cache/GridCacheInterceptorSelfTest.java     |    9 +-
 .../GridCacheInterceptorWithStoreSelfTest.java  |    2 +-
 .../cache/GridCacheIteratorPerformanceTest.java |   27 +-
 .../GridCacheKeyCheckNearEnabledSelfTest.java   |    4 +-
 .../cache/GridCacheKeyCheckSelfTest.java        |   25 +-
 .../processors/cache/GridCacheLeakTest.java     |   33 +-
 .../cache/GridCacheLifecycleAwareSelfTest.java  |   52 +-
 .../GridCacheLocalTxStoreExceptionSelfTest.java |    6 +-
 .../cache/GridCacheLuceneQueryIndexTest.java    |   42 +-
 .../GridCacheMarshallerTxAbstractTest.java      |   28 +-
 .../GridCacheMarshallingNodeJoinSelfTest.java   |   50 +-
 .../cache/GridCacheMemoryModeSelfTest.java      |   46 +-
 .../GridCacheMissingCommitVersionSelfTest.java  |   33 +-
 ...GridCacheMixedPartitionExchangeSelfTest.java |   47 +-
 .../cache/GridCacheMultiUpdateLockSelfTest.java |   46 +-
 ...ridCacheMultinodeUpdateAbstractSelfTest.java |   26 +-
 ...ultinodeUpdateAtomicNearEnabledSelfTest.java |    8 +-
 .../GridCacheMultinodeUpdateAtomicSelfTest.java |    8 +-
 ...inodeUpdateNearEnabledNoBackupsSelfTest.java |    4 +-
 ...CacheMultinodeUpdateNearEnabledSelfTest.java |    8 +-
 .../cache/GridCacheMultinodeUpdateSelfTest.java |    8 +-
 .../cache/GridCacheMvccFlagsTest.java           |   16 +-
 .../cache/GridCacheMvccManagerSelfTest.java     |   31 +-
 .../cache/GridCacheMvccPartitionedSelfTest.java |   29 +-
 .../processors/cache/GridCacheMvccSelfTest.java |   59 +-
 .../GridCacheNearTxStoreExceptionSelfTest.java  |    6 +-
 .../cache/GridCacheNestedTxAbstractTest.java    |   40 +-
 .../cache/GridCacheObjectToStringSelfTest.java  |   41 +-
 ...ffHeapAtomicMultiThreadedUpdateSelfTest.java |    6 +-
 ...HeapMultiThreadedUpdateAbstractSelfTest.java |   34 +-
 ...CacheOffHeapMultiThreadedUpdateSelfTest.java |   18 +-
 .../processors/cache/GridCacheOffHeapTest.java  |   35 +-
 .../GridCacheOffHeapTieredAbstractSelfTest.java |   49 +-
 .../GridCacheOffHeapTieredAtomicSelfTest.java   |    6 +-
 ...heOffHeapTieredEvictionAbstractSelfTest.java |   34 +-
 ...acheOffHeapTieredEvictionAtomicSelfTest.java |    6 +-
 .../GridCacheOffHeapTieredEvictionSelfTest.java |    6 +-
 .../cache/GridCacheOffHeapTieredSelfTest.java   |    6 +-
 .../cache/GridCacheOffheapUpdateSelfTest.java   |   26 +-
 .../GridCacheOnCopyFlagAbstractSelfTest.java    |   40 +-
 .../GridCacheOnCopyFlagAtomicSelfTest.java      |    9 +-
 .../cache/GridCacheOnCopyFlagLocalSelfTest.java |    9 +-
 .../GridCacheOnCopyFlagReplicatedSelfTest.java  |    9 +-
 ...ridCacheOnCopyFlagTxPartitionedSelfTest.java |    9 +-
 .../GridCacheOrderedPreloadingSelfTest.java     |   29 +-
 .../cache/GridCacheP2PUndeploySelfTest.java     |   80 +-
 .../GridCachePartitionedAffinitySpreadTest.java |   19 +-
 .../cache/GridCachePartitionedGetSelfTest.java  |   42 +-
 .../GridCachePartitionedLocalStoreSelfTest.java |   11 +-
 ...chePartitionedOffHeapLocalStoreSelfTest.java |   11 +-
 ...hePartitionedProjectionAffinitySelfTest.java |   31 +-
 .../cache/GridCachePartitionedWritesTest.java   |   28 +-
 .../GridCachePreloadingEvictionsSelfTest.java   |   61 +-
 .../cache/GridCachePutAllFailoverSelfTest.java  |  163 +-
 .../processors/cache/GridCachePutAllTask.java   |   37 +-
 .../cache/GridCacheQueryEmbeddedValue.java      |    9 +-
 .../GridCacheQueryIndexingDisabledSelfTest.java |   20 +-
 .../GridCacheQueryInternalKeysSelfTest.java     |   33 +-
 .../GridCacheReferenceCleanupSelfTest.java      |   45 +-
 .../cache/GridCacheReloadSelfTest.java          |   29 +-
 .../GridCacheReplicatedLocalStoreSelfTest.java  |   11 +-
 ...ridCacheReplicatedSynchronousCommitTest.java |   50 +-
 ...CacheReplicatedTxStoreExceptionSelfTest.java |    8 +-
 ...heReplicatedUsersAffinityMapperSelfTest.java |    7 +-
 .../GridCacheReturnValueTransferSelfTest.java   |   35 +-
 .../cache/GridCacheSlowTxWarnTest.java          |   26 +-
 .../processors/cache/GridCacheStopSelfTest.java |   82 +-
 .../cache/GridCacheStorePutxSelfTest.java       |   39 +-
 .../cache/GridCacheStoreValueBytesSelfTest.java |   24 +-
 .../cache/GridCacheSwapPreloadSelfTest.java     |   46 +-
 .../cache/GridCacheSwapReloadSelfTest.java      |   54 +-
 ...acheTcpClientDiscoveryMultiThreadedTest.java |   31 +-
 .../processors/cache/GridCacheTestEntryEx.java  |   40 +-
 .../processors/cache/GridCacheTestKey.java      |    7 +-
 .../processors/cache/GridCacheTestStore.java    |   45 +-
 .../processors/cache/GridCacheTestValue.java    |    9 +-
 .../processors/cache/GridCacheTestValue2.java   |   11 +-
 ...cheTransactionalAbstractMetricsSelfTest.java |   22 +-
 .../cache/GridCacheTtlManagerLoadTest.java      |   23 +-
 .../cache/GridCacheTtlManagerSelfTest.java      |   35 +-
 ...ridCacheTxPartitionedLocalStoreSelfTest.java |   11 +-
 .../GridCacheTxUsersAffinityMapperSelfTest.java |    7 +-
 .../cache/GridCacheUtilsSelfTest.java           |   18 +-
 .../GridCacheValueBytesPreloadingSelfTest.java  |   26 +-
 ...idCacheValueConsistencyAbstractSelfTest.java |   37 +-
 ...istencyTransactionalNearEnabledSelfTest.java |    4 +-
 ...heValueConsistencyTransactionalSelfTest.java |    6 +-
 .../GridCacheVariableTopologySelfTest.java      |   48 +-
 .../cache/GridCacheVersionMultinodeTest.java    |   39 +-
 .../cache/GridCacheVersionSelfTest.java         |   16 +-
 ...ProjectionForCachesOnDaemonNodeSelfTest.java |   20 +-
 ...teAtomicCacheEntryProcessorNodeJoinTest.java |   32 +
 .../IgniteCacheAbstractStopBusySelfTest.java    |   62 +-
 .../cache/IgniteCacheAbstractTest.java          |   47 +-
 ...IgniteCacheAtomicCopyOnReadDisabledTest.java |    6 +-
 .../cache/IgniteCacheAtomicInvokeTest.java      |   14 +-
 .../cache/IgniteCacheAtomicLocalInvokeTest.java |   11 +-
 .../IgniteCacheAtomicLocalPeekModesTest.java    |    9 +-
 .../IgniteCacheAtomicLocalStoreValueTest.java   |   11 +-
 ...niteCacheAtomicLocalWithStoreInvokeTest.java |    7 +-
 .../IgniteCacheAtomicNearEnabledInvokeTest.java |    4 +-
 ...iteCacheAtomicNearEnabledStoreValueTest.java |    4 +-
 .../IgniteCacheAtomicNearPeekModesTest.java     |    2 +-
 .../cache/IgniteCacheAtomicPeekModesTest.java   |   12 +-
 ...eCacheAtomicPrimaryWriteOrderInvokeTest.java |   14 +-
 ...maryWriteOrderNearEnabledStoreValueTest.java |    4 +-
 ...heAtomicPrimaryWriteOrderStoreValueTest.java |    6 +-
 ...micPrimaryWriteOrderWithStoreInvokeTest.java |    7 +-
 ...IgniteCacheAtomicPutAllFailoverSelfTest.java |    6 +-
 ...gniteCacheAtomicReplicatedPeekModesTest.java |    6 +-
 .../IgniteCacheAtomicStopBusySelfTest.java      |    6 +-
 .../cache/IgniteCacheAtomicStoreValueTest.java  |   14 +-
 ...teCacheConfigurationDefaultTemplateTest.java |   16 +-
 .../IgniteCacheConfigurationTemplateTest.java   |   36 +-
 .../IgniteCacheContainsKeyAbstractSelfTest.java |   23 +-
 ...niteCacheCopyOnReadDisabledAbstractTest.java |   12 +-
 .../cache/IgniteCacheCreateRestartSelfTest.java |  106 +
 .../cache/IgniteCacheDynamicStopSelfTest.java   |   18 +-
 .../IgniteCacheEntryListenerAbstractTest.java   |   71 +-
 ...IgniteCacheEntryListenerAtomicLocalTest.java |   11 +-
 ...eCacheEntryListenerAtomicReplicatedTest.java |    6 +-
 .../IgniteCacheEntryListenerAtomicTest.java     |   14 +-
 ...eCacheEntryListenerEagerTtlDisabledTest.java |    6 +-
 .../IgniteCacheEntryListenerTxLocalTest.java    |   11 +-
 ...gniteCacheEntryListenerTxReplicatedTest.java |    6 +-
 .../cache/IgniteCacheEntryListenerTxTest.java   |   11 +-
 .../IgniteCacheEntryProcessorNodeJoinTest.java  |  303 ++
 .../IgniteCacheInterceptorSelfTestSuite.java    |    4 +-
 .../cache/IgniteCacheInvokeAbstractTest.java    |   47 +-
 .../cache/IgniteCacheInvokeReadThroughTest.java |   22 +-
 .../IgniteCacheManyAsyncOperationsTest.java     |  113 +
 .../cache/IgniteCacheNearLockValueSelfTest.java |   45 +-
 .../IgniteCacheP2pUnmarshallingErrorTest.java   |   23 +-
 ...gniteCacheP2pUnmarshallingNearErrorTest.java |    7 +-
 ...CacheP2pUnmarshallingRebalanceErrorTest.java |   11 +-
 .../IgniteCacheP2pUnmarshallingTxErrorTest.java |   25 +-
 .../IgniteCachePartitionMapUpdateTest.java      |   32 +-
 .../cache/IgniteCachePeekModesAbstractTest.java |   53 +-
 .../cache/IgniteCachePutAllRestartTest.java     |   42 +-
 .../cache/IgniteCacheStartStopLoadTest.java     |   20 +-
 .../IgniteCacheStoreValueAbstractTest.java      |   46 +-
 .../IgniteCacheTopologySafeGetSelfTest.java     |  222 ++
 ...gniteCacheTransactionalStopBusySelfTest.java |    9 +-
 .../IgniteCacheTxCopyOnReadDisabledTest.java    |    6 +-
 .../cache/IgniteCacheTxInvokeTest.java          |   11 +-
 .../cache/IgniteCacheTxLocalInvokeTest.java     |   11 +-
 .../cache/IgniteCacheTxLocalPeekModesTest.java  |    9 +-
 .../cache/IgniteCacheTxLocalStoreValueTest.java |   11 +-
 .../IgniteCacheTxNearEnabledInvokeTest.java     |    4 +-
 .../IgniteCacheTxNearEnabledStoreValueTest.java |    4 +-
 .../cache/IgniteCacheTxNearPeekModesTest.java   |    2 +-
 .../cache/IgniteCacheTxPeekModesTest.java       |   12 +-
 .../cache/IgniteCacheTxPreloadNoWriteTest.java  |   40 +-
 .../IgniteCacheTxReplicatedPeekModesTest.java   |    6 +-
 .../cache/IgniteCacheTxStoreValueTest.java      |   11 +-
 .../cache/IgniteCachingProviderSelfTest.java    |   18 +-
 .../IgniteClientAffinityAssignmentSelfTest.java |   34 +-
 .../IgniteDaemonNodeMarshallerCacheTest.java    |   41 +-
 .../cache/IgniteDynamicCacheAndNodeStop.java    |   95 +
 ...eDynamicCacheStartNoExchangeTimeoutTest.java |   55 +-
 .../cache/IgniteDynamicCacheStartSelfTest.java  |   57 +-
 ...niteDynamicCacheWithConfigStartSelfTest.java |   15 +-
 .../IgniteDynamicClientCacheStartSelfTest.java  |   38 +-
 .../cache/IgniteExchangeFutureHistoryTest.java  |   17 +-
 .../cache/IgniteInternalCacheTypesTest.java     |   30 +-
 .../cache/IgniteOnePhaseCommitNearSelfTest.java |  243 ++
 .../cache/IgnitePutAllLargeBatchSelfTest.java   |   37 +-
 ...tAllUpdateNonPreloadedPartitionSelfTest.java |   26 +-
 ...teStartCacheInTransactionAtomicSelfTest.java |    6 +-
 .../IgniteStartCacheInTransactionSelfTest.java  |   28 +-
 .../cache/IgniteStaticCacheStartSelfTest.java   |   14 +-
 .../cache/IgniteSystemCacheOnClientTest.java    |   25 +-
 ...gniteTopologyValidatorAbstractCacheTest.java |   66 +-
 ...iteTopologyValidatorAbstractTxCacheTest.java |   12 +-
 ...ValidatorNearPartitionedAtomicCacheTest.java |    4 +-
 ...logyValidatorNearPartitionedTxCacheTest.java |    4 +-
 ...logyValidatorPartitionedAtomicCacheTest.java |   12 +-
 ...TopologyValidatorPartitionedTxCacheTest.java |    4 +-
 ...ologyValidatorReplicatedAtomicCacheTest.java |   12 +-
 ...eTopologyValidatorReplicatedTxCacheTest.java |    4 +-
 .../processors/cache/IgniteTxAbstractTest.java  |   48 +-
 .../IgniteTxConcurrentGetAbstractTest.java      |   33 +-
 .../IgniteTxExceptionAbstractSelfTest.java      |   78 +-
 .../cache/IgniteTxMultiNodeAbstractTest.java    |   65 +-
 .../IgniteTxMultiThreadedAbstractTest.java      |   36 +-
 .../cache/IgniteTxReentryAbstractSelfTest.java  |   51 +-
 .../IgniteTxSingleThreadedAbstractTest.java     |   11 +-
 .../IgniteTxStoreExceptionAbstractSelfTest.java |   48 +-
 .../cache/OffHeapTieredTransactionSelfTest.java |   40 +-
 ...IgniteCacheAbstractExecutionContextTest.java |   17 +-
 .../IgniteCacheAtomicExecutionContextTest.java  |   12 +-
 ...niteCacheContinuousExecutionContextTest.java |    5 +-
 ...IgniteCacheIsolatedExecutionContextTest.java |    5 +-
 ...niteCacheP2PDisableExecutionContextTest.java |    4 +-
 ...iteCachePartitionedExecutionContextTest.java |   12 +-
 .../IgniteCachePrivateExecutionContextTest.java |    5 +-
 ...niteCacheReplicatedExecutionContextTest.java |   12 +-
 .../IgniteCacheSharedExecutionContextTest.java  |    5 +-
 .../IgniteCacheTxExecutionContextTest.java      |   12 +-
 ...eAbstractDataStructuresFailoverSelfTest.java |   41 +-
 ...actQueueFailoverDataConsistencySelfTest.java |   41 +-
 ...CacheAtomicReferenceApiSelfAbstractTest.java |    8 +-
 ...cheAtomicReferenceMultiNodeAbstractTest.java |   11 +-
 ...idCacheAtomicStampedApiSelfAbstractTest.java |    8 +-
 .../GridCacheMultiNodeDataStructureTest.java    |   11 +-
 .../GridCacheQueueApiSelfAbstractTest.java      |   31 +-
 .../GridCacheQueueCleanupSelfTest.java          |   41 +-
 ...ridCacheQueueJoinedNodeSelfAbstractTest.java |   36 +-
 ...GridCacheQueueMultiNodeAbstractSelfTest.java |   72 +-
 ...dCacheQueueMultiNodeConsistencySelfTest.java |   92 +-
 ...CacheQueueRotativeMultiNodeAbstractTest.java |   28 +-
 .../GridCacheSequenceApiSelfAbstractTest.java   |   43 +-
 ...dCacheSequenceMultiNodeAbstractSelfTest.java |   27 +-
 .../GridCacheSetAbstractSelfTest.java           |   45 +-
 .../GridCacheSetFailoverAbstractSelfTest.java   |   38 +-
 .../IgniteAtomicLongApiAbstractSelfTest.java    |   17 +-
 .../IgniteAtomicsAbstractTest.java              |   15 +-
 .../IgniteClientDataStructuresAbstractTest.java |   30 +-
 .../IgniteClientDataStructuresTest.java         |    2 +-
 ...IgniteClientDiscoveryDataStructuresTest.java |    2 +-
 .../IgniteCollectionAbstractTest.java           |   30 +-
 .../IgniteCountDownLatchAbstractSelfTest.java   |   44 +-
 .../IgniteDataStructureUniqueNameTest.java      |   42 +-
 .../IgniteDataStructureWithJobTest.java         |   28 +-
 .../GridCacheLocalAtomicOffheapSetSelfTest.java |    6 +-
 .../GridCacheLocalAtomicQueueApiSelfTest.java   |    9 +-
 .../local/GridCacheLocalAtomicSetSelfTest.java  |   14 +-
 .../GridCacheLocalOffheapQueueApiSelfTest.java  |    6 +-
 .../local/GridCacheLocalQueueApiSelfTest.java   |   14 +-
 .../GridCacheLocalSequenceApiSelfTest.java      |    8 +-
 .../local/GridCacheLocalSetSelfTest.java        |   14 +-
 .../local/IgniteLocalAtomicLongApiSelfTest.java |    8 +-
 .../IgniteLocalCountDownLatchSelfTest.java      |   24 +-
 ...artitionedAtomicOffheapQueueApiSelfTest.java |    6 +-
 ...omicOffheapQueueCreateMultiNodeSelfTest.java |    6 +-
 ...onedAtomicOffheapQueueMultiNodeSelfTest.java |    6 +-
 ...dCachePartitionedAtomicQueueApiSelfTest.java |    9 +-
 ...ionedAtomicQueueCreateMultiNodeSelfTest.java |    9 +-
 ...micQueueFailoverDataConsistencySelfTest.java |   11 +-
 ...PartitionedAtomicQueueMultiNodeSelfTest.java |    9 +-
 ...itionedAtomicQueueRotativeMultiNodeTest.java |    9 +-
 ...hePartitionedAtomicReferenceApiSelfTest.java |   10 +-
 ...PartitionedAtomicReferenceMultiNodeTest.java |   10 +-
 ...titionedAtomicSequenceMultiThreadedTest.java |   23 +-
 ...chePartitionedAtomicSetFailoverSelfTest.java |   11 +-
 .../GridCachePartitionedAtomicSetSelfTest.java  |    6 +-
 ...achePartitionedAtomicStampedApiSelfTest.java |   10 +-
 ...rtitionedDataStructuresFailoverSelfTest.java |   14 +-
 ...idCachePartitionedNodeRestartTxSelfTest.java |   39 +-
 ...artitionedOffHeapValuesQueueApiSelfTest.java |    6 +-
 ...achePartitionedOffHeapValuesSetSelfTest.java |    6 +-
 ...edOffheapDataStructuresFailoverSelfTest.java |    6 +-
 ...hePartitionedOffheapSetFailoverSelfTest.java |    6 +-
 .../GridCachePartitionedQueueApiSelfTest.java   |   14 +-
 ...PartitionedQueueCreateMultiNodeSelfTest.java |   46 +-
 ...dCachePartitionedQueueEntryMoveSelfTest.java |  231 +-
 ...nedQueueFailoverDataConsistencySelfTest.java |   11 +-
 ...CachePartitionedQueueJoinedNodeSelfTest.java |   14 +-
 ...dCachePartitionedQueueMultiNodeSelfTest.java |   14 +-
 ...hePartitionedQueueRotativeMultiNodeTest.java |   14 +-
 ...GridCachePartitionedSequenceApiSelfTest.java |   10 +-
 ...chePartitionedSequenceMultiNodeSelfTest.java |   10 +-
 ...GridCachePartitionedSetFailoverSelfTest.java |   11 +-
 .../GridCachePartitionedSetSelfTest.java        |   14 +-
 .../IgnitePartitionedAtomicLongApiSelfTest.java |    8 +-
 ...IgnitePartitionedCountDownLatchSelfTest.java |    8 +-
 .../IgnitePartitionedSetNoBackupsSelfTest.java  |   16 +-
 ...cheReplicatedAtomicReferenceApiSelfTest.java |    8 +-
 ...eReplicatedAtomicReferenceMultiNodeTest.java |    8 +-
 ...CacheReplicatedAtomicStampedApiSelfTest.java |    8 +-
 ...eplicatedDataStructuresFailoverSelfTest.java |   14 +-
 .../GridCacheReplicatedQueueApiSelfTest.java    |   14 +-
 ...idCacheReplicatedQueueMultiNodeSelfTest.java |   14 +-
 ...cheReplicatedQueueRotativeMultiNodeTest.java |   14 +-
 .../GridCacheReplicatedSequenceApiSelfTest.java |    8 +-
 ...acheReplicatedSequenceMultiNodeSelfTest.java |   10 +-
 .../GridCacheReplicatedSetSelfTest.java         |   14 +-
 .../IgniteReplicatedAtomicLongApiSelfTest.java  |    8 +-
 .../IgniteReplicatedCountDownLatchSelfTest.java |    8 +-
 .../distributed/CacheAffEarlySelfTest.java      |  245 ++
 ...acheAsyncOperationsFailoverAbstractTest.java |  368 ++
 .../CacheAsyncOperationsFailoverAtomicTest.java |   32 +
 .../CacheAsyncOperationsFailoverTxTest.java     |   32 +
 .../distributed/CacheAsyncOperationsTest.java   |  283 ++
 ...CacheAtomicNearUpdateTopologyChangeTest.java |   11 +-
 .../CacheGetFutureHangsSelfTest.java            |   35 +-
 ...CacheLoadingConcurrentGridStartSelfTest.java |   40 +-
 .../CacheNoValueClassOnServerNodeTest.java      |   30 +-
 .../CachePutAllFailoverAbstractTest.java        |  334 ++
 .../CachePutAllFailoverAtomicTest.java          |   30 +
 .../distributed/CachePutAllFailoverTxTest.java  |   30 +
 .../CacheTxNearUpdateTopologyChangeTest.java    |    8 +-
 ...tractDistributedByteArrayValuesSelfTest.java |   32 +-
 .../GridCacheAbstractJobExecutionTest.java      |   44 +-
 .../GridCacheAbstractNodeRestartSelfTest.java   |  206 +-
 ...tractPartitionedByteArrayValuesSelfTest.java |   17 +-
 .../GridCacheAbstractPrimarySyncSelfTest.java   |   32 +-
 .../GridCacheBasicOpAbstractTest.java           |   45 +-
 .../GridCacheClientModesAbstractSelfTest.java   |   30 +-
 ...ientModesTcpClientDiscoveryAbstractTest.java |   13 +-
 .../GridCacheEntrySetAbstractSelfTest.java      |   31 +-
 ...acheEntrySetIterationPreloadingSelfTest.java |   19 +-
 .../distributed/GridCacheEventAbstractTest.java |   51 +-
 ...heExpiredEntriesPreloadAbstractSelfTest.java |   36 +-
 .../distributed/GridCacheLockAbstractTest.java  |   50 +-
 .../distributed/GridCacheMixedModeSelfTest.java |   15 +-
 .../GridCacheModuloAffinityFunction.java        |   21 +-
 .../GridCacheMultiNodeAbstractTest.java         |   46 +-
 .../GridCacheMultiNodeLockAbstractTest.java     |   88 +-
 ...dCacheMultithreadedFailoverAbstractTest.java |   79 +-
 .../GridCacheNodeFailureAbstractTest.java       |   59 +-
 ...ridCachePartitionNotLoadedEventSelfTest.java |   76 +-
 ...dCachePartitionedAffinityFilterSelfTest.java |   38 +-
 ...chePartitionedReloadAllAbstractSelfTest.java |   51 +-
 .../GridCachePreloadEventsAbstractSelfTest.java |   41 +-
 .../GridCachePreloadLifecycleAbstractTest.java  |   30 +-
 ...GridCachePreloadRestartAbstractSelfTest.java |   40 +-
 .../GridCacheTransformEventSelfTest.java        |   70 +-
 .../distributed/IgniteCache150ClientsTest.java  |   44 +-
 .../IgniteCacheAtomicMessageRecoveryTest.java   |    6 +-
 .../IgniteCacheAtomicNodeJoinTest.java          |   11 +-
 .../IgniteCacheAtomicNodeRestartTest.java       |    8 +-
 ...niteCacheClientNodeChangingTopologyTest.java |  120 +-
 .../IgniteCacheClientNodeConcurrentStart.java   |   25 +-
 ...teCacheClientNodePartitionsExchangeTest.java |   72 +-
 .../IgniteCacheClientReconnectTest.java         |   36 +-
 .../distributed/IgniteCacheManyClientsTest.java |   49 +-
 .../IgniteCacheMessageRecoveryAbstractTest.java |   59 +-
 .../IgniteCacheNearOffheapGetSelfTest.java      |   29 +-
 .../IgniteCacheNodeJoinAbstractTest.java        |   68 +-
 .../IgniteCachePutGetRestartAbstractTest.java   |   42 +-
 .../IgniteCacheServerNodeConcurrentStart.java   |   18 +-
 .../IgniteCacheSizeFailoverTest.java            |  116 +
 .../IgniteCacheSystemTransactionsSelfTest.java  |   39 +-
 .../IgniteCacheTxFairAffinityNodeJoinTest.java  |    6 +-
 .../IgniteCacheTxMessageRecoveryTest.java       |   11 +-
 ...arDisabledFairAffinityPutGetRestartTest.java |    6 +-
 ...iteCacheTxNearDisabledPutGetRestartTest.java |    4 +-
 .../distributed/IgniteCacheTxNodeJoinTest.java  |    8 +-
 .../IgniteCrossCacheTxStoreSelfTest.java        |   39 +-
 ...iteTxConsistencyRestartAbstractSelfTest.java |   47 +-
 .../distributed/IgniteTxGetAfterStopTest.java   |   22 +-
 ...xOriginatingNodeFailureAbstractSelfTest.java |   57 +-
 ...cOriginatingNodeFailureAbstractSelfTest.java |   68 +-
 .../IgniteTxPreloadAbstractTest.java            |   41 +-
 .../IgniteTxTimeoutAbstractTest.java            |   30 +-
 ...tPartitionedOnlyByteArrayValuesSelfTest.java |   18 +-
 ...heAbstractTransformWriteThroughSelfTest.java |   42 +-
 ...acheAtomicExpiredEntriesPreloadSelfTest.java |   13 +-
 .../dht/GridCacheAtomicFullApiSelfTest.java     |   29 +-
 .../dht/GridCacheAtomicNearCacheSelfTest.java   |   58 +-
 ...idCacheAtomicNearEnabledFullApiSelfTest.java |    4 +-
 ...EnabledPrimaryWriteOrderFullApiSelfTest.java |    4 +-
 ...eAtomicPrimaryWriteOrderFullApiSelfTest.java |    6 +-
 ...tomicPrimaryWriteOrderReloadAllSelfTest.java |    6 +-
 .../dht/GridCacheAtomicReloadAllSelfTest.java   |    8 +-
 .../dht/GridCacheClientOnlySelfTest.java        |   15 +-
 .../dht/GridCacheColocatedDebugTest.java        |   64 +-
 .../dht/GridCacheColocatedFailoverSelfTest.java |   10 +-
 ...eColocatedOptimisticTransactionSelfTest.java |   34 +-
 ...ridCacheColocatedPreloadRestartSelfTest.java |    4 +-
 .../GridCacheColocatedPrimarySyncSelfTest.java  |    6 +-
 .../GridCacheColocatedReloadAllSelfTest.java    |    4 +-
 .../GridCacheColocatedTxExceptionSelfTest.java  |   12 +-
 ...ssimisticOriginatingNodeFailureSelfTest.java |   17 +-
 ...dCacheColocatedTxSingleThreadedSelfTest.java |   26 +-
 .../GridCacheDaemonNodePartitionedSelfTest.java |    6 +-
 ...cheDhtAtomicEvictionNearReadersSelfTest.java |    6 +-
 .../GridCacheDhtAtomicRemoveFailureTest.java    |   27 +-
 .../GridCacheDhtClientRemoveFailureTest.java    |    2 +-
 .../dht/GridCacheDhtEntrySelfTest.java          |   47 +-
 .../dht/GridCacheDhtEntrySetSelfTest.java       |   10 +-
 ...GridCacheDhtEvictionNearReadersSelfTest.java |   52 +-
 .../dht/GridCacheDhtEvictionSelfTest.java       |   53 +-
 .../GridCacheDhtEvictionsDisabledSelfTest.java  |   24 +-
 ...idCacheDhtExpiredEntriesPreloadSelfTest.java |   10 +-
 .../dht/GridCacheDhtInternalEntrySelfTest.java  |   43 +-
 .../dht/GridCacheDhtMappingSelfTest.java        |   30 +-
 .../dht/GridCacheDhtMultiBackupTest.java        |   18 +-
 .../dht/GridCacheDhtPreloadBigDataSelfTest.java |   39 +-
 .../dht/GridCacheDhtPreloadDelayedSelfTest.java |   65 +-
 .../GridCacheDhtPreloadDisabledSelfTest.java    |   56 +-
 .../GridCacheDhtPreloadMessageCountTest.java    |   50 +-
 ...ridCacheDhtPreloadMultiThreadedSelfTest.java |   34 +-
 .../dht/GridCacheDhtPreloadOffHeapSelfTest.java |    8 +-
 .../dht/GridCacheDhtPreloadPutGetSelfTest.java  |   44 +-
 .../dht/GridCacheDhtPreloadSelfTest.java        |   74 +-
 .../GridCacheDhtPreloadStartStopSelfTest.java   |   50 +-
 .../dht/GridCacheDhtPreloadUnloadSelfTest.java  |   40 +-
 .../dht/GridCacheDhtRemoveFailureTest.java      |   24 +-
 .../distributed/dht/GridCacheDhtTestUtils.java  |   42 +-
 .../dht/GridCacheDhtTxPreloadSelfTest.java      |   10 +-
 .../dht/GridCacheGlobalLoadTest.java            |   39 +-
 ...icOffHeapTieredMultiNodeFullApiSelfTest.java |    6 +-
 ...ledFairAffinityMultiNodeFullApiSelfTest.java |    6 +-
 ...ePartitionedNearDisabledFullApiSelfTest.java |    6 +-
 ...achePartitionedNearDisabledLockSelfTest.java |    6 +-
 ...ePartitionedNearDisabledMetricsSelfTest.java |   18 +-
 ...nedNearDisabledMultiNodeFullApiSelfTest.java |    6 +-
 ...bledMultiNodeP2PDisabledFullApiSelfTest.java |    6 +-
 ...ionedNearDisabledOffHeapFullApiSelfTest.java |    6 +-
 ...DisabledOffHeapMultiNodeFullApiSelfTest.java |    6 +-
 ...abledOffHeapTieredAtomicFullApiSelfTest.java |   13 +-
 ...earDisabledOffHeapTieredFullApiSelfTest.java |    6 +-
 ...edOffHeapTieredMultiNodeFullApiSelfTest.java |    6 +-
 ...NearDisabledOptimisticTxNodeRestartTest.java |    4 +-
 ...isabledTxOriginatingNodeFailureSelfTest.java |    4 +-
 ...dOnlyP2PDisabledByteArrayValuesSelfTest.java |    2 +-
 ...edOnlyP2PEnabledByteArrayValuesSelfTest.java |    2 +-
 ...idCachePartitionedPreloadEventsSelfTest.java |   37 +-
 ...dCachePartitionedTopologyChangeSelfTest.java |   65 +-
 ...ransformWriteThroughBatchUpdateSelfTest.java |    2 +-
 ...itionedTxOriginatingNodeFailureSelfTest.java |   19 +-
 ...ridCachePartitionedUnloadEventsSelfTest.java |   42 +-
 .../dht/GridCacheTxNodeFailureSelfTest.java     |  405 +++
 .../dht/GridNearCacheTxNodeFailureSelfTest.java |   31 +
 ...gniteAtomicLongChangingTopologySelfTest.java |  278 ++
 ...IgniteCacheContainsKeyColocatedSelfTest.java |    8 +-
 .../IgniteCacheCrossCacheTxFailoverTest.java    |  472 +++
 .../dht/IgniteCacheLockFailoverSelfTest.java    |   24 +-
 .../dht/IgniteCacheMultiTxLockSelfTest.java     |   42 +-
 ...rDisabledPrimaryNodeFailureRecoveryTest.java |    4 +-
 ...rtitionedPrimaryNodeFailureRecoveryTest.java |    4 +-
 ...woBackupsPrimaryNodeFailureRecoveryTest.java |    4 +-
 ...ePrimaryNodeFailureRecoveryAbstractTest.java |   72 +-
 .../IgniteCachePutRetryAbstractSelfTest.java    |  470 ++-
 .../dht/IgniteCachePutRetryAtomicSelfTest.java  |   85 +-
 ...gniteCachePutRetryTransactionalSelfTest.java |  209 +-
 ...teTxConsistencyColocatedRestartSelfTest.java |    8 +-
 .../dht/IgniteTxReentryColocatedSelfTest.java   |   15 +-
 ...cClientInvalidPartitionHandlingSelfTest.java |    2 +-
 .../GridCacheAtomicClientRemoveFailureTest.java |    2 +-
 .../atomic/GridCacheAtomicFailoverSelfTest.java |   16 +-
 ...eAtomicInvalidPartitionHandlingSelfTest.java |  179 +-
 .../atomic/GridCacheAtomicPreloadSelfTest.java  |   40 +-
 ...AtomicPrimaryWriteOrderFailoverSelfTest.java |    6 +-
 ...tomicPrimaryWriteOrderRemoveFailureTest.java |   29 +-
 .../GridCacheAtomicRemoveFailureTest.java       |   29 +-
 ...idCacheAtomicReplicatedFailoverSelfTest.java |    8 +-
 ...lueConsistencyAtomicNearEnabledSelfTest.java |    4 +-
 ...micPrimaryWriteOrderNearEnabledSelfTest.java |    8 +-
 ...sistencyAtomicPrimaryWriteOrderSelfTest.java |    6 +-
 ...GridCacheValueConsistencyAtomicSelfTest.java |    8 +-
 ...CacheContainsKeyColocatedAtomicSelfTest.java |    9 +-
 ...gniteCacheContainsKeyNearAtomicSelfTest.java |    7 +-
 ...PutRetryAtomicPrimaryWriteOrderSelfTest.java |   32 +
 ...tNearPartitionedByteArrayValuesSelfTest.java |    6 +-
 ...unctionExcludeNeighborsAbstractSelfTest.java |   43 +-
 ...nlyFairAffinityMultiNodeFullApiSelfTest.java |    6 +-
 ...tomicClientOnlyMultiNodeFullApiSelfTest.java |   43 +-
 ...OnlyMultiNodeP2PDisabledFullApiSelfTest.java |    4 +-
 ...yOnReadDisabledMultiNodeFullApiSelfTest.java |    4 +-
 ...micFairAffinityMultiNodeFullApiSelfTest.java |    6 +-
 ...GridCacheAtomicMultiNodeFullApiSelfTest.java |   14 +-
 ...omicMultiNodeP2PDisabledFullApiSelfTest.java |   12 +-
 ...ledFairAffinityMultiNodeFullApiSelfTest.java |    6 +-
 ...omicNearEnabledMultiNodeFullApiSelfTest.java |    4 +-
 ...imaryWriteOrderMultiNodeFullApiSelfTest.java |    4 +-
 ...ridCacheAtomicNearEvictionEventSelfTest.java |   11 +-
 .../GridCacheAtomicNearEvictionSelfTest.java    |    6 +-
 .../GridCacheAtomicNearMultiNodeSelfTest.java   |    6 +-
 ...eAtomicNearOnlyMultiNodeFullApiSelfTest.java |   27 +-
 ...OnlyMultiNodeP2PDisabledFullApiSelfTest.java |    4 +-
 .../GridCacheAtomicNearReadersSelfTest.java     |    6 +-
 .../GridCacheAtomicNearRemoveFailureTest.java   |   27 +-
 .../GridCacheAtomicOffHeapFullApiSelfTest.java  |   12 +-
 ...heAtomicOffHeapMultiNodeFullApiSelfTest.java |   12 +-
 ...icOffHeapTieredMultiNodeFullApiSelfTest.java |    6 +-
 ...idCacheAtomicPartitionedMetricsSelfTest.java |   16 +-
 ...AtomicPartitionedTckMetricsSelfTestImpl.java |    9 +-
 ...derFairAffinityMultiNodeFullApiSelfTest.java |    6 +-
 ...imaryWriteOrderMultiNodeFullApiSelfTest.java |    6 +-
 ...rderMultiNodeP2PDisabledFullApiSelfTest.java |    6 +-
 ...cPrimaryWriteOrderNearRemoveFailureTest.java |   27 +-
 ...PrimaryWriteOrderOffHeapFullApiSelfTest.java |    6 +-
 ...yWriteOrderOffHeapTieredFullApiSelfTest.java |    6 +-
 ...ityOrderOffHeapMultiNodeFullApiSelfTest.java |    6 +-
 ...erOffHeapTieredMultiNodeFullApiSelfTest.java |    6 +-
 .../near/GridCacheGetStoreErrorSelfTest.java    |   43 +-
 .../GridCacheNearEvictionEventSelfTest.java     |    9 +-
 .../near/GridCacheNearEvictionSelfTest.java     |   41 +-
 ...dCacheNearExpiredEntriesPreloadSelfTest.java |    8 +-
 .../near/GridCacheNearJobExecutionSelfTest.java |   13 +-
 .../near/GridCacheNearMetricsSelfTest.java      |   16 +-
 .../near/GridCacheNearMultiGetSelfTest.java     |   49 +-
 .../near/GridCacheNearMultiNodeSelfTest.java    |   79 +-
 .../near/GridCacheNearOneNodeSelfTest.java      |   40 +-
 ...nlyFairAffinityMultiNodeFullApiSelfTest.java |    6 +-
 ...idCacheNearOnlyMultiNodeFullApiSelfTest.java |   61 +-
 ...OnlyMultiNodeP2PDisabledFullApiSelfTest.java |    4 +-
 .../near/GridCacheNearOnlySelfTest.java         |   16 +-
 .../near/GridCacheNearOnlyTopologySelfTest.java |   47 +-
 .../GridCacheNearPartitionedClearSelfTest.java  |   35 +-
 ...ionedP2PDisabledByteArrayValuesSelfTest.java |    2 +-
 ...tionedP2PEnabledByteArrayValuesSelfTest.java |    2 +-
 .../GridCacheNearPreloadRestartSelfTest.java    |    4 +-
 .../near/GridCacheNearPrimarySyncSelfTest.java  |    6 +-
 .../GridCacheNearReaderPreloadSelfTest.java     |   34 +-
 .../near/GridCacheNearReadersSelfTest.java      |   59 +-
 .../near/GridCacheNearReloadAllSelfTest.java    |    4 +-
 .../near/GridCacheNearRemoveFailureTest.java    |   24 +-
 .../near/GridCacheNearTxExceptionSelfTest.java  |   10 +-
 .../near/GridCacheNearTxForceKeyTest.java       |   79 +
 .../near/GridCacheNearTxMultiNodeSelfTest.java  |   55 +-
 ...ssimisticOriginatingNodeFailureSelfTest.java |   15 +-
 .../near/GridCacheNearTxPreloadSelfTest.java    |   11 +-
 ...AffinityExcludeNeighborsPerformanceTest.java |   35 +-
 ...rtitionedAffinityHashIdResolverSelfTest.java |   30 +-
 .../GridCachePartitionedAffinitySelfTest.java   |   57 +-
 ...ionedAtomicGetAndTransformStoreSelfTest.java |    6 +-
 .../near/GridCachePartitionedBasicApiTest.java  |   12 +-
 .../GridCachePartitionedBasicOpSelfTest.java    |   14 +-
 ...ePartitionedBasicStoreMultiNodeSelfTest.java |   42 +-
 .../GridCachePartitionedBasicStoreSelfTest.java |    8 +-
 ...ionedClientOnlyNoPrimaryFullApiSelfTest.java |   19 +-
 ...yOnReadDisabledMultiNodeFullApiSelfTest.java |    4 +-
 .../GridCachePartitionedEntryLockSelfTest.java  |    9 +-
 .../near/GridCachePartitionedEventSelfTest.java |   15 +-
 .../GridCachePartitionedEvictionSelfTest.java   |   50 +-
 ...titionedExplicitLockNodeFailureSelfTest.java |   42 +-
 .../GridCachePartitionedFailoverSelfTest.java   |    8 +-
 ...GridCachePartitionedFilteredPutSelfTest.java |   32 +-
 ...PartitionedFullApiMultithreadedSelfTest.java |   13 +-
 .../GridCachePartitionedFullApiSelfTest.java    |   23 +-
 ...PartitionedGetAndTransformStoreSelfTest.java |    8 +-
 ...idCachePartitionedHitsAndMissesSelfTest.java |   56 +-
 .../GridCachePartitionedIteratorsSelfTest.java  |    8 +-
 .../GridCachePartitionedLoadCacheSelfTest.java  |   36 +-
 .../near/GridCachePartitionedLockSelfTest.java  |   14 +-
 .../GridCachePartitionedMetricsSelfTest.java    |   13 +-
 ...achePartitionedMultiNodeCounterSelfTest.java |   72 +-
 ...achePartitionedMultiNodeFullApiSelfTest.java |   47 +-
 ...idCachePartitionedMultiNodeLockSelfTest.java |   21 +-
 ...onedMultiNodeP2PDisabledFullApiSelfTest.java |    4 +-
 .../GridCachePartitionedMultiNodeSelfTest.java  |   13 +-
 ...ePartitionedMultiThreadedPutGetSelfTest.java |   44 +-
 ...NearDisabledBasicStoreMultiNodeSelfTest.java |    4 +-
 ...itionedNearOnlyNoPrimaryFullApiSelfTest.java |    4 +-
 .../near/GridCachePartitionedNestedTxTest.java  |   11 +-
 ...GridCachePartitionedNodeFailureSelfTest.java |   13 +-
 .../GridCachePartitionedNodeRestartTest.java    |   24 +-
 ...dCachePartitionedOffHeapFullApiSelfTest.java |    6 +-
 ...titionedOffHeapMultiNodeFullApiSelfTest.java |    6 +-
 ...edOffHeapTieredMultiNodeFullApiSelfTest.java |    9 +-
 ...ePartitionedOptimisticTxNodeRestartTest.java |   27 +-
 ...achePartitionedPreloadLifecycleSelfTest.java |   42 +-
 .../GridCachePartitionedStorePutSelfTest.java   |   34 +-
 ...GridCachePartitionedTxConcurrentGetTest.java |   11 +-
 ...GridCachePartitionedTxMultiNodeSelfTest.java |   11 +-
 ...CachePartitionedTxMultiThreadedSelfTest.java |   24 +-
 .../near/GridCachePartitionedTxReadTest.java    |    6 +-
 .../GridCachePartitionedTxSalvageSelfTest.java  |   49 +-
 ...achePartitionedTxSingleThreadedSelfTest.java |   26 +-
 .../GridCachePartitionedTxTimeoutSelfTest.java  |   18 +-
 .../near/GridCachePutArrayValueSelfTest.java    |   21 +-
 ...idCacheRendezvousAffinityClientSelfTest.java |   28 +-
 ...ffinityFunctionExcludeNeighborsSelfTest.java |    6 +-
 .../near/GridPartitionedBackupLoadSelfTest.java |   34 +-
 .../IgniteCacheContainsKeyNearSelfTest.java     |    6 +-
 .../near/IgniteCacheNearOnlyTxTest.java         |  122 +-
 .../near/IgniteCacheNearReadCommittedTest.java  |   19 +-
 .../near/IgniteCacheNearTxRollbackTest.java     |   39 +-
 .../near/IgniteTxReentryNearSelfTest.java       |   15 +-
 .../near/NoneRebalanceModeSelfTest.java         |   15 +-
 ...stractReplicatedByteArrayValuesSelfTest.java |   16 +-
 ...ridCacheAtomicReplicatedMetricsSelfTest.java |   12 +-
 .../GridCacheDaemonNodeReplicatedSelfTest.java  |    6 +-
 ...nedFairAffinityMultiNodeFullApiSelfTest.java |    8 +-
 ...ridCacheReplicatedAtomicFullApiSelfTest.java |    6 +-
 ...catedAtomicGetAndTransformStoreSelfTest.java |    6 +-
 ...eplicatedAtomicMultiNodeFullApiSelfTest.java |   11 +-
 ...imaryWriteOrderMultiNodeFullApiSelfTest.java |    6 +-
 .../GridCacheReplicatedBasicApiTest.java        |    9 +-
 .../GridCacheReplicatedBasicOpSelfTest.java     |    9 +-
 .../GridCacheReplicatedBasicStoreSelfTest.java  |    8 +-
 .../GridCacheReplicatedEntrySetSelfTest.java    |    8 +-
 .../GridCacheReplicatedEventSelfTest.java       |    8 +-
 ...ridCacheReplicatedEvictionEventSelfTest.java |    9 +-
 .../GridCacheReplicatedEvictionSelfTest.java    |   31 +-
 .../GridCacheReplicatedFailoverSelfTest.java    |   10 +-
 ...eReplicatedFullApiMultithreadedSelfTest.java |   13 +-
 ...ReplicatedFullApiMultithreadedSelfTest1.java |    4 +-
 .../GridCacheReplicatedFullApiSelfTest.java     |   12 +-
 ...eReplicatedGetAndTransformStoreSelfTest.java |    8 +-
 .../GridCacheReplicatedInvalidateSelfTest.java  |   56 +-
 .../GridCacheReplicatedIteratorsSelfTest.java   |    8 +-
 .../GridCacheReplicatedJobExecutionTest.java    |   11 +-
 .../GridCacheReplicatedLockSelfTest.java        |    8 +-
 .../GridCacheReplicatedMarshallerTxTest.java    |   11 +-
 .../GridCacheReplicatedMetricsSelfTest.java     |    9 +-
 ...CacheReplicatedMultiNodeFullApiSelfTest.java |    2 +-
 ...ridCacheReplicatedMultiNodeLockSelfTest.java |   16 +-
 ...atedMultiNodeP2PDisabledFullApiSelfTest.java |    4 +-
 .../GridCacheReplicatedMultiNodeSelfTest.java   |    9 +-
 ...licatedNearOnlyMultiNodeFullApiSelfTest.java |    8 +-
 .../GridCacheReplicatedNodeFailureSelfTest.java |   11 +-
 .../GridCacheReplicatedNodeRestartSelfTest.java |   21 +-
 ...idCacheReplicatedOffHeapFullApiSelfTest.java |    6 +-
 ...plicatedOffHeapMultiNodeFullApiSelfTest.java |    6 +-
 ...eReplicatedOffHeapTieredFullApiSelfTest.java |    6 +-
 ...edOffHeapTieredMultiNodeFullApiSelfTest.java |    6 +-
 ...catedP2PDisabledByteArrayValuesSelfTest.java |    2 +-
 ...icatedP2PEnabledByteArrayValuesSelfTest.java |    2 +-
 ...ridCacheReplicatedPreloadEventsSelfTest.java |    8 +-
 .../GridCacheReplicatedTxConcurrentGetTest.java |   11 +-
 .../GridCacheReplicatedTxExceptionSelfTest.java |   12 +-
 ...GridCacheReplicatedTxMultiNodeBasicTest.java |   10 +-
 ...dCacheReplicatedTxMultiThreadedSelfTest.java |   23 +-
 ...licatedTxOriginatingNodeFailureSelfTest.java |    7 +-
 ...ssimisticOriginatingNodeFailureSelfTest.java |   18 +-
 .../GridCacheReplicatedTxReadTest.java          |    6 +-
 ...CacheReplicatedTxSingleThreadedSelfTest.java |   22 +-
 .../GridCacheReplicatedTxTimeoutSelfTest.java   |   15 +-
 ...idCacheReplicatedUnswapAdvancedSelfTest.java |   43 +-
 .../GridCacheSyncReplicatedPreloadSelfTest.java |   34 +-
 .../replicated/GridReplicatedTxPreloadTest.java |    8 +-
 ...acheAtomicReplicatedNodeRestartSelfTest.java |   11 +-
 ...CacheReplicatedPreloadLifecycleSelfTest.java |   48 +-
 ...idCacheReplicatedPreloadOffHeapSelfTest.java |    6 +-
 .../GridCacheReplicatedPreloadSelfTest.java     |   68 +-
 ...eplicatedPreloadStartStopEventsSelfTest.java |   29 +-
 ...GridCacheReplicatedPreloadUndeploysTest.java |   13 +-
 .../cache/eviction/EvictionAbstractTest.java    |   71 +-
 .../GridCacheBatchEvictUnswapSelfTest.java      |   30 +-
 ...heConcurrentEvictionConsistencySelfTest.java |   52 +-
 .../GridCacheConcurrentEvictionsSelfTest.java   |   44 +-
 .../GridCacheDistributedEvictionsSelfTest.java  |   45 +-
 .../GridCacheEmptyEntriesAbstractSelfTest.java  |   44 +-
 .../GridCacheEmptyEntriesLocalSelfTest.java     |    6 +-
 ...ridCacheEmptyEntriesPartitionedSelfTest.java |    6 +-
 .../GridCacheEvictionFilterSelfTest.java        |   45 +-
 .../GridCacheEvictionLockUnlockSelfTest.java    |   46 +-
 .../GridCacheEvictionTouchSelfTest.java         |   51 +-
 .../cache/eviction/GridCacheMockEntry.java      |   15 +-
 ...cheSynchronousEvictionsFailoverSelfTest.java |   37 +-
 .../fifo/FifoEvictionPolicySelfTest.java        |    6 +-
 .../eviction/lru/LruEvictionPolicySelfTest.java |   10 +-
 .../lru/LruNearEvictionPolicySelfTest.java      |   65 +-
 .../LruNearOnlyNearEvictionPolicySelfTest.java  |   91 +-
 .../RandomEvictionPolicyCacheSizeSelfTest.java  |   13 +-
 .../random/RandomEvictionPolicySelfTest.java    |   16 +-
 .../SortedEvictionPolicyPerformanceTest.java    |   26 +-
 .../sorted/SortedEvictionPolicySelfTest.java    |    6 +-
 .../IgniteCacheAtomicExpiryPolicyTest.java      |   14 +-
 ...iteCacheAtomicExpiryPolicyWithStoreTest.java |   12 +-
 .../IgniteCacheAtomicLocalExpiryPolicyTest.java |   11 +-
 ...AtomicPrimaryWriteOrderExpiryPolicyTest.java |    6 +-
 ...maryWriteOrderWithStoreExpiryPolicyTest.java |    7 +-
 ...teCacheAtomicReplicatedExpiryPolicyTest.java |    6 +-
 ...iteCacheAtomicWithStoreExpiryPolicyTest.java |    7 +-
 .../IgniteCacheClientNearCacheExpiryTest.java   |   29 +-
 .../IgniteCacheExpiryPolicyAbstractTest.java    |   72 +-
 .../IgniteCacheExpiryPolicyTestSuite.java       |    6 +-
 ...eCacheExpiryPolicyWithStoreAbstractTest.java |   35 +-
 .../expiry/IgniteCacheTtlCleanupSelfTest.java   |   26 +-
 .../expiry/IgniteCacheTxExpiryPolicyTest.java   |   11 +-
 .../IgniteCacheTxExpiryPolicyWithStoreTest.java |    9 +-
 .../IgniteCacheTxLocalExpiryPolicyTest.java     |   11 +-
 ...IgniteCacheTxReplicatedExpiryPolicyTest.java |    6 +-
 .../IgniteCacheTxWithStoreExpiryPolicyTest.java |    7 +-
 .../IgniteCacheAtomicLoadAllTest.java           |   14 +-
 .../IgniteCacheAtomicLoaderWriterTest.java      |   14 +-
 .../IgniteCacheAtomicLocalLoadAllTest.java      |   11 +-
 ...CacheAtomicLocalNoLoadPreviousValueTest.java |   11 +-
 ...IgniteCacheAtomicLocalNoReadThroughTest.java |   11 +-
 ...gniteCacheAtomicLocalNoWriteThroughTest.java |   11 +-
 ...tomicNearEnabledNoLoadPreviousValueTest.java |    4 +-
 ...CacheAtomicNearEnabledNoReadThroughTest.java |    4 +-
 ...acheAtomicNearEnabledNoWriteThroughTest.java |    4 +-
 ...gniteCacheAtomicNoLoadPreviousValueTest.java |   14 +-
 .../IgniteCacheAtomicNoReadThroughTest.java     |   14 +-
 .../IgniteCacheAtomicNoWriteThroughTest.java    |   14 +-
 .../IgniteCacheAtomicStoreSessionTest.java      |   14 +-
 ...eCacheAtomicStoreSessionWriteBehindTest.java |    9 +-
 ...IgniteCacheJdbcBlobStoreNodeRestartTest.java |   14 +-
 .../IgniteCacheLoadAllAbstractTest.java         |   33 +-
 .../IgniteCacheLoaderWriterAbstractTest.java    |   35 +-
 ...iteCacheNoLoadPreviousValueAbstractTest.java |   30 +-
 .../IgniteCacheNoReadThroughAbstractTest.java   |   40 +-
 .../IgniteCacheNoWriteThroughAbstractTest.java  |   33 +-
 ...IgniteCacheStoreNodeRestartAbstractTest.java |   12 +-
 .../IgniteCacheStoreSessionAbstractTest.java    |   44 +-
 ...acheStoreSessionWriteBehindAbstractTest.java |   35 +-
 .../integration/IgniteCacheTxLoadAllTest.java   |   11 +-
 .../IgniteCacheTxLoaderWriterTest.java          |   11 +-
 .../IgniteCacheTxLocalLoadAllTest.java          |   11 +-
 ...niteCacheTxLocalNoLoadPreviousValueTest.java |   11 +-
 .../IgniteCacheTxLocalNoReadThroughTest.java    |   11 +-
 .../IgniteCacheTxLocalNoWriteThroughTest.java   |   11 +-
 ...cheTxNearEnabledNoLoadPreviousValueTest.java |    4 +-
 ...niteCacheTxNearEnabledNoReadThroughTest.java |    4 +-
 ...iteCacheTxNearEnabledNoWriteThroughTest.java |    4 +-
 .../IgniteCacheTxNoLoadPreviousValueTest.java   |   11 +-
 .../IgniteCacheTxNoReadThroughTest.java         |   11 +-
 .../IgniteCacheTxNoWriteThroughTest.java        |   11 +-
 .../IgniteCacheTxStoreSessionTest.java          |   30 +-
 ...gniteCacheTxStoreSessionWriteBehindTest.java |    6 +-
 ...CacheLocalOffHeapAndSwapMetricsSelfTest.java |   17 +-
 ...dCacheAtomicLocalMetricsNoStoreSelfTest.java |    4 +-
 .../GridCacheAtomicLocalMetricsSelfTest.java    |   12 +-
 ...dCacheAtomicLocalTckMetricsSelfTestImpl.java |    9 +-
 .../local/GridCacheDaemonNodeLocalSelfTest.java |   39 +-
 .../GridCacheLocalAtomicBasicStoreSelfTest.java |    6 +-
 .../GridCacheLocalAtomicFullApiSelfTest.java    |    6 +-
 ...LocalAtomicGetAndTransformStoreSelfTest.java |    6 +-
 ...dCacheLocalAtomicOffHeapFullApiSelfTest.java |    6 +-
 ...LocalAtomicOffHeapTieredFullApiSelfTest.java |    6 +-
 .../local/GridCacheLocalBasicApiSelfTest.java   |    9 +-
 ...cheLocalBasicStoreMultithreadedSelfTest.java |    6 +-
 .../local/GridCacheLocalBasicStoreSelfTest.java |    8 +-
 .../GridCacheLocalByteArrayValuesSelfTest.java  |   40 +-
 .../local/GridCacheLocalEventSelfTest.java      |    8 +-
 .../GridCacheLocalEvictionEventSelfTest.java    |    9 +-
 ...dCacheLocalFullApiMultithreadedSelfTest.java |   13 +-
 .../local/GridCacheLocalFullApiSelfTest.java    |   25 +-
 ...dCacheLocalGetAndTransformStoreSelfTest.java |    8 +-
 .../GridCacheLocalIsolatedNodesSelfTest.java    |   18 +-
 .../local/GridCacheLocalIteratorsSelfTest.java  |    8 +-
 .../local/GridCacheLocalLoadAllSelfTest.java    |   27 +-
 .../cache/local/GridCacheLocalLockSelfTest.java |   29 +-
 .../local/GridCacheLocalMetricsSelfTest.java    |    9 +-
 .../GridCacheLocalMultithreadedSelfTest.java    |   36 +-
 .../GridCacheLocalOffHeapFullApiSelfTest.java   |    4 +-
 .../GridCacheLocalTxExceptionSelfTest.java      |   10 +-
 .../GridCacheLocalTxMultiThreadedSelfTest.java  |   13 +-
 .../cache/local/GridCacheLocalTxReadTest.java   |    6 +-
 .../GridCacheLocalTxSingleThreadedSelfTest.java |   13 +-
 .../local/GridCacheLocalTxTimeoutSelfTest.java  |   36 +-
 ...OnlyFairAffinityMultiJvmFullApiSelfTest.java |    4 +-
 ...AtomicClientOnlyMultiJvmFullApiSelfTest.java |    4 +-
 ...tOnlyMultiJvmP2PDisabledFullApiSelfTest.java |    4 +-
 ...pyOnReadDisabledMultiJvmFullApiSelfTest.java |    4 +-
 ...omicFairAffinityMultiJvmFullApiSelfTest.java |    4 +-
 .../GridCacheAtomicMultiJvmFullApiSelfTest.java |    4 +-
 ...tomicMultiJvmP2PDisabledFullApiSelfTest.java |    4 +-
 ...bledFairAffinityMultiJvmFullApiSelfTest.java |    4 +-
 ...tomicNearEnabledMultiJvmFullApiSelfTest.java |    4 +-
 ...rimaryWriteOrderMultiJvmFullApiSelfTest.java |    4 +-
 ...heAtomicNearOnlyMultiJvmFullApiSelfTest.java |    4 +-
 ...rOnlyMultiJvmP2PDisabledFullApiSelfTest.java |    4 +-
 ...cheAtomicOffHeapMultiJvmFullApiSelfTest.java |    4 +-
 ...micOffHeapTieredMultiJvmFullApiSelfTest.java |    4 +-
 ...rderFairAffinityMultiJvmFullApiSelfTest.java |    4 +-
 ...rimaryWriteOrderMultiJvmFullApiSelfTest.java |    4 +-
 ...OrderMultiJvmP2PDisabledFullApiSelfTest.java |    4 +-
 ...rityOrderOffHeapMultiJvmFullApiSelfTest.java |    4 +-
 ...derOffHeapTieredMultiJvmFullApiSelfTest.java |    4 +-
 ...OnlyFairAffinityMultiJvmFullApiSelfTest.java |    4 +-
 ...ridCacheNearOnlyMultiJvmFullApiSelfTest.java |    4 +-
 ...rOnlyMultiJvmP2PDisabledFullApiSelfTest.java |    4 +-
 ...pyOnReadDisabledMultiJvmFullApiSelfTest.java |    4 +-
 ...onedFairAffinityMultiJvmFullApiSelfTest.java |    4 +-
 ...CachePartitionedMultiJvmFullApiSelfTest.java |    4 +-
 ...ionedMultiJvmP2PDisabledFullApiSelfTest.java |    4 +-
 ...micOffHeapTieredMultiJvmFullApiSelfTest.java |    4 +-
 ...bledFairAffinityMultiJvmFullApiSelfTest.java |    4 +-
 ...onedNearDisabledMultiJvmFullApiSelfTest.java |    4 +-
 ...abledMultiJvmP2PDisabledFullApiSelfTest.java |    4 +-
 ...rDisabledOffHeapMultiJvmFullApiSelfTest.java |    4 +-
 ...ledOffHeapTieredMultiJvmFullApiSelfTest.java |    4 +-
 ...rtitionedOffHeapMultiJvmFullApiSelfTest.java |    4 +-
 ...nedOffHeapTieredMultiJvmFullApiSelfTest.java |    4 +-
 ...ReplicatedAtomicMultiJvmFullApiSelfTest.java |    4 +-
 ...rimaryWriteOrderMultiJvmFullApiSelfTest.java |    4 +-
 ...dCacheReplicatedMultiJvmFullApiSelfTest.java |    4 +-
 ...catedMultiJvmP2PDisabledFullApiSelfTest.java |    4 +-
 ...plicatedNearOnlyMultiJvmFullApiSelfTest.java |    4 +-
 ...eplicatedOffHeapMultiJvmFullApiSelfTest.java |    4 +-
 ...tedOffHeapTieredMultiJvmFullApiSelfTest.java |    4 +-
 .../GridCacheSwapScanQueryAbstractSelfTest.java |  159 +-
 .../query/GridCacheSwapScanQuerySelfTest.java   |    2 +-
 .../cache/query/GridCircularQueueTest.java      |    9 +-
 ...ridCacheContinuousQueryAbstractSelfTest.java |  105 +-
 ...ontinuousQueryAtomicNearEnabledSelfTest.java |    4 +-
 ...ontinuousQueryAtomicP2PDisabledSelfTest.java |    2 +-
 .../GridCacheContinuousQueryAtomicSelfTest.java |    8 +-
 ...CacheContinuousQueryLocalAtomicSelfTest.java |    6 +-
 .../GridCacheContinuousQueryLocalSelfTest.java  |    6 +-
 ...eContinuousQueryPartitionedOnlySelfTest.java |    8 +-
 ...uousQueryPartitionedP2PDisabledSelfTest.java |    2 +-
 ...CacheContinuousQueryPartitionedSelfTest.java |    6 +-
 ...ContinuousQueryReplicatedAtomicSelfTest.java |    6 +-
 ...nuousQueryReplicatedP2PDisabledSelfTest.java |    2 +-
 ...dCacheContinuousQueryReplicatedSelfTest.java |   27 +-
 .../IgniteCacheContinuousQueryClientTest.java   |  141 +
 ...idCacheWriteBehindStoreAbstractSelfTest.java |   25 +-
 .../GridCacheWriteBehindStoreAbstractTest.java  |   46 +-
 .../GridCacheWriteBehindStoreLocalTest.java     |    4 +-
 ...heWriteBehindStoreMultithreadedSelfTest.java |   10 +-
 ...BehindStorePartitionedMultiNodeSelfTest.java |   43 +-
 ...ridCacheWriteBehindStorePartitionedTest.java |    4 +-
 ...GridCacheWriteBehindStoreReplicatedTest.java |    4 +-
 .../GridCacheWriteBehindStoreSelfTest.java      |   22 +-
 ...CacheClientWriteBehindStoreAbstractTest.java |   22 +-
 ...teCacheClientWriteBehindStoreAtomicTest.java |    9 +-
 .../IgnteCacheClientWriteBehindStoreTxTest.java |    6 +-
 .../CacheVersionedEntryAbstractTest.java        |  173 +
 ...nedEntryLocalAtomicSwapDisabledSelfTest.java |   46 +
 ...ersionedEntryLocalTransactionalSelfTest.java |   41 +
 ...edEntryPartitionedAtomicOffHeapSelfTest.java |   35 +
 ...VersionedEntryPartitionedAtomicSelfTest.java |   36 +
 ...PartitionedTransactionalOffHeapSelfTest.java |   36 +
 ...edEntryPartitionedTransactionalSelfTest.java |   36 +
 ...nedEntryReplicatedAtomicOffHeapSelfTest.java |   35 +
 ...eVersionedEntryReplicatedAtomicSelfTest.java |   36 +
 ...yReplicatedTransactionalOffHeapSelfTest.java |   36 +
 ...nedEntryReplicatedTransactionalSelfTest.java |   36 +
 .../clock/GridTimeSyncProcessorSelfTest.java    |   29 +-
 .../closure/GridClosureProcessorRemoteTest.java |   21 +-
 .../closure/GridClosureProcessorSelfTest.java   |   51 +-
 .../continuous/GridEventConsumeSelfTest.java    |   73 +-
 .../continuous/GridMessageListenSelfTest.java   |   43 +-
 .../DataStreamProcessorSelfTest.java            |   83 +-
 .../datastreamer/DataStreamerImplSelfTest.java  |   36 +-
 .../DataStreamerMultiThreadedSelfTest.java      |   38 +-
 .../DataStreamerMultinodeCreateCacheTest.java   |   25 +-
 .../IgniteDataStreamerPerformanceTest.java      |   34 +-
 .../processors/igfs/IgfsAbstractSelfTest.java   |  177 +-
 .../processors/igfs/IgfsAttributesSelfTest.java |   21 +-
 .../igfs/IgfsBackupFailoverSelfTest.java        |  601 ++++
 .../igfs/IgfsBackupsDualAsyncSelfTest.java      |   40 +
 .../igfs/IgfsBackupsDualSyncSelfTest.java       |   40 +
 .../igfs/IgfsBackupsPrimarySelfTest.java        |   40 +
 ...sCachePerBlockLruEvictionPolicySelfTest.java |   55 +-
 .../processors/igfs/IgfsCacheSelfTest.java      |   37 +-
 .../igfs/IgfsClientCacheSelfTest.java           |   22 +-
 .../processors/igfs/IgfsCommonAbstractTest.java |    8 +-
 .../igfs/IgfsDataManagerSelfTest.java           |   57 +-
 .../igfs/IgfsDualAbstractSelfTest.java          |   46 +-
 .../processors/igfs/IgfsDualAsyncSelfTest.java  |    4 +-
 .../processors/igfs/IgfsDualSyncSelfTest.java   |    4 +-
 .../igfs/IgfsExUniversalFileSystemAdapter.java  |   12 +-
 .../processors/igfs/IgfsFileInfoSelfTest.java   |   20 +-
 .../processors/igfs/IgfsFileMapSelfTest.java    |   21 +-
 ...IgfsGroupDataBlockKeyMapperHashSelfTest.java |   10 +-
 .../igfs/IgfsMetaManagerSelfTest.java           |   50 +-
 .../processors/igfs/IgfsMetricsSelfTest.java    |   44 +-
 .../igfs/IgfsModeResolverSelfTest.java          |   17 +-
 .../processors/igfs/IgfsModesSelfTest.java      |   48 +-
 .../processors/igfs/IgfsOneClientNodeTest.java  |   27 +-
 .../igfs/IgfsPrimaryOffheapTieredSelfTest.java  |    6 +-
 .../igfs/IgfsPrimaryOffheapValuesSelfTest.java  |    6 +-
 .../processors/igfs/IgfsPrimarySelfTest.java    |    4 +-
 .../processors/igfs/IgfsProcessorSelfTest.java  |   70 +-
 .../igfs/IgfsProcessorValidationSelfTest.java   |   54 +-
 ...IpcEndpointRegistrationAbstractSelfTest.java |   47 +-
 ...dpointRegistrationOnLinuxAndMacSelfTest.java |   10 +-
 ...pcEndpointRegistrationOnWindowsSelfTest.java |   19 +-
 .../processors/igfs/IgfsSizeSelfTest.java       |   76 +-
 .../processors/igfs/IgfsStartCacheTest.java     |   47 +-
 .../processors/igfs/IgfsStreamsSelfTest.java    |   66 +-
 .../processors/igfs/IgfsTaskSelfTest.java       |   64 +-
 .../processors/igfs/IgfsTestInputStream.java    |    5 +-
 .../igfs/UniversalFileSystemAdapter.java        |    8 +-
 .../IgfsAbstractRecordResolverSelfTest.java     |   41 +-
 ...IgfsByteDelimiterRecordResolverSelfTest.java |   10 +-
 .../IgfsFixedLengthRecordResolverSelfTest.java  |   10 +-
 ...sNewLineDelimiterRecordResolverSelfTest.java |   13 +-
 ...fsStringDelimiterRecordResolverSelfTest.java |   14 +-
 .../GridJobMetricsProcessorLoadTest.java        |   10 +-
 .../port/GridPortProcessorSelfTest.java         |   16 +-
 .../cache/GridCacheCommandHandlerSelfTest.java  |   45 +-
 .../service/ClosureServiceClientsNodesTest.java |   34 +-
 .../processors/service/DummyService.java        |   13 +-
 .../service/GridServiceClientNodeTest.java      |   18 +-
 .../GridServiceProcessorAbstractSelfTest.java   |   49 +-
 ...ServiceProcessorMultiNodeConfigSelfTest.java |   13 +-
 .../GridServiceProcessorMultiNodeSelfTest.java  |   12 +-
 .../GridServiceProcessorProxySelfTest.java      |   16 +-
 .../GridServiceProcessorSingleNodeSelfTest.java |    5 +-
 .../GridServiceReassignmentSelfTest.java        |   22 +-
 .../timeout/GridTimeoutProcessorSelfTest.java   |   25 +-
 .../product/GridProductVersionSelfTest.java     |   12 +-
 ...artupWithSpecifiedWorkDirectorySelfTest.java |   35 +-
 ...dStartupWithUndefinedIgniteHomeSelfTest.java |   31 +-
 .../internal/util/GridTestClockTimer.java       |    2 +-
 .../util/IgniteExceptionRegistrySelfTest.java   |   15 +-
 .../internal/util/IgniteUtilsSelfTest.java      |   66 +-
 .../util/future/GridCompoundFutureSelfTest.java |   17 +-
 .../util/future/GridEmbeddedFutureSelfTest.java |   25 +-
 .../util/future/GridFutureAdapterSelfTest.java  |   29 +-
 .../future/GridFutureListenPerformanceTest.java |   19 +-
 .../util/future/GridFutureQueueTest.java        |   10 +-
 .../util/future/IgniteFutureImplTest.java       |   23 +-
 .../nio/GridNioEmbeddedFutureSelfTest.java      |    9 +-
 .../util/future/nio/GridNioFutureSelfTest.java  |   24 +-
 ...GridUnsafeDataOutputArraySizingSelfTest.java |    6 +-
 .../ipc/shmem/IgfsSharedMemoryTestClient.java   |   15 +-
 .../ipc/shmem/IgfsSharedMemoryTestServer.java   |   18 +-
 .../IpcSharedMemoryCrashDetectionSelfTest.java  |   36 +-
 .../ipc/shmem/IpcSharedMemoryFakeClient.java    |    6 +-
 .../IpcSharedMemoryNativeLoaderSelfTest.java    |   15 +-
 .../ipc/shmem/IpcSharedMemoryNodeStartup.java   |   28 +-
 .../ipc/shmem/IpcSharedMemorySpaceSelfTest.java |   26 +-
 .../ipc/shmem/IpcSharedMemoryUtilsSelfTest.java |   11 +-
 .../LoadWithCorruptedLibFileTestRunner.java     |    6 +-
 .../IpcSharedMemoryBenchmarkParty.java          |    4 +-
 .../IpcSharedMemoryBenchmarkReader.java         |   22 +-
 .../IpcSharedMemoryBenchmarkWriter.java         |   21 +-
 .../nio/GridNioDelimitedBufferSelfTest.java     |    8 +-
 .../internal/util/nio/GridNioSelfTest.java      |   59 +-
 .../util/nio/GridNioSessionMetaKeySelfTest.java |   13 +-
 .../internal/util/nio/GridNioSslSelfTest.java   |   17 +-
 .../internal/util/nio/GridRoundTripTest.java    |   20 +-
 .../nio/impl/GridNioFilterChainSelfTest.java    |   27 +-
 .../util/offheap/GridByteArrayWrapper.java      |    4 +-
 .../offheap/GridOffHeapMapAbstractSelfTest.java |   47 +-
 .../GridOffHeapMapPerformanceAbstractTest.java  |   16 +-
 ...idOffHeapPartitionedMapAbstractSelfTest.java |   51 +-
 ...apPartitionedMapPerformanceAbstractTest.java |   25 +-
 .../unsafe/GridOffheapSnapTreeSelfTest.java     |   20 +-
 .../unsafe/GridUnsafeMapPerformanceTest.java    |    8 +-
 .../offheap/unsafe/GridUnsafeMapSelfTest.java   |    8 +-
 .../unsafe/GridUnsafeMemoryPerformanceTest.java |   65 +
 .../unsafe/GridUnsafeMemorySelfTest.java        |   61 +-
 ...GridUnsafePartitionedMapPerformanceTest.java |    8 +-
 .../GridUnsafePartitionedMapSelfTest.java       |    6 +-
 .../unsafe/GridUnsafeStripedLruSefTest.java     |   10 +-
 .../tostring/GridToStringBuilderSelfTest.java   |   17 +-
 .../ignite/jvmtest/BlockingQueueTest.java       |   17 +-
 .../ignite/jvmtest/ConcurrentMapTest.java       |   27 +-
 .../org/apache/ignite/jvmtest/FileIOTest.java   |   15 +-
 .../apache/ignite/jvmtest/FileLocksTest.java    |   12 +-
 .../ignite/jvmtest/LinkedHashMapTest.java       |   10 +-
 .../ignite/jvmtest/MultipleFileIOTest.java      |   11 +-
 .../ignite/jvmtest/NetworkFailureTest.java      |   29 +-
 .../QueueSizeCounterMultiThreadedTest.java      |   22 +-
 .../jvmtest/ReadWriteLockMultiThreadedTest.java |   18 +-
 .../org/apache/ignite/jvmtest/RegExpTest.java   |   12 +-
 .../jvmtest/ServerSocketMultiThreadedTest.java  |   23 +-
 .../ignite/lang/GridBasicPerformanceTest.java   |   49 +-
 .../ignite/lang/GridByteArrayListSelfTest.java  |   17 +-
 .../ignite/lang/GridFuncPerformanceTest.java    |   13 +-
 .../lang/GridFutureListenPerformanceTest.java   |   20 +-
 .../GridMetadataAwareAdapterLoadSelfTest.java   |   13 +-
 .../lang/GridMetadataAwareAdapterSelfTest.java  |   10 +-
 .../ignite/lang/GridSetWrapperSelfTest.java     |   16 +-
 .../lang/GridSystemCurrentTimeMillisTest.java   |    2 +-
 .../ignite/lang/GridThreadPriorityTest.java     |    4 +-
 .../apache/ignite/lang/GridTupleSelfTest.java   |   15 +-
 .../org/apache/ignite/lang/GridXSelfTest.java   |   16 +-
 .../apache/ignite/lang/IgniteUuidSelfTest.java  |   20 +-
 ...dBoundedConcurrentLinkedHashMapSelfTest.java |   10 +-
 ...GridBoundedConcurrentOrderedMapSelfTest.java |   14 +-
 .../GridCircularBufferPerformanceTest.java      |   18 +-
 .../lang/utils/GridCircularBufferSelfTest.java  |   13 +-
 .../utils/GridConcurrentHashMapSelfTest.java    |   13 +-
 .../GridConcurrentLinkedHashMapSelfTest.java    |   14 +-
 .../GridConcurrentWeakHashSetSelfTest.java      |   20 +-
 .../lang/utils/GridConsistentHashSelfTest.java  |   23 +-
 .../lang/utils/GridLeanIdentitySetSelfTest.java |   10 +-
 .../lang/utils/GridLeanMapPerformanceTest.java  |    9 +-
 .../ignite/lang/utils/GridLeanMapSelfTest.java  |   16 +-
 .../ignite/lang/utils/GridListSetSelfTest.java  |   15 +-
 .../lang/utils/GridStripedLockSelfTest.java     |   16 +-
 .../ignite/loadtest/GridLoadTestStatistics.java |   16 +-
 .../loadtest/GridSingleExecutionTest.java       |   59 +-
 ...idFileSwapSpaceSpiMultithreadedLoadTest.java |   33 +-
 .../loadtests/GridCacheLoadPopulationTask.java  |   20 +-
 .../loadtests/GridCacheMultiNodeLoadTest.java   |   25 +-
 .../cache/GridCacheAbstractLoadTest.java        |   58 +-
 .../loadtests/cache/GridCacheBenchmark.java     |   20 +-
 .../cache/GridCacheDataStructuresLoadTest.java  |   32 +-
 .../loadtests/cache/GridCacheLoadTest.java      |   19 +-
 .../loadtests/cache/GridCacheSwapLoadTest.java  |   39 +-
 .../GridCacheWriteBehindStoreLoadTest.java      |   33 +-
 .../capacity/GridCapacityLoadTest.java          |   17 +-
 .../colocation/GridTestCacheStore.java          |   24 +-
 .../loadtests/colocation/GridTestConstants.java |    2 +-
 .../loadtests/colocation/GridTestKey.java       |   11 +-
 .../colocation/GridTestLifecycleBean.java       |   10 +-
 .../loadtests/colocation/GridTestMain.java      |   33 +-
 .../communication/GridIoManagerBenchmark.java   |   54 +-
 .../communication/GridIoManagerBenchmark0.java  |   57 +-
 .../GridTcpCommunicationBenchmark.java          |    2 +-
 .../communication/GridTestMessage.java          |   18 +-
 .../GridContinuousOperationsLoadTest.java       |   60 +-
 .../GridCachePartitionedAtomicLongLoadTest.java |   42 +-
 .../direct/multisplit/GridLoadTestJob.java      |   11 +-
 .../multisplit/GridLoadTestJobTarget.java       |    2 +-
 .../direct/multisplit/GridLoadTestTask.java     |   23 +-
 .../multisplit/GridMultiSplitsLoadTest.java     |   31 +-
 .../GridMultiSplitsRedeployLoadTest.java        |   22 +-
 .../GridSingleSplitNewNodesTestJobTarget.java   |    6 +-
 .../GridSingleSplitNewNodesTestTask.java        |   24 +-
 ...ridSingleSplitsNewNodesAbstractLoadTest.java |   27 +-
 ...idSingleSplitsNewNodesMulticastLoadTest.java |   10 +-
 .../GridSingleSplitsRedeployLoadTest.java       |   32 +-
 .../direct/session/GridSessionLoadTest.java     |   20 +-
 .../direct/session/GridSessionLoadTestJob.java  |   13 +-
 .../direct/session/GridSessionLoadTestTask.java |   25 +-
 .../direct/stealing/GridStealingLoadTest.java   |   31 +-
 .../stealing/GridStealingLoadTestJob.java       |   17 +-
 .../stealing/GridStealingLoadTestTask.java      |   20 +-
 .../loadtests/discovery/GridGcTimeoutTest.java  |   13 +-
 .../ignite/loadtests/dsi/GridDsiClient.java     |   40 +-
 .../loadtests/dsi/GridDsiLifecycleBean.java     |   13 +-
 .../ignite/loadtests/dsi/GridDsiMessage.java    |    6 +-
 .../ignite/loadtests/dsi/GridDsiPerfJob.java    |   44 +-
 .../ignite/loadtests/dsi/GridDsiRequest.java    |    7 +-
 .../loadtests/dsi/GridDsiRequestTask.java       |   13 +-
 .../ignite/loadtests/dsi/GridDsiResponse.java   |    7 +-
 .../loadtests/dsi/GridDsiResponseTask.java      |   13 +-
 .../ignite/loadtests/dsi/GridDsiSession.java    |    7 +-
 .../cacheget/GridBenchmarkCacheGetLoadTest.java |    8 +-
 ...dBoundedConcurrentLinkedHashSetLoadTest.java |   29 +-
 .../loadtests/hashmap/GridCacheTestContext.java |   45 +-
 .../loadtests/hashmap/GridHashMapLoadTest.java  |   20 +-
 .../job/GridJobExecutionLoadTestClient.java     |   31 +-
 ...GridJobExecutionLoadTestClientSemaphore.java |   40 +-
 .../job/GridJobExecutionLoadTestJob.java        |   10 +-
 .../job/GridJobExecutionLoadTestServer.java     |    4 +-
 .../job/GridJobExecutionLoadTestTask.java       |   20 +-
 .../job/GridJobExecutionSingleNodeLoadTest.java |   44 +-
 ...JobExecutionSingleNodeSemaphoreLoadTest.java |   50 +-
 .../ignite/loadtests/job/GridJobLoadTest.java   |   29 +-
 .../loadtests/job/GridJobLoadTestJob.java       |   23 +-
 .../loadtests/job/GridJobLoadTestParams.java    |    2 +-
 .../loadtests/job/GridJobLoadTestSubmitter.java |   16 +-
 .../loadtests/job/GridJobLoadTestTask.java      |   20 +-
 .../GridConcurrentLinkedHashMapBenchmark.java   |   13 +-
 .../mapper/GridContinuousMapperLoadTest1.java   |    8 +-
 .../mapper/GridContinuousMapperLoadTest2.java   |   13 +-
 .../mapper/GridContinuousMapperTask1.java       |   32 +-
 .../mapper/GridContinuousMapperTask2.java       |   26 +-
 .../loadtests/mapper/GridNodeStartup.java       |   11 +-
 .../ignite/loadtests/mapper/TestObject.java     |    7 +-
 .../mergesort/GridMergeSortLoadTask.java        |   24 +-
 .../mergesort/GridMergeSortLoadTest.java        |   19 +-
 .../loadtests/nio/GridNioBenchmarkClient.java   |   17 +-
 .../loadtests/nio/GridNioBenchmarkTest.java     |   25 +-
 .../GridOffHeapMapPerformanceAbstractTest.java  |   20 +-
 ...apPartitionedMapPerformanceAbstractTest.java |   28 +-
 .../unsafe/GridUnsafeMapPerformanceTest.java    |    8 +-
 ...GridUnsafePartitionedMapPerformanceTest.java |    8 +-
 .../swap/GridSwapEvictAllBenchmark.java         |   45 +-
 .../loadtests/util/GridCumulativeAverage.java   |    2 +-
 .../ignite/loadtests/util/GridLoadTestArgs.java |   14 +-
 .../ignite/logger/java/JavaLoggerTest.java      |   15 +-
 .../marshaller/GridMarshallerAbstractTest.java  |   82 +-
 .../GridMarshallerExternalizableBean.java       |   10 +-
 .../GridMarshallerPerformanceTest.java          |   43 +-
 .../marshaller/GridMarshallerResourceBean.java  |   37 +-
 .../GridMarshallerTestAbstractBean.java         |    2 +-
 .../marshaller/GridMarshallerTestBean.java      |    9 +-
 .../GridMarshallerTestInheritedBean.java        |    2 +-
 .../marshaller/MarshallerContextTestImpl.java   |   15 +-
 .../jdk/GridJdkMarshallerSelfTest.java          |    7 +-
 .../OptimizedMarshallerEnumSelfTest.java        |    6 +-
 .../OptimizedMarshallerNodeFailoverTest.java    |  134 +-
 .../optimized/OptimizedMarshallerSelfTest.java  |   21 +-
 ...arshallerSerialPersistentFieldsSelfTest.java |   12 +-
 .../optimized/OptimizedMarshallerTest.java      |   40 +-
 .../OptimizedObjectStreamSelfTest.java          |   73 +-
 .../TestTcpDiscoveryIpFinderAdapter.java        |   11 +-
 ...GridMessagingNoPeerClassLoadingSelfTest.java |   20 +-
 .../ignite/messaging/GridMessagingSelfTest.java |   62 +-
 .../IgniteMessagingWithClientTest.java          |   39 +-
 .../p2p/GridAbstractMultinodeRedeployTest.java  |   24 +-
 ...MultinodeRedeployContinuousModeSelfTest.java |    6 +-
 ...idMultinodeRedeployIsolatedModeSelfTest.java |    6 +-
 ...ridMultinodeRedeployPrivateModeSelfTest.java |    6 +-
 ...GridMultinodeRedeploySharedModeSelfTest.java |    6 +-
 .../ignite/p2p/GridP2PClassLoadingSelfTest.java |   36 +-
 .../GridP2PContinuousDeploymentSelfTest.java    |   25 +-
 .../GridP2PDifferentClassLoaderSelfTest.java    |   19 +-
 .../p2p/GridP2PDoubleDeploymentSelfTest.java    |   23 +-
 .../p2p/GridP2PHotRedeploymentSelfTest.java     |   15 +-
 .../p2p/GridP2PJobClassLoaderSelfTest.java      |   22 +-
 .../p2p/GridP2PLocalDeploymentSelfTest.java     |   33 +-
 .../GridP2PMissedResourceCacheSizeSelfTest.java |   31 +-
 .../ignite/p2p/GridP2PNodeLeftSelfTest.java     |   17 +-
 .../p2p/GridP2PRecursionTaskSelfTest.java       |   25 +-
 .../p2p/GridP2PRemoteClassLoadersSelfTest.java  |   32 +-
 .../p2p/GridP2PSameClassLoaderSelfTest.java     |   19 +-
 .../org/apache/ignite/p2p/GridP2PTestJob.java   |   17 +-
 .../org/apache/ignite/p2p/GridP2PTestTask.java  |   22 +-
 .../p2p/GridP2PTestTaskExecutionTest.java       |   12 +-
 .../ignite/p2p/GridP2PTimeoutSelfTest.java      |   22 +-
 .../ignite/p2p/GridP2PUndeploySelfTest.java     |   27 +-
 .../ignite/p2p/GridSwapSpaceCustomKey.java      |    7 +-
 .../ignite/p2p/GridSwapSpaceCustomValue.java    |    7 +-
 ...SessionCancelSiblingsFromFutureSelfTest.java |   44 +-
 ...ridSessionCancelSiblingsFromJobSelfTest.java |   49 +-
 ...idSessionCancelSiblingsFromTaskSelfTest.java |   47 +-
 .../GridSessionCheckpointAbstractSelfTest.java  |   36 +-
 .../session/GridSessionCheckpointSelfTest.java  |   15 +-
 .../GridSessionCollisionSpiSelfTest.java        |   37 +-
 ...idSessionFutureWaitJobAttributeSelfTest.java |   37 +-
 ...dSessionFutureWaitTaskAttributeSelfTest.java |   38 +-
 .../session/GridSessionJobFailoverSelfTest.java |   31 +-
 ...GridSessionJobWaitTaskAttributeSelfTest.java |   41 +-
 .../ignite/session/GridSessionLoadSelfTest.java |   42 +-
 .../GridSessionSetFutureAttributeSelfTest.java  |   44 +-
 ...nSetFutureAttributeWaitListenerSelfTest.java |   42 +-
 .../GridSessionSetJobAttribute2SelfTest.java    |   28 +-
 ...GridSessionSetJobAttributeOrderSelfTest.java |   31 +-
 .../GridSessionSetJobAttributeSelfTest.java     |   41 +-
 ...sionSetJobAttributeWaitListenerSelfTest.java |   43 +-
 .../GridSessionSetTaskAttributeSelfTest.java    |   35 +-
 ...GridSessionTaskWaitJobAttributeSelfTest.java |   35 +-
 .../GridSessionWaitAttributeSelfTest.java       |   43 +-
 .../ignite/session/GridThreadSerialNumber.java  |    2 +-
 .../spi/GridSpiLocalHostInjectionTest.java      |   16 +-
 .../spi/GridSpiStartStopAbstractTest.java       |    4 +-
 .../org/apache/ignite/spi/GridTcpForwarder.java |   15 +-
 .../spi/GridTcpSpiForwardingSelfTest.java       |   35 +-
 .../GridCheckpointSpiAbstractTest.java          |    6 +-
 .../spi/checkpoint/GridCheckpointTestState.java |    4 +-
 .../cache/CacheCheckpointSpiConfigSelfTest.java |    5 +-
 .../CacheCheckpointSpiSecondCacheSelfTest.java  |   19 +-
 .../cache/CacheCheckpointSpiSelfTest.java       |    6 +-
 .../CacheCheckpointSpiStartStopSelfTest.java    |    6 +-
 .../jdbc/JdbcCheckpointSpiConfigSelfTest.java   |   19 +-
 .../JdbcCheckpointSpiCustomConfigSelfTest.java  |    8 +-
 .../JdbcCheckpointSpiDefaultConfigSelfTest.java |    8 +-
 .../JdbcCheckpointSpiStartStopSelfTest.java     |    8 +-
 ...GridSharedFsCheckpointSpiConfigSelfTest.java |    8 +-
 ...redFsCheckpointSpiMultiThreadedSelfTest.java |   26 +-
 ...heckpointSpiMultipleDirectoriesSelfTest.java |   19 +-
 .../GridSharedFsCheckpointSpiSelfTest.java      |   15 +-
 ...dSharedFsCheckpointSpiStartStopSelfTest.java |    6 +-
 .../spi/collision/GridCollisionTestContext.java |    5 +-
 .../collision/GridTestCollisionJobContext.java  |   12 +-
 .../collision/GridTestCollisionTaskSession.java |   15 +-
 ...GridFifoQueueCollisionSpiConfigSelfTest.java |    5 +-
 .../GridFifoQueueCollisionSpiSelfTest.java      |   17 +-
 ...dFifoQueueCollisionSpiStartStopSelfTest.java |    6 +-
 ...bStealingCollisionSpiAttributesSelfTest.java |   36 +-
 ...idJobStealingCollisionSpiConfigSelfTest.java |    5 +-
 ...alingCollisionSpiCustomTopologySelfTest.java |   42 +-
 .../GridJobStealingCollisionSpiSelfTest.java    |   47 +-
 ...obStealingCollisionSpiStartStopSelfTest.java |   18 +-
 ...PriorityQueueCollisionSpiConfigSelfTest.java |    5 +-
 .../GridPriorityQueueCollisionSpiSelfTest.java  |   27 +-
 ...orityQueueCollisionSpiStartStopSelfTest.java |    6 +-
 .../GridAbstractCommunicationSelfTest.java      |   47 +-
 .../communication/GridCacheMessageSelfTest.java |   46 +-
 .../spi/communication/GridTestMessage.java      |   13 +-
 .../tcp/GridCacheDhtLockBackupSelfTest.java     |   53 +-
 .../GridTcpCommunicationSpiAbstractTest.java    |   20 +-
 ...mmunicationSpiConcurrentConnectSelfTest.java |   59 +-
 .../GridTcpCommunicationSpiConfigSelfTest.java  |    5 +-
 .../tcp/GridTcpCommunicationSpiLanTest.java     |   41 +-
 ...cpCommunicationSpiMultithreadedSelfTest.java |   68 +-
 ...pCommunicationSpiMultithreadedShmemTest.java |    2 +-
 ...dTcpCommunicationSpiRecoveryAckSelfTest.java |   51 +-
 ...tionSpiRecoveryFailureDetectionSelfTest.java |    4 +-
 ...GridTcpCommunicationSpiRecoverySelfTest.java |   56 +-
 .../GridTcpCommunicationSpiShmemSelfTest.java   |    4 +-
 .../tcp/GridTcpCommunicationSpiSslSelfTest.java |    4 +-
 ...ridTcpCommunicationSpiStartStopSelfTest.java |    9 +-
 ...unicationSpiTcpFailureDetectionSelfTest.java |    6 +-
 ...cpCommunicationSpiTcpNoDelayOffSelfTest.java |    2 +-
 .../tcp/GridTcpCommunicationSpiTcpSelfTest.java |    4 +-
 .../tcp/IgniteCacheSslStartStopSelfTest.java    |   20 +-
 ...CommunicationRecoveryAckClosureSelfTest.java |   54 +-
 .../local/GridLocalDeploymentSpiSelfTest.java   |   27 +-
 ...GridLocalDeploymentSpiStartStopSelfTest.java |    6 +-
 .../AbstractDiscoveryRandomStartStopTest.java   |   23 +-
 .../discovery/AbstractDiscoverySelfTest.java    |   53 +-
 .../spi/discovery/AbstractDiscoveryTest.java    |   22 +-
 ...ClusterMetricsSnapshotSerializeSelfTest.java |    9 +-
 ...pClientDiscoveryMarshallerCheckSelfTest.java |   18 +-
 ...lientDiscoverySpiFailureTimeoutSelfTest.java |   18 +-
 .../tcp/TcpClientDiscoverySpiMulticastTest.java |   17 +-
 .../tcp/TcpClientDiscoverySpiSelfTest.java      |   94 +-
 .../tcp/TcpDiscoveryConcurrentStartTest.java    |   19 +-
 .../TcpDiscoveryMarshallerCheckSelfTest.java    |   18 +-
 .../tcp/TcpDiscoveryMultiThreadedTest.java      |   39 +-
 ...DiscoveryNodeConfigConsistentIdSelfTest.java |   13 +-
 .../TcpDiscoveryNodeConsistentIdSelfTest.java   |   14 +-
 .../discovery/tcp/TcpDiscoveryRestartTest.java  |   41 +-
 .../spi/discovery/tcp/TcpDiscoverySelfTest.java |  237 +-
 .../tcp/TcpDiscoverySnapshotHistoryTest.java    |   15 +-
 .../tcp/TcpDiscoverySpiConfigSelfTest.java      |    6 +-
 .../TcpDiscoverySpiFailureTimeoutSelfTest.java  |   42 +-
 .../tcp/TcpDiscoverySpiRandomStartStopTest.java |   10 +-
 .../discovery/tcp/TcpDiscoverySpiSelfTest.java  |   11 +-
 .../tcp/TcpDiscoverySpiSslSelfTest.java         |    2 +-
 .../tcp/TcpDiscoverySpiStartStopSelfTest.java   |   22 +-
 .../tcp/TcpDiscoverySpiWildcardSelfTest.java    |   10 +-
 .../discovery/tcp/TcpDiscoverySslSelfTest.java  |    6 +-
 .../TcpDiscoveryIpFinderAbstractSelfTest.java   |   19 +-
 .../jdbc/TcpDiscoveryJdbcIpFinderSelfTest.java  |    8 +-
 .../TcpDiscoveryMulticastIpFinderSelfTest.java  |   12 +-
 .../TcpDiscoverySharedFsIpFinderSelfTest.java   |    9 +-
 .../vm/TcpDiscoveryVmIpFinderSelfTest.java      |   10 +-
 ...MemoryEventStorageMultiThreadedSelfTest.java |   17 +-
 ...GridMemoryEventStorageSpiConfigSelfTest.java |    5 +-
 .../GridMemoryEventStorageSpiSelfTest.java      |   17 +-
 ...dMemoryEventStorageSpiStartStopSelfTest.java |    6 +-
 .../spi/failover/GridFailoverTestContext.java   |   11 +-
 .../GridAlwaysFailoverSpiConfigSelfTest.java    |    5 +-
 .../always/GridAlwaysFailoverSpiSelfTest.java   |   26 +-
 .../GridAlwaysFailoverSpiStartStopSelfTest.java |    8 +-
 ...ridJobStealingFailoverSpiConfigSelfTest.java |    5 +-
 ...idJobStealingFailoverSpiOneNodeSelfTest.java |   25 +-
 .../GridJobStealingFailoverSpiSelfTest.java     |   35 +-
 ...JobStealingFailoverSpiStartStopSelfTest.java |   18 +-
 .../never/GridNeverFailoverSpiSelfTest.java     |   19 +-
 .../GridNeverFailoverSpiStartStopSelfTest.java  |    8 +-
 ...dAdaptiveLoadBalancingSpiConfigSelfTest.java |    5 +-
 ...iveLoadBalancingSpiMultipleNodeSelfTest.java |   22 +-
 .../GridAdaptiveLoadBalancingSpiSelfTest.java   |   23 +-
 ...aptiveLoadBalancingSpiStartStopSelfTest.java |    6 +-
 ...alancingNotPerTaskMultithreadedSelfTest.java |   28 +-
 ...dRobinLoadBalancingSpiLocalNodeSelfTest.java |   15 +-
 ...inLoadBalancingSpiMultipleNodesSelfTest.java |   28 +-
 ...RobinLoadBalancingSpiNotPerTaskSelfTest.java |   34 +-
 ...dRobinLoadBalancingSpiStartStopSelfTest.java |    6 +-
 ...nLoadBalancingSpiTopologyChangeSelfTest.java |   26 +-
 .../roundrobin/GridRoundRobinTestUtils.java     |   15 +-
 ...tedRandomLoadBalancingSpiConfigSelfTest.java |    5 +-
 ...dWeightedRandomLoadBalancingSpiSelfTest.java |   18 +-
 ...RandomLoadBalancingSpiStartStopSelfTest.java |    6 +-
 ...dRandomLoadBalancingSpiWeightedSelfTest.java |   28 +-
 .../GridSwapSpaceSpiAbstractSelfTest.java       |   40 +-
 .../file/GridFileSwapCompactionSelfTest.java    |   18 +-
 .../file/GridFileSwapSpaceSpiSelfTest.java      |   34 +-
 .../inmemory/GridTestSwapSpaceSpi.java          |   40 +-
 .../noop/GridNoopSwapSpaceSpiSelfTest.java      |   14 +-
 .../startup/GridRandomCommandLineLoader.java    |   54 +-
 .../ignite/startup/GridVmNodesStarter.java      |   47 +-
 .../cmdline/GridCommandLineLoaderTest.java      |   17 +-
 .../GridCommandLineTransformerSelfTest.java     |    9 +-
 .../startup/servlet/GridServletLoaderTest.java  |   21 +-
 .../stream/socket/SocketStreamerSelfTest.java   |   79 +-
 .../ignite/testframework/GridFileLock.java      |   16 +-
 .../testframework/GridJarClassLoader.java       |   18 +-
 .../ignite/testframework/GridLoadTestUtils.java |   22 +-
 .../testframework/GridSpiTestContext.java       |   66 +-
 .../ignite/testframework/GridStringLogger.java  |    8 +-
 .../testframework/GridTestClassLoader.java      |   13 +-
 .../GridTestExternalClassLoader.java            |   19 +-
 .../ignite/testframework/GridTestNode.java      |   24 +-
 .../GridTestSafeThreadFactory.java              |   26 +-
 .../ignite/testframework/GridTestThread.java    |    4 +-
 .../ignite/testframework/GridTestUtils.java     |  223 +-
 .../config/GridTestProperties.java              |   24 +-
 .../http/GridEmbeddedHttpServer.java            |   20 +-
 .../testframework/junits/GridAbstractTest.java  |  125 +-
 .../junits/GridTestKernalContext.java           |   24 +-
 .../ignite/testframework/junits/IgniteMock.java |   46 +-
 .../junits/IgniteTestResources.java             |   38 +-
 .../cache/GridAbstractCacheStoreSelfTest.java   |   44 +-
 .../junits/cache/TestCacheSession.java          |   13 +-
 .../cache/TestThreadLocalCacheSession.java      |   11 +-
 .../junits/common/GridAbstractExamplesTest.java |    9 +-
 .../GridAbstractLifecycleAwareSelfTest.java     |   14 +-
 .../junits/common/GridCommonAbstractTest.java   |  196 +-
 .../junits/common/GridCommonTest.java           |    9 +-
 .../logger/GridLog4jRollingFileAppender.java    |   18 +-
 .../junits/logger/GridTestLog4jLogger.java      |   49 +-
 .../junits/multijvm/AffinityProcessProxy.java   |   18 +-
 .../multijvm/IgniteCacheProcessProxy.java       |   50 +-
 .../multijvm/IgniteClusterProcessProxy.java     |   31 +-
 .../multijvm/IgniteEventsProcessProxy.java      |   22 +-
 .../junits/multijvm/IgniteNodeRunner.java       |   46 +-
 .../junits/multijvm/IgniteProcessProxy.java     |   88 +-
 .../junits/spi/GridSpiAbstractConfigTest.java   |   13 +-
 .../junits/spi/GridSpiAbstractTest.java         |   66 +-
 .../testframework/junits/spi/GridSpiTest.java   |   16 +-
 .../junits/spi/GridSpiTestConfig.java           |    8 +-
 .../ignite/testsuites/IgniteBasicTestSuite.java |   47 +-
 .../IgniteCacheDataStructuresSelfTestSuite.java |   67 +-
 .../IgniteCacheEvictionSelfTestSuite.java       |   32 +-
 .../IgniteCacheFailoverTestSuite.java           |   48 +-
 .../IgniteCacheFailoverTestSuite2.java          |   30 +-
 .../IgniteCacheFailoverTestSuite3.java          |   62 +
 ...IgniteCacheFullApiMultiJvmSelfTestSuite.java |   46 +-
 .../IgniteCacheFullApiSelfTestSuite.java        |   86 +-
 .../IgniteCacheIteratorsSelfTestSuite.java      |   10 +-
 .../IgniteCacheMetricsSelfTestSuite.java        |   21 +-
 .../IgniteCacheNearOnlySelfTestSuite.java       |    9 +-
 ...niteCacheP2pUnmarshallingErrorTestSuite.java |   14 +-
 .../testsuites/IgniteCacheRestartTestSuite.java |   23 +-
 .../IgniteCacheRestartTestSuite2.java           |   47 +
 .../IgniteCacheTcpClientDiscoveryTestSuite.java |    8 +-
 .../ignite/testsuites/IgniteCacheTestSuite.java |  248 +-
 .../testsuites/IgniteCacheTestSuite2.java       |  116 +-
 .../testsuites/IgniteCacheTestSuite3.java       |   72 +-
 .../testsuites/IgniteCacheTestSuite4.java       |  124 +-
 .../IgniteCacheTxRecoverySelfTestSuite.java     |   16 +-
 .../IgniteCacheWriteBehindTestSuite.java        |   15 +-
 .../testsuites/IgniteClientNodesTestSuite.java  |    8 +-
 .../IgniteClientReconnectTestSuite.java         |   16 +-
 .../testsuites/IgniteClientTestSuite.java       |    6 +-
 .../testsuites/IgniteComputeGridTestSuite.java  |   62 +-
 .../IgniteContinuousTaskSelfTestSuite.java      |    9 +-
 .../IgniteExternalizableSelfTestSuite.java      |    6 +-
 .../ignite/testsuites/IgniteIgfsTestSuite.java  |   50 +-
 .../IgniteIpcSharedMemorySelfTestSuite.java     |    9 +-
 .../IgniteJobMetricsSelfTestSuite.java          |    6 +-
 .../testsuites/IgniteKernalSelfTestSuite.java   |   55 +-
 .../testsuites/IgniteLangSelfTestSuite.java     |   34 +-
 .../testsuites/IgniteLoggingSelfTestSuite.java  |    5 +-
 .../IgniteMarshallerSelfTestSuite.java          |   20 +-
 .../testsuites/IgniteP2PSelfTestSuite.java      |   21 +-
 .../testsuites/IgniteRestHandlerTestSuite.java  |    6 +-
 .../IgniteSpiCheckpointSelfTestSuite.java       |   19 +-
 .../IgniteSpiCollisionSelfTestSuite.java        |   18 +-
 .../IgniteSpiCommunicationSelfTestSuite.java    |   18 +-
 .../IgniteSpiDeploymentSelfTestSuite.java       |    7 +-
 .../IgniteSpiDiscoverySelfTestSuite.java        |   31 +-
 .../IgniteSpiEventStorageSelfTestSuite.java     |    9 +-
 .../IgniteSpiFailoverSelfTestSuite.java         |   16 +-
 .../IgniteSpiIndexingSelfTestSuite.java         |    6 +-
 .../IgniteSpiLoadBalancingSelfTestSuite.java    |   22 +-
 .../IgniteSpiSwapSpaceSelfTestSuite.java        |    9 +-
 .../ignite/testsuites/IgniteSpiTestSuite.java   |    6 +-
 .../testsuites/IgniteStartUpTestSuite.java      |    6 +-
 .../testsuites/IgniteStreamSelfTestSuite.java   |    7 +-
 .../IgniteTaskSessionSelfTestSuite.java         |   24 +-
 .../IgniteTimeoutProcessorSelfTestSuite.java    |    7 +-
 .../testsuites/IgniteUtilSelfTestSuite.java     |   41 +-
 .../GridThreadPoolExecutorServiceSelfTest.java  |   21 +-
 .../apache/ignite/thread/GridThreadTest.java    |    9 +-
 ...dConcurrentLinkedDequeMultiThreadedTest.java |   19 +-
 .../util/GridConcurrentLinkedDequeSelfTest.java |   11 +-
 ...rrentLinkedHashMapMultiThreadedSelfTest.java |   34 +-
 .../apache/ignite/util/GridIndexFillTest.java   |   21 +-
 .../apache/ignite/util/GridLogThrottleTest.java |    7 +-
 .../ignite/util/GridLongListSelfTest.java       |    8 +-
 .../apache/ignite/util/GridQueueSelfTest.java   |    9 +-
 .../apache/ignite/util/GridRandomSelfTest.java  |   13 +-
 .../ignite/util/GridSnapshotLockSelfTest.java   |   20 +-
 .../util/GridSpinReadWriteLockSelfTest.java     |   14 +-
 .../util/GridStringBuilderFactorySelfTest.java  |    9 +-
 .../util/GridTopologyHeapSizeSelfTest.java      |   30 +-
 .../apache/ignite/util/GridTransientTest.java   |   12 +-
 .../ignite/util/TestTcpCommunicationSpi.java    |   17 +-
 .../ignite/util/mbeans/GridMBeanSelfTest.java   |   17 +-
 modules/docker/download_ignite.sh               |    2 +-
 modules/extdata/p2p/pom.xml                     |    2 +-
 .../tests/p2p/CacheDeploymentTestKey.java       |    2 +-
 .../tests/p2p/CacheDeploymentTestTask1.java     |   23 +-
 .../tests/p2p/CacheDeploymentTestTask2.java     |   22 +-
 .../tests/p2p/CacheDeploymentTestTask3.java     |   23 +-
 .../tests/p2p/CacheDeploymentTestValue.java     |    4 +-
 .../tests/p2p/CacheDeploymentTestValue2.java    |    2 +-
 .../ignite/tests/p2p/ExcludeNodeFilter.java     |   11 +-
 .../p2p/GridCacheDeploymentTestValue3.java      |    7 +-
 .../tests/p2p/GridEventConsumeFilter.java       |    6 +-
 .../GridEventConsumeProjectionPredicate.java    |    6 +-
 .../tests/p2p/GridExternalAffinityFunction.java |   21 +-
 .../p2p/GridExternalAffinityKeyMapper.java      |    4 +-
 .../tests/p2p/GridP2PAwareTestUserResource.java |   11 +-
 .../p2p/GridP2PContinuousDeploymentTask1.java   |   20 +-
 .../p2p/GridP2PContinuousDeploymentTask2.java   |   16 +-
 .../p2p/GridP2PEventFilterExternalPath1.java    |   11 +-
 .../p2p/GridP2PEventFilterExternalPath2.java    |   11 +-
 .../tests/p2p/GridSingleSplitTestJobTarget.java |    2 +-
 .../tests/p2p/GridTestMessageListener.java      |   15 +-
 .../ignite/tests/p2p/GridTestMessageTopic.java  |    4 +-
 .../ignite/tests/p2p/JobStealingTask.java       |   27 +-
 .../tests/p2p/P2PTestTaskExternalPath1.java     |   31 +-
 .../tests/p2p/P2PTestTaskExternalPath2.java     |   31 +-
 .../ignite/tests/p2p/SingleSplitTestTask.java   |   14 +-
 .../ignite/tests/p2p/TestUserResource.java      |    4 +-
 .../CacheNoValueClassOnServerTestClient.java    |   17 +-
 .../apache/ignite/tests/p2p/cache/Person.java   |    4 +-
 .../p2p/startcache/CacheAllNodesFilter.java     |    6 +-
 .../CacheConfigurationP2PTestClient.java        |   20 +-
 .../tests/p2p/startcache/Organization1.java     |    9 +-
 .../tests/p2p/startcache/Organization2.java     |    9 +-
 .../extdata/uri/modules/uri-dependency/pom.xml  |    2 +-
 .../deployment/uri/tasks/GarHelloWorldBean.java |   10 +-
 modules/extdata/uri/pom.xml                     |    2 +-
 .../deployment/uri/tasks/GarHelloWorldTask.java |   24 +-
 .../GridUriDeploymentAbstractTestTask.java      |   10 +-
 .../uri/tasks/GridUriDeploymentDependency1.java |   10 +-
 .../uri/tasks/GridUriDeploymentDependency2.java |   10 +-
 .../tasks/GridUriDeploymentInnerTestTask.java   |   10 +-
 .../GridUriDeploymentInterfaceTestTask.java     |    4 +-
 .../GridUriDeploymentNonePublicTestTask.java    |   10 +-
 .../uri/tasks/GridUriDeploymentTestTask0.java   |   10 +-
 .../uri/tasks/GridUriDeploymentTestTask1.java   |   15 +-
 .../uri/tasks/GridUriDeploymentTestTask2.java   |   15 +-
 .../uri/tasks/GridUriDeploymentTestTask3.java   |   10 +-
 .../uri/tasks/GridUriDeploymentTestTask4.java   |   10 +-
 .../uri/tasks/GridUriDeploymentTestTask5.java   |   10 +-
 .../uri/tasks/GridUriDeploymentTestTask6.java   |   10 +-
 .../GridUriDeploymentTestWithNameTask0.java     |   11 +-
 .../GridUriDeploymentTestWithNameTask3.java     |   11 +-
 .../GridUriDeploymentTestWithNameTask4.java     |   11 +-
 .../GridUriDeploymentTestWithNameTask5.java     |   11 +-
 .../GridUriDeploymentTestWithNameTask6.java     |   11 +-
 .../GridUriDeploymentTestWithNameTask7.java     |   11 +-
 modules/gce/pom.xml                             |    3 +-
 .../gce/TcpDiscoveryGoogleStorageIpFinder.java  |   53 +-
 ...pDiscoveryGoogleStorageIpFinderSelfTest.java |   11 +-
 .../ignite/testsuites/IgniteGCETestSuite.java   |    6 +-
 modules/geospatial/pom.xml                      |    3 +-
 .../query/h2/opt/GridH2SpatialIndex.java        |   43 +-
 .../query/h2/GridH2IndexingGeoSelfTest.java     |   40 +-
 .../testsuites/GeoSpatialIndexingTestSuite.java |    6 +-
 modules/hadoop/pom.xml                          |    3 +-
 .../fs/IgniteHadoopFileSystemCounterWriter.java |   31 +-
 .../fs/IgniteHadoopIgfsSecondaryFileSystem.java |   65 +-
 .../hadoop/fs/v1/IgniteHadoopFileSystem.java    |   97 +-
 .../hadoop/fs/v2/IgniteHadoopFileSystem.java    |  102 +-
 .../IgniteHadoopClientProtocolProvider.java     |   40 +-
 .../mapreduce/IgniteHadoopMapReducePlanner.java |   51 +-
 .../processors/hadoop/HadoopClassLoader.java    |   47 +-
 .../processors/hadoop/HadoopComponent.java      |    5 +-
 .../processors/hadoop/HadoopContext.java        |   25 +-
 .../processors/hadoop/HadoopDefaultJobInfo.java |   20 +-
 .../internal/processors/hadoop/HadoopImpl.java  |   12 +-
 .../hadoop/HadoopMapReduceCounterGroup.java     |   14 +-
 .../hadoop/HadoopMapReduceCounters.java         |   30 +-
 .../processors/hadoop/HadoopProcessor.java      |   32 +-
 .../internal/processors/hadoop/HadoopSetup.java |   37 +-
 .../hadoop/HadoopTaskCancelledException.java    |    4 +-
 .../internal/processors/hadoop/HadoopUtils.java |   32 +-
 .../hadoop/SecondaryFileSystemProvider.java     |   33 +-
 .../hadoop/counter/HadoopCounterAdapter.java    |   12 +-
 .../hadoop/counter/HadoopCountersImpl.java      |   23 +-
 .../hadoop/counter/HadoopLongCounter.java       |    7 +-
 .../counter/HadoopPerformanceCounter.java       |   29 +-
 .../hadoop/fs/HadoopFileSystemCacheUtils.java   |   23 +-
 .../hadoop/fs/HadoopFileSystemsUtils.java       |    8 +-
 .../hadoop/fs/HadoopLazyConcurrentMap.java      |   18 +-
 .../hadoop/fs/HadoopLocalFileSystemV1.java      |    8 +-
 .../hadoop/fs/HadoopLocalFileSystemV2.java      |   18 +-
 .../processors/hadoop/fs/HadoopParameters.java  |    2 +-
 .../hadoop/fs/HadoopRawLocalFileSystem.java     |   30 +-
 .../processors/hadoop/igfs/HadoopIgfs.java      |   20 +-
 .../igfs/HadoopIgfsCommunicationException.java  |    4 +-
 .../hadoop/igfs/HadoopIgfsEndpoint.java         |   21 +-
 .../processors/hadoop/igfs/HadoopIgfsEx.java    |    9 +-
 .../hadoop/igfs/HadoopIgfsFuture.java           |    6 +-
 .../hadoop/igfs/HadoopIgfsInProc.java           |   38 +-
 .../hadoop/igfs/HadoopIgfsInputStream.java      |   23 +-
 .../processors/hadoop/igfs/HadoopIgfsIo.java    |   10 +-
 .../processors/hadoop/igfs/HadoopIgfsIpcIo.java |   53 +-
 .../hadoop/igfs/HadoopIgfsIpcIoListener.java    |    2 +-
 .../hadoop/igfs/HadoopIgfsJclLogger.java        |   13 +-
 .../hadoop/igfs/HadoopIgfsOutProc.java          |   61 +-
 .../hadoop/igfs/HadoopIgfsOutputStream.java     |   14 +-
 .../hadoop/igfs/HadoopIgfsProperties.java       |   13 +-
 .../hadoop/igfs/HadoopIgfsProxyInputStream.java |   12 +-
 .../igfs/HadoopIgfsProxyOutputStream.java       |   10 +-
 ...fsSecondaryFileSystemPositionedReadable.java |   13 +-
 .../hadoop/igfs/HadoopIgfsStreamDelegate.java   |    4 +-
 .../igfs/HadoopIgfsStreamEventListener.java     |    4 +-
 .../processors/hadoop/igfs/HadoopIgfsUtils.java |   21 +-
 .../hadoop/igfs/HadoopIgfsWrapper.java          |  143 +-
 .../hadoop/jobtracker/HadoopJobMetadata.java    |   30 +-
 .../hadoop/jobtracker/HadoopJobTracker.java     |  108 +-
 .../hadoop/message/HadoopMessage.java           |    4 +-
 .../planner/HadoopDefaultMapReducePlan.java     |   12 +-
 .../hadoop/proto/HadoopClientProtocol.java      |   59 +-
 .../proto/HadoopProtocolJobCountersTask.java    |   12 +-
 .../proto/HadoopProtocolJobStatusTask.java      |   19 +-
 .../hadoop/proto/HadoopProtocolKillJobTask.java |   12 +-
 .../proto/HadoopProtocolNextTaskIdTask.java     |    7 +-
 .../proto/HadoopProtocolSubmitJobTask.java      |   16 +-
 .../hadoop/proto/HadoopProtocolTaskAdapter.java |   31 +-
 .../proto/HadoopProtocolTaskArguments.java      |   13 +-
 .../hadoop/shuffle/HadoopShuffle.java           |   37 +-
 .../hadoop/shuffle/HadoopShuffleAck.java        |   15 +-
 .../hadoop/shuffle/HadoopShuffleJob.java        |   64 +-
 .../hadoop/shuffle/HadoopShuffleMessage.java    |   25 +-
 .../HadoopConcurrentHashMultimap.java           |   29 +-
 .../shuffle/collections/HadoopHashMultimap.java |   12 +-
 .../collections/HadoopHashMultimapBase.java     |   15 +-
 .../shuffle/collections/HadoopMultimap.java     |   13 +-
 .../shuffle/collections/HadoopMultimapBase.java |   33 +-
 .../shuffle/collections/HadoopSkipList.java     |   29 +-
 .../shuffle/streams/HadoopDataInStream.java     |   11 +-
 .../shuffle/streams/HadoopDataOutStream.java    |   13 +-
 .../shuffle/streams/HadoopOffheapBuffer.java    |    2 +-
 .../HadoopEmbeddedTaskExecutor.java             |   25 +-
 .../taskexecutor/HadoopExecutorService.java     |   27 +-
 .../hadoop/taskexecutor/HadoopRunnableTask.java |   36 +-
 .../taskexecutor/HadoopTaskExecutorAdapter.java |   14 +-
 .../hadoop/taskexecutor/HadoopTaskState.java    |    2 +-
 .../hadoop/taskexecutor/HadoopTaskStatus.java   |   14 +-
 .../external/HadoopExternalTaskExecutor.java    |   70 +-
 .../external/HadoopExternalTaskMetadata.java    |    9 +-
 .../external/HadoopJobInfoUpdateRequest.java    |   18 +-
 .../external/HadoopPrepareForJobRequest.java    |   18 +-
 .../external/HadoopProcessDescriptor.java       |    9 +-
 .../external/HadoopProcessStartedAck.java       |   11 +-
 .../external/HadoopTaskExecutionRequest.java    |   20 +-
 .../external/HadoopTaskFinishedMessage.java     |   16 +-
 .../child/HadoopChildProcessRunner.java         |   58 +-
 .../child/HadoopExternalProcessStarter.java     |   31 +-
 .../HadoopAbstractCommunicationClient.java      |    8 +-
 .../HadoopCommunicationClient.java              |    8 +-
 .../HadoopExternalCommunication.java            |   74 +-
 .../HadoopHandshakeTimeoutException.java        |    6 +-
 .../communication/HadoopIpcToNioAdapter.java    |   27 +-
 .../communication/HadoopMarshallerFilter.java   |   12 +-
 .../communication/HadoopMessageListener.java    |    6 +-
 .../HadoopTcpNioCommunicationClient.java        |   14 +-
 .../hadoop/v1/HadoopV1CleanupTask.java          |   16 +-
 .../processors/hadoop/v1/HadoopV1Counter.java   |   15 +-
 .../processors/hadoop/v1/HadoopV1MapTask.java   |   25 +-
 .../hadoop/v1/HadoopV1OutputCollector.java      |   21 +-
 .../hadoop/v1/HadoopV1Partitioner.java          |   10 +-
 .../hadoop/v1/HadoopV1ReduceTask.java           |   18 +-
 .../processors/hadoop/v1/HadoopV1Reporter.java  |   10 +-
 .../processors/hadoop/v1/HadoopV1SetupTask.java |   14 +-
 .../processors/hadoop/v1/HadoopV1Splitter.java  |   25 +-
 .../processors/hadoop/v1/HadoopV1Task.java      |   18 +-
 .../processors/hadoop/v2/HadoopDaemon.java      |    5 +-
 .../hadoop/v2/HadoopExternalSplit.java          |   10 +-
 .../hadoop/v2/HadoopNativeCodeLoader.java       |    6 +-
 .../hadoop/v2/HadoopSerializationWrapper.java   |   19 +-
 .../hadoop/v2/HadoopShutdownHookManager.java    |    8 +-
 .../hadoop/v2/HadoopSplitWrapper.java           |   11 +-
 .../hadoop/v2/HadoopV2CleanupTask.java          |   13 +-
 .../processors/hadoop/v2/HadoopV2Context.java   |   33 +-
 .../processors/hadoop/v2/HadoopV2Counter.java   |   11 +-
 .../processors/hadoop/v2/HadoopV2Job.java       |   85 +-
 .../hadoop/v2/HadoopV2JobResourceManager.java   |   39 +-
 .../processors/hadoop/v2/HadoopV2MapTask.java   |   19 +-
 .../hadoop/v2/HadoopV2Partitioner.java          |   10 +-
 .../hadoop/v2/HadoopV2ReduceTask.java           |   14 +-
 .../processors/hadoop/v2/HadoopV2SetupTask.java |   13 +-
 .../processors/hadoop/v2/HadoopV2Splitter.java  |   28 +-
 .../processors/hadoop/v2/HadoopV2Task.java      |   20 +-
 .../hadoop/v2/HadoopV2TaskContext.java          |   83 +-
 .../hadoop/v2/HadoopWritableSerialization.java  |   17 +-
 .../HadoopClientProtocolEmbeddedSelfTest.java   |    4 +-
 .../hadoop/HadoopClientProtocolSelfTest.java    |   59 +-
 .../ignite/igfs/Hadoop1DualAbstractTest.java    |   25 +-
 .../igfs/Hadoop1OverIgfsDualAsyncTest.java      |    2 +-
 .../igfs/Hadoop1OverIgfsDualSyncTest.java       |    2 +-
 ...oopFileSystemUniversalFileSystemAdapter.java |   19 +-
 .../HadoopIgfs20FileSystemAbstractSelfTest.java |   99 +-
 ...Igfs20FileSystemLoopbackPrimarySelfTest.java |    8 +-
 ...oopIgfs20FileSystemShmemPrimarySelfTest.java |    8 +-
 .../igfs/HadoopIgfsDualAbstractSelfTest.java    |   62 +-
 .../igfs/HadoopIgfsDualAsyncSelfTest.java       |    4 +-
 .../ignite/igfs/HadoopIgfsDualSyncSelfTest.java |    4 +-
 ...oopSecondaryFileSystemConfigurationTest.java |   70 +-
 .../apache/ignite/igfs/IgfsEventsTestSuite.java |   25 +-
 .../igfs/IgfsNearOnlyMultiNodeSelfTest.java     |   50 +-
 .../IgniteHadoopFileSystemAbstractSelfTest.java |  106 +-
 .../IgniteHadoopFileSystemClientSelfTest.java   |   57 +-
 ...IgniteHadoopFileSystemHandshakeSelfTest.java |   52 +-
 .../IgniteHadoopFileSystemIpcCacheSelfTest.java |   51 +-
 .../IgniteHadoopFileSystemLoggerSelfTest.java   |   42 +-
 ...niteHadoopFileSystemLoggerStateSelfTest.java |   46 +-
 ...adoopFileSystemLoopbackAbstractSelfTest.java |    6 +-
 ...SystemLoopbackEmbeddedDualAsyncSelfTest.java |    4 +-
 ...eSystemLoopbackEmbeddedDualSyncSelfTest.java |    4 +-
 ...leSystemLoopbackEmbeddedPrimarySelfTest.java |    4 +-
 ...SystemLoopbackEmbeddedSecondarySelfTest.java |    4 +-
 ...SystemLoopbackExternalDualAsyncSelfTest.java |    4 +-
 ...eSystemLoopbackExternalDualSyncSelfTest.java |    4 +-
 ...leSystemLoopbackExternalPrimarySelfTest.java |    4 +-
 ...SystemLoopbackExternalSecondarySelfTest.java |    4 +-
 ...teHadoopFileSystemSecondaryModeSelfTest.java |   47 +-
 ...teHadoopFileSystemShmemAbstractSelfTest.java |   19 +-
 ...ileSystemShmemEmbeddedDualAsyncSelfTest.java |    4 +-
 ...FileSystemShmemEmbeddedDualSyncSelfTest.java |    4 +-
 ...pFileSystemShmemEmbeddedPrimarySelfTest.java |    4 +-
 ...ileSystemShmemEmbeddedSecondarySelfTest.java |    4 +-
 ...ileSystemShmemExternalDualAsyncSelfTest.java |    4 +-
 ...FileSystemShmemExternalDualSyncSelfTest.java |    4 +-
 ...pFileSystemShmemExternalPrimarySelfTest.java |    4 +-
 ...ileSystemShmemExternalSecondarySelfTest.java |    4 +-
 .../hadoop/HadoopAbstractSelfTest.java          |   38 +-
 .../hadoop/HadoopAbstractWordCountTest.java     |   19 +-
 .../hadoop/HadoopClassLoaderTest.java           |    6 +-
 .../hadoop/HadoopCommandLineTest.java           |   48 +-
 .../HadoopDefaultMapReducePlannerSelfTest.java  |   68 +-
 .../hadoop/HadoopFileSystemsTest.java           |   21 +-
 .../processors/hadoop/HadoopGroupingTest.java   |   43 +-
 .../hadoop/HadoopJobTrackerSelfTest.java        |   44 +-
 .../hadoop/HadoopMapReduceEmbeddedSelfTest.java |   34 +-
 .../processors/hadoop/HadoopMapReduceTest.java  |   78 +-
 .../hadoop/HadoopPopularWordsTest.java          |   40 +-
 .../HadoopSerializationWrapperSelfTest.java     |   21 +-
 .../processors/hadoop/HadoopSharedMap.java      |    7 +-
 .../hadoop/HadoopSortingExternalTest.java       |    7 +-
 .../processors/hadoop/HadoopSortingTest.java    |   50 +-
 .../hadoop/HadoopSplitWrapperSelfTest.java      |   22 +-
 .../processors/hadoop/HadoopStartup.java        |    6 +-
 .../hadoop/HadoopTaskExecutionSelfTest.java     |   60 +-
 .../hadoop/HadoopTasksAllVersionsTest.java      |   22 +-
 .../processors/hadoop/HadoopTasksV1Test.java    |   15 +-
 .../processors/hadoop/HadoopTasksV2Test.java    |   26 +-
 .../hadoop/HadoopTestRoundRobinMrPlanner.java   |   19 +-
 .../hadoop/HadoopTestTaskContext.java           |   25 +-
 .../processors/hadoop/HadoopTestUtils.java      |   19 +-
 .../processors/hadoop/HadoopV2JobSelfTest.java  |   30 +-
 .../hadoop/HadoopValidationSelfTest.java        |    4 +-
 .../hadoop/examples/HadoopWordCount1.java       |   14 +-
 .../hadoop/examples/HadoopWordCount1Map.java    |   17 +-
 .../hadoop/examples/HadoopWordCount1Reduce.java |   16 +-
 .../hadoop/examples/HadoopWordCount2.java       |   18 +-
 .../hadoop/examples/HadoopWordCount2Mapper.java |   15 +-
 .../examples/HadoopWordCount2Reducer.java       |   13 +-
 .../collections/HadoopAbstractMapTest.java      |   29 +-
 .../HadoopConcurrentHashMultimapSelftest.java   |   41 +-
 .../collections/HadoopHashMapSelfTest.java      |   25 +-
 .../collections/HadoopSkipListSelfTest.java     |   46 +-
 .../streams/HadoopDataStreamSelfTest.java       |   11 +-
 .../taskexecutor/HadoopExecutorServiceTest.java |   15 +-
 .../HadoopExternalTaskExecutionSelfTest.java    |   46 +-
 .../HadoopExternalCommunicationSelfTest.java    |   30 +-
 .../testsuites/IgniteHadoopTestSuite.java       |   91 +-
 .../IgniteIgfsLinuxAndMacOSTestSuite.java       |   20 +-
 modules/hibernate/pom.xml                       |    3 +-
 .../HibernateAbstractRegionAccessStrategy.java  |    9 +-
 .../HibernateAccessStrategyAdapter.java         |   33 +-
 .../hibernate/HibernateCollectionRegion.java    |   14 +-
 .../cache/hibernate/HibernateEntityRegion.java  |   15 +-
 .../hibernate/HibernateGeneralDataRegion.java   |   15 +-
 .../hibernate/HibernateNaturalIdRegion.java     |   15 +-
 .../HibernateNonStrictAccessStrategy.java       |   23 +-
 .../hibernate/HibernateQueryResultsRegion.java  |   10 +-
 .../HibernateReadOnlyAccessStrategy.java        |   14 +-
 .../HibernateReadWriteAccessStrategy.java       |   26 +-
 .../ignite/cache/hibernate/HibernateRegion.java |   14 +-
 .../cache/hibernate/HibernateRegionFactory.java |   31 +-
 .../hibernate/HibernateTimestampsRegion.java    |    8 +-
 .../HibernateTransactionalAccessStrategy.java   |   14 +-
 .../HibernateTransactionalDataRegion.java       |   22 +-
 .../hibernate/CacheHibernateBlobStore.java      |   57 +-
 .../hibernate/CacheHibernateBlobStoreEntry.java |    7 +-
 .../CacheHibernateBlobStoreFactory.java         |   26 +-
 .../CacheHibernateStoreSessionListener.java     |   32 +-
 .../HibernateL2CacheConfigurationSelfTest.java  |   57 +-
 .../hibernate/HibernateL2CacheSelfTest.java     |   78 +-
 .../HibernateL2CacheTransactionalSelfTest.java  |   34 +-
 .../CacheHibernateBlobStoreNodeRestartTest.java |   12 +-
 .../CacheHibernateBlobStoreSelfTest.java        |   15 +-
 .../CacheHibernateStoreFactorySelfTest.java     |   42 +-
 ...heHibernateStoreSessionListenerSelfTest.java |   36 +-
 .../testsuites/IgniteHibernateTestSuite.java    |   13 +-
 modules/indexing/pom.xml                        |    3 +-
 .../query/h2/GridH2ResultSetIterator.java       |   16 +-
 .../processors/query/h2/IgniteH2Indexing.java   |  196 +-
 .../query/h2/opt/GridH2AbstractKeyValueRow.java |   61 +-
 .../processors/query/h2/opt/GridH2Cursor.java   |   12 +-
 .../query/h2/opt/GridH2IndexBase.java           |   66 +-
 .../query/h2/opt/GridH2KeyValueRowOffheap.java  |   19 +-
 .../query/h2/opt/GridH2KeyValueRowOnheap.java   |    8 +-
 .../processors/query/h2/opt/GridH2Row.java      |    6 +-
 .../query/h2/opt/GridH2RowDescriptor.java       |   16 +-
 .../processors/query/h2/opt/GridH2Table.java    |   58 +-
 .../query/h2/opt/GridH2TreeIndex.java           |   41 +-
 .../processors/query/h2/opt/GridH2Utils.java    |   10 +-
 .../query/h2/opt/GridH2ValueCacheObject.java    |   19 +-
 .../query/h2/opt/GridLuceneDirectory.java       |   21 +-
 .../processors/query/h2/opt/GridLuceneFile.java |   10 +-
 .../query/h2/opt/GridLuceneIndex.java           |   64 +-
 .../query/h2/opt/GridLuceneInputStream.java     |   13 +-
 .../query/h2/opt/GridLuceneLockFactory.java     |   10 +-
 .../query/h2/opt/GridLuceneOutputStream.java    |    9 +-
 .../query/h2/opt/GridSearchRowPointer.java      |    6 +-
 .../query/h2/sql/GridSqlAggregateFunction.java  |   12 +-
 .../processors/query/h2/sql/GridSqlAlias.java   |    7 +-
 .../processors/query/h2/sql/GridSqlColumn.java  |    4 +-
 .../processors/query/h2/sql/GridSqlConst.java   |    7 +-
 .../processors/query/h2/sql/GridSqlElement.java |    5 +-
 .../query/h2/sql/GridSqlFunction.java           |   22 +-
 .../query/h2/sql/GridSqlFunctionType.java       |    4 +-
 .../processors/query/h2/sql/GridSqlJoin.java    |   10 +-
 .../query/h2/sql/GridSqlOperation.java          |    6 +-
 .../query/h2/sql/GridSqlOperationType.java      |    5 +-
 .../query/h2/sql/GridSqlParameter.java          |    4 +-
 .../query/h2/sql/GridSqlPlaceholder.java        |    4 +-
 .../processors/query/h2/sql/GridSqlQuery.java   |    9 +-
 .../query/h2/sql/GridSqlQueryParser.java        |   82 +-
 .../query/h2/sql/GridSqlQuerySplitter.java      |   34 +-
 .../processors/query/h2/sql/GridSqlSelect.java  |    9 +-
 .../query/h2/sql/GridSqlSortColumn.java         |    2 +-
 .../query/h2/sql/GridSqlSubquery.java           |    4 +-
 .../processors/query/h2/sql/GridSqlTable.java   |    9 +-
 .../processors/query/h2/sql/GridSqlType.java    |    8 +-
 .../processors/query/h2/sql/GridSqlUnion.java   |    9 +-
 .../processors/query/h2/sql/GridSqlValue.java   |    2 +-
 .../query/h2/twostep/GridMapQueryExecutor.java  |   97 +-
 .../query/h2/twostep/GridMergeIndex.java        |   44 +-
 .../h2/twostep/GridMergeIndexUnsorted.java      |   25 +-
 .../query/h2/twostep/GridMergeTable.java        |   29 +-
 .../h2/twostep/GridReduceQueryExecutor.java     |  114 +-
 .../query/h2/twostep/GridResultPage.java        |   27 +-
 .../query/h2/twostep/GridThreadLocalTable.java  |   53 +-
 .../query/h2/twostep/msg/GridH2Array.java       |   26 +-
 .../query/h2/twostep/msg/GridH2Boolean.java     |   13 +-
 .../query/h2/twostep/msg/GridH2Byte.java        |   13 +-
 .../query/h2/twostep/msg/GridH2Bytes.java       |   13 +-
 .../query/h2/twostep/msg/GridH2CacheObject.java |   19 +-
 .../query/h2/twostep/msg/GridH2Date.java        |   13 +-
 .../query/h2/twostep/msg/GridH2Decimal.java     |   16 +-
 .../query/h2/twostep/msg/GridH2Double.java      |   13 +-
 .../query/h2/twostep/msg/GridH2Float.java       |   13 +-
 .../query/h2/twostep/msg/GridH2Geometry.java    |   15 +-
 .../query/h2/twostep/msg/GridH2Integer.java     |   13 +-
 .../query/h2/twostep/msg/GridH2JavaObject.java  |   13 +-
 .../query/h2/twostep/msg/GridH2Long.java        |   13 +-
 .../query/h2/twostep/msg/GridH2Null.java        |   13 +-
 .../query/h2/twostep/msg/GridH2Short.java       |   13 +-
 .../query/h2/twostep/msg/GridH2String.java      |   13 +-
 .../query/h2/twostep/msg/GridH2Time.java        |   13 +-
 .../query/h2/twostep/msg/GridH2Timestamp.java   |   13 +-
 .../query/h2/twostep/msg/GridH2Uuid.java        |   13 +-
 .../h2/twostep/msg/GridH2ValueMessage.java      |   15 +-
 .../twostep/msg/GridH2ValueMessageFactory.java  |   19 +-
 .../CacheAbstractQueryMetricsSelfTest.java      |  302 +-
 .../cache/CacheConfigurationP2PTest.java        |   36 +-
 .../cache/CacheConfigurationP2PTestServer.java  |   13 +-
 .../cache/CacheLocalQueryMetricsSelfTest.java   |    4 +-
 ...titionedQueryMetricsDistributedSelfTest.java |    4 +-
 ...chePartitionedQueryMetricsLocalSelfTest.java |    4 +-
 ...plicatedQueryMetricsDistributedSelfTest.java |    4 +-
 ...acheReplicatedQueryMetricsLocalSelfTest.java |    4 +-
 ...CacheScanPartitionQueryFallbackSelfTest.java |  361 +-
 .../cache/GridCacheCrossCacheQuerySelfTest.java |   45 +-
 .../cache/GridCacheOffHeapAndSwapSelfTest.java  |   68 +-
 .../cache/GridCacheOffHeapSelfTest.java         |   61 +-
 .../GridCacheOffheapIndexEntryEvictTest.java    |   42 +-
 .../cache/GridCacheOffheapIndexGetSelfTest.java |   46 +-
 .../GridCacheQueryIndexDisabledSelfTest.java    |   32 +-
 .../GridCacheQuerySerializationSelfTest.java    |   37 +-
 .../cache/GridCacheQuerySimpleBenchmark.java    |   46 +-
 .../cache/GridCacheQueryTestValue.java          |    8 +-
 ...idCacheReduceQueryMultithreadedSelfTest.java |   34 +-
 .../processors/cache/GridCacheSwapSelfTest.java |   63 +-
 .../cache/GridIndexingWithNoopSwapSelfTest.java |   38 +-
 .../IgniteCacheAbstractFieldsQuerySelfTest.java |   73 +-
 .../cache/IgniteCacheAbstractQuerySelfTest.java |  120 +-
 .../IgniteCacheCollocatedQuerySelfTest.java     |   40 +-
 ...acheConfigurationPrimitiveTypesSelfTest.java |   19 +-
 .../IgniteCacheFieldsQueryNoDataSelfTest.java   |   26 +-
 .../cache/IgniteCacheLargeResultSelfTest.java   |   29 +-
 .../cache/IgniteCacheOffheapEvictQueryTest.java |   50 +-
 ...CacheOffheapTieredMultithreadedSelfTest.java |   44 +-
 ...niteCacheP2pUnmarshallingQueryErrorTest.java |    9 +-
 ...hePartitionedQueryMultiThreadedSelfTest.java |   59 +-
 ...teCacheQueryEvictsMultiThreadedSelfTest.java |    2 +-
 .../cache/IgniteCacheQueryIndexSelfTest.java    |   26 +-
 .../cache/IgniteCacheQueryLoadSelfTest.java     |   52 +-
 ...QueryMultiThreadedOffHeapTieredSelfTest.java |    7 +-
 .../IgniteCacheQueryMultiThreadedSelfTest.java  |   73 +-
 ...QueryOffheapEvictsMultiThreadedSelfTest.java |    7 +-
 ...eCacheQueryOffheapMultiThreadedSelfTest.java |    2 +-
 ...gniteCacheSqlQueryMultiThreadedSelfTest.java |   47 +-
 ...ClientReconnectCacheQueriesFailoverTest.java |   37 +-
 .../cache/IgniteClientReconnectQueriesTest.java |   44 +-
 .../IgniteCacheAtomicFieldsQuerySelfTest.java   |   16 +-
 ...cheAtomicNearEnabledFieldsQuerySelfTest.java |    4 +-
 ...niteCacheAtomicNearEnabledQuerySelfTest.java |    8 +-
 .../near/IgniteCacheAtomicQuerySelfTest.java    |    8 +-
 ...lientQueryReplicatedNodeRestartSelfTest.java |   72 +-
 ...artitionedFieldsQueryP2PEnabledSelfTest.java |    4 +-
 ...niteCachePartitionedFieldsQuerySelfTest.java |   13 +-
 ...achePartitionedQueryP2PDisabledSelfTest.java |    4 +-
 .../IgniteCachePartitionedQuerySelfTest.java    |   26 +-
 .../IgniteCacheQueryNodeRestartSelfTest.java    |   57 +-
 .../IgniteCacheQueryNodeRestartSelfTest2.java   |   66 +-
 ...ReplicatedFieldsQueryP2PEnabledSelfTest.java |    4 +-
 ...gniteCacheReplicatedFieldsQuerySelfTest.java |   25 +-
 ...CacheReplicatedQueryP2PDisabledSelfTest.java |    4 +-
 .../IgniteCacheReplicatedQuerySelfTest.java     |  123 +-
 .../IgniteCacheLocalAtomicQuerySelfTest.java    |    6 +-
 .../IgniteCacheLocalFieldsQuerySelfTest.java    |    8 +-
 .../local/IgniteCacheLocalQuerySelfTest.java    |   22 +-
 ...dCacheAbstractReduceFieldsQuerySelfTest.java |   54 +-
 ...ridCacheReduceFieldsQueryAtomicSelfTest.java |    8 +-
 ...GridCacheReduceFieldsQueryLocalSelfTest.java |    6 +-
 ...cheReduceFieldsQueryPartitionedSelfTest.java |   15 +-
 ...acheReduceFieldsQueryReplicatedSelfTest.java |    6 +-
 .../cache/ttl/CacheTtlAbstractSelfTest.java     |   62 +-
 .../CacheTtlOffheapAtomicAbstractSelfTest.java  |    4 +-
 .../ttl/CacheTtlOffheapAtomicLocalSelfTest.java |    4 +-
 ...acheTtlOffheapAtomicPartitionedSelfTest.java |    4 +-
 ...TtlOffheapTransactionalAbstractSelfTest.java |    4 +-
 ...cheTtlOffheapTransactionalLocalSelfTest.java |    4 +-
 .../CacheTtlOnheapAtomicAbstractSelfTest.java   |    4 +-
 .../ttl/CacheTtlOnheapAtomicLocalSelfTest.java  |    4 +-
 ...eTtlOnheapTransactionalAbstractSelfTest.java |    4 +-
 ...acheTtlOnheapTransactionalLocalSelfTest.java |    4 +-
 ...lOnheapTransactionalPartitionedSelfTest.java |    4 +-
 .../query/IgniteSqlSplitterSelfTest.java        |  248 ++
 .../query/h2/GridH2IndexRebuildTest.java        |   33 +-
 .../query/h2/GridH2IndexingInMemSelfTest.java   |    2 +-
 .../query/h2/GridH2IndexingOffheapSelfTest.java |    2 +-
 .../h2/GridIndexingSpiAbstractSelfTest.java     |   60 +-
 .../query/h2/opt/GridH2TableSelfTest.java       |   42 +-
 .../h2/sql/AbstractH2CompareQueryTest.java      |   46 +-
 .../query/h2/sql/BaseH2CompareQueryTest.java    |   39 +-
 .../query/h2/sql/GridQueryParsingTest.java      |   52 +-
 .../query/h2/sql/H2CompareBigQueryTest.java     |   26 +-
 .../loadtests/h2indexing/GridTestEntity.java    |    7 +-
 .../loadtests/h2indexing/GridTreeBenchmark.java |   16 +-
 .../tcp/GridOrderedMessageCancelSelfTest.java   |   56 +-
 .../IgniteCacheQuerySelfTestSuite.java          |   82 +-
 .../IgniteCacheWithIndexingTestSuite.java       |   23 +-
 .../IgniteH2IndexingSpiTestSuite.java           |   10 +-
 modules/jcl/pom.xml                             |    3 +-
 .../org/apache/ignite/logger/jcl/JclLogger.java |    9 +-
 .../apache/ignite/logger/jcl/JclLoggerTest.java |   10 +-
 .../ignite/testsuites/IgniteJclTestSuite.java   |    6 +-
 modules/jms11/pom.xml                           |   98 +
 .../apache/ignite/stream/jms11/JmsStreamer.java |  539 +++
 .../ignite/stream/jms11/MessageTransformer.java |   42 +
 .../stream/jms11/IgniteJmsStreamerTest.java     |  565 +++
 .../jms11/IgniteJmsStreamerTestSuite.java       |   34 +
 .../ignite/stream/jms11/TestTransformers.java   |  128 +
 modules/jta/pom.xml                             |    3 +-
 .../apache/ignite/cache/jta/CacheTmLookup.java  |   11 +-
 .../cache/jta/jndi/CacheJndiTmLookup.java       |   16 +-
 .../jta/reflect/CacheReflectionTmLookup.java    |   13 +-
 .../processors/cache/jta/CacheJtaManager.java   |   23 +-
 .../cache/jta/GridCacheXAResource.java          |   28 +-
 ...CacheJtaConfigurationValidationSelfTest.java |   22 +-
 .../processors/cache/GridCacheJtaSelfTest.java  |   31 +-
 .../cache/GridCacheReplicatedJtaSelfTest.java   |    6 +-
 .../GridTmLookupLifecycleAwareSelfTest.java     |   31 +-
 .../ignite/testsuites/IgniteJtaTestSuite.java   |    9 +-
 modules/kafka/pom.xml                           |    3 +-
 .../ignite/stream/kafka/KafkaStreamer.java      |   27 +-
 .../kafka/IgniteKafkaStreamerSelfTestSuite.java |    4 +-
 .../stream/kafka/KafkaEmbeddedBroker.java       |   43 +-
 .../kafka/KafkaIgniteStreamerSelfTest.java      |   42 +-
 .../ignite/stream/kafka/SimplePartitioner.java  |    6 +-
 modules/log4j/pom.xml                           |    3 +-
 .../log4j/Log4JDailyRollingFileAppender.java    |   12 +-
 .../Log4JExternallyRolledFileAppender.java      |   11 +-
 .../ignite/logger/log4j/Log4JFileAppender.java  |   12 +-
 .../apache/ignite/logger/log4j/Log4JLogger.java |   53 +-
 .../ignite/logger/log4j/Log4jFileAware.java     |    4 +-
 .../logger/log4j/Log4jNodeIdFilePath.java       |   15 +-
 .../logger/log4j/Log4jRollingFileAppender.java  |   12 +-
 .../log4j/GridLog4jCorrectFileNameTest.java     |   24 +-
 .../logger/log4j/GridLog4jInitializedTest.java  |   10 +-
 .../logger/log4j/GridLog4jLoggingFileTest.java  |   13 +-
 .../logger/log4j/GridLog4jLoggingPathTest.java  |    8 +-
 .../logger/log4j/GridLog4jLoggingUrlTest.java   |   13 +-
 .../log4j/GridLog4jNotInitializedTest.java      |    8 +-
 .../ignite/testsuites/IgniteLog4jTestSuite.java |    8 +-
 modules/log4j2/pom.xml                          |    3 +-
 .../ignite/logger/log4j2/Log4J2Logger.java      |   72 +-
 .../ignite/logger/log4j2/package-info.java      |    2 +-
 .../log4j2/GridLog4j2CorrectFileNameTest.java   |    3 +-
 .../log4j2/GridLog4j2InitializedTest.java       |    9 +-
 .../log4j2/GridLog4j2LoggingFileTest.java       |    5 +-
 .../logger/log4j2/Log4j2LoggerSelfTest.java     |   27 +-
 .../log4j2/Log4j2LoggerVerboseModeSelfTest.java |   22 +-
 .../ignite/logger/log4j2/package-info.java      |    2 +-
 .../testsuites/IgniteLog4j2TestSuite.java       |    6 +-
 modules/mesos/pom.xml                           |    3 +-
 .../apache/ignite/mesos/ClusterProperties.java  |   19 +-
 .../apache/ignite/mesos/IgniteFramework.java    |   19 +-
 .../apache/ignite/mesos/IgniteScheduler.java    |   21 +-
 .../org/apache/ignite/mesos/IgniteTask.java     |    2 +-
 .../ignite/mesos/resource/IgniteProvider.java   |   16 +-
 .../ignite/mesos/resource/JettyServer.java      |    8 +-
 .../ignite/mesos/resource/ResourceHandler.java  |   22 +-
 .../ignite/mesos/resource/ResourceProvider.java |   16 +-
 .../org/apache/ignite/IgniteMesosTestSuite.java |    6 +-
 .../ignite/mesos/IgniteSchedulerSelfTest.java   |   16 +-
 modules/platform/licenses/apache-2.0.txt        |  202 ++
 modules/platform/pom.xml                        |   80 +
 modules/platform/src/main/cpp/README.txt        |  103 +
 .../platform/src/main/cpp/common/Makefile.am    |   45 +
 .../platform/src/main/cpp/common/configure.ac   |   62 +
 .../src/main/cpp/common/ignite-common.pc.in     |    9 +
 .../src/main/cpp/common/include/Makefile.am     |   22 +
 .../common/include/ignite/common/concurrent.h   |  210 ++
 .../cpp/common/include/ignite/common/exports.h  |  145 +
 .../cpp/common/include/ignite/common/java.h     |  652 ++++
 .../cpp/common/os/linux/include/Makefile.am     |   21 +
 .../os/linux/include/ignite/common/common.h     |   81 +
 .../linux/include/ignite/common/concurrent_os.h |  394 +++
 .../src/main/cpp/common/os/linux/src/common.cpp |   59 +
 .../cpp/common/os/linux/src/concurrent_os.cpp   |  175 +
 .../os/win/include/ignite/common/common.h       |   56 +
 .../win/include/ignite/common/concurrent_os.h   |  406 +++
 .../src/main/cpp/common/os/win/src/common.cpp   |   65 +
 .../cpp/common/os/win/src/concurrent_os.cpp     |  151 +
 .../src/main/cpp/common/project/README.TXT      |    1 +
 .../src/main/cpp/common/project/vs/README.TXT   |    1 +
 .../main/cpp/common/project/vs/common.vcxproj   |  202 ++
 .../common/project/vs/common.vcxproj.filters    |   54 +
 .../src/main/cpp/common/project/vs/module.def   |   99 +
 .../src/main/cpp/common/project/vs/targetver.h  |   25 +
 .../src/main/cpp/common/src/concurrent.cpp      |   94 +
 .../src/main/cpp/common/src/exports.cpp         |  413 +++
 .../platform/src/main/cpp/common/src/java.cpp   | 2205 ++++++++++++
 .../platform/src/main/cpp/core-test/Makefile.am |   49 +
 .../main/cpp/core-test/config/cache-query.xml   |   91 +
 .../main/cpp/core-test/config/cache-test.xml    |  129 +
 .../src/main/cpp/core-test/configure.ac         |   62 +
 .../src/main/cpp/core-test/include/Makefile.am  |   22 +
 .../include/ignite/portable_test_defs.h         |  320 ++
 .../include/ignite/portable_test_utils.h        |  516 +++
 .../cpp/core-test/include/teamcity_messages.h   |   55 +
 .../src/main/cpp/core-test/project/README.TXT   |    1 +
 .../main/cpp/core-test/project/vs/README.TXT    |    1 +
 .../cpp/core-test/project/vs/core-test.vcxproj  |  174 +
 .../project/vs/core-test.vcxproj.filters        |   68 +
 .../main/cpp/core-test/src/cache_query_test.cpp |  656 ++++
 .../src/main/cpp/core-test/src/cache_test.cpp   |  486 +++
 .../main/cpp/core-test/src/concurrent_test.cpp  |  186 +
 .../cpp/core-test/src/handle_registry_test.cpp  |  176 +
 .../main/cpp/core-test/src/ignition_test.cpp    |  102 +
 .../src/portable_reader_writer_raw_test.cpp     | 1532 +++++++++
 .../src/portable_reader_writer_test.cpp         | 1951 +++++++++++
 .../cpp/core-test/src/portable_session_test.cpp |  257 ++
 .../cpp/core-test/src/portable_test_defs.cpp    |   65 +
 .../main/cpp/core-test/src/teamcity_boost.cpp   |  159 +
 .../cpp/core-test/src/teamcity_messages.cpp     |  150 +
 modules/platform/src/main/cpp/core/Makefile.am  |   66 +
 modules/platform/src/main/cpp/core/configure.ac |   62 +
 modules/platform/src/main/cpp/core/ignite.pc.in |    9 +
 .../src/main/cpp/core/include/Makefile.am       |   61 +
 .../main/cpp/core/include/ignite/cache/cache.h  | 1153 +++++++
 .../cpp/core/include/ignite/cache/cache_entry.h |  118 +
 .../core/include/ignite/cache/cache_peek_mode.h |   71 +
 .../cpp/core/include/ignite/cache/query/query.h |   27 +
 .../include/ignite/cache/query/query_argument.h |  125 +
 .../include/ignite/cache/query/query_cursor.h   |  191 +
 .../include/ignite/cache/query/query_scan.h     |  151 +
 .../core/include/ignite/cache/query/query_sql.h |  253 ++
 .../include/ignite/cache/query/query_text.h     |  159 +
 .../src/main/cpp/core/include/ignite/guid.h     |  112 +
 .../src/main/cpp/core/include/ignite/ignite.h   |  154 +
 .../core/include/ignite/ignite_configuration.h  |   92 +
 .../main/cpp/core/include/ignite/ignite_error.h |  260 ++
 .../src/main/cpp/core/include/ignite/ignition.h |  195 ++
 .../core/include/ignite/impl/cache/cache_impl.h |  418 +++
 .../ignite/impl/cache/query/query_impl.h        |  115 +
 .../core/include/ignite/impl/handle_registry.h  |  202 ++
 .../include/ignite/impl/ignite_environment.h    |  130 +
 .../cpp/core/include/ignite/impl/ignite_impl.h  |  146 +
 .../core/include/ignite/impl/interop/interop.h  |   25 +
 .../ignite/impl/interop/interop_input_stream.h  |  234 ++
 .../ignite/impl/interop/interop_memory.h        |  280 ++
 .../ignite/impl/interop/interop_output_stream.h |  234 ++
 .../cpp/core/include/ignite/impl/operations.h   |  452 +++
 .../ignite/impl/portable/portable_common.h      |  146 +
 .../ignite/impl/portable/portable_id_resolver.h |  106 +
 .../impl/portable/portable_metadata_handler.h   |  102 +
 .../impl/portable/portable_metadata_manager.h   |  120 +
 .../impl/portable/portable_metadata_snapshot.h  |  122 +
 .../impl/portable/portable_metadata_updater.h   |   53 +
 .../portable/portable_metadata_updater_impl.h   |   65 +
 .../ignite/impl/portable/portable_reader_impl.h | 1130 ++++++
 .../ignite/impl/portable/portable_utils.h       |  344 ++
 .../ignite/impl/portable/portable_writer_impl.h |  859 +++++
 .../cpp/core/include/ignite/portable/portable.h |   29 +
 .../include/ignite/portable/portable_consts.h   |  106 +
 .../ignite/portable/portable_containers.h       |  525 +++
 .../ignite/portable/portable_raw_reader.h       |  324 ++
 .../ignite/portable/portable_raw_writer.h       |  300 ++
 .../include/ignite/portable/portable_reader.h   |  355 ++
 .../include/ignite/portable/portable_type.h     |  293 ++
 .../include/ignite/portable/portable_writer.h   |  335 ++
 .../main/cpp/core/os/linux/include/Makefile.am  |   20 +
 .../core/os/linux/include/ignite/impl/utils.h   |  155 +
 .../main/cpp/core/os/linux/src/impl/utils.cpp   |  439 +++
 .../cpp/core/os/win/include/ignite/impl/utils.h |  155 +
 .../src/main/cpp/core/os/win/src/impl/utils.cpp |  453 +++
 .../src/main/cpp/core/project/README.TXT        |    1 +
 .../src/main/cpp/core/project/vs/README.TXT     |    1 +
 .../src/main/cpp/core/project/vs/core.vcxproj   |  272 ++
 .../cpp/core/project/vs/core.vcxproj.filters    |  246 ++
 modules/platform/src/main/cpp/core/src/guid.cpp |   65 +
 .../platform/src/main/cpp/core/src/ignite.cpp   |   43 +
 .../src/main/cpp/core/src/ignite_error.cpp      |  222 ++
 .../platform/src/main/cpp/core/src/ignition.cpp |  468 +++
 .../main/cpp/core/src/impl/cache/cache_impl.cpp |  388 +++
 .../core/src/impl/cache/query/query_impl.cpp    |  193 ++
 .../main/cpp/core/src/impl/handle_registry.cpp  |  234 ++
 .../cpp/core/src/impl/ignite_environment.cpp    |  167 +
 .../src/main/cpp/core/src/impl/ignite_impl.cpp  |   42 +
 .../src/impl/interop/interop_input_stream.cpp   |  215 ++
 .../core/src/impl/interop/interop_memory.cpp    |  182 +
 .../src/impl/interop/interop_output_stream.cpp  |  215 ++
 .../impl/portable/portable_metadata_handler.cpp |   78 +
 .../impl/portable/portable_metadata_manager.cpp |  201 ++
 .../portable/portable_metadata_snapshot.cpp     |   70 +
 .../impl/portable/portable_metadata_updater.cpp |   32 +
 .../portable/portable_metadata_updater_impl.cpp |   94 +
 .../src/impl/portable/portable_reader_impl.cpp  |  683 ++++
 .../core/src/impl/portable/portable_utils.cpp   |  214 ++
 .../src/impl/portable/portable_writer_impl.cpp  |  600 ++++
 .../core/src/portable/portable_containers.cpp   |   76 +
 .../core/src/portable/portable_raw_reader.cpp   |  135 +
 .../core/src/portable/portable_raw_writer.cpp   |  147 +
 .../cpp/core/src/portable/portable_reader.cpp   |  142 +
 .../cpp/core/src/portable/portable_type.cpp     |   51 +
 .../cpp/core/src/portable/portable_writer.cpp   |  154 +
 .../platform/src/main/cpp/project/vs/ignite.sln |   48 +
 .../Apache.Ignite.Core.csproj                   |  376 ++
 .../Cache/CacheAtomicUpdateTimeoutException.cs  |   67 +
 .../Cache/CacheEntryProcessorException.cs       |   79 +
 .../Apache.Ignite.Core/Cache/CacheException.cs  |   68 +
 .../Cache/CachePartialUpdateException.cs        |  119 +
 .../Apache.Ignite.Core/Cache/CachePeekMode.cs   |   68 +
 .../Cache/Event/CacheEntryEventType.cs          |   41 +
 .../Cache/Event/ICacheEntryEvent.cs             |   40 +
 .../Cache/Event/ICacheEntryEventFilter.cs       |   31 +
 .../Cache/Event/ICacheEntryEventListener.cs     |   33 +
 .../Cache/Expiry/ExpiryPolicy.cs                |   89 +
 .../Cache/Expiry/IExpiryPolicy.cs               |   59 +
 .../dotnet/Apache.Ignite.Core/Cache/ICache.cs   |  542 +++
 .../Apache.Ignite.Core/Cache/ICacheAffinity.cs  |  161 +
 .../Apache.Ignite.Core/Cache/ICacheEntry.cs     |   37 +
 .../Cache/ICacheEntryFilter.cs                  |   34 +
 .../Cache/ICacheEntryProcessor.cs               |   45 +
 .../Cache/ICacheEntryProcessorResult.cs         |   40 +
 .../Apache.Ignite.Core/Cache/ICacheLock.cs      |   58 +
 .../Apache.Ignite.Core/Cache/ICacheMetrics.cs   |  486 +++
 .../Cache/IMutableCacheEntry.cs                 |   47 +
 .../Cache/Query/Continuous/ContinuousQuery.cs   |  170 +
 .../Query/Continuous/IContinuousQueryHandle.cs  |   51 +
 .../Cache/Query/IQueryCursor.cs                 |   40 +
 .../Apache.Ignite.Core/Cache/Query/QueryBase.cs |   82 +
 .../Apache.Ignite.Core/Cache/Query/ScanQuery.cs |   77 +
 .../Cache/Query/SqlFieldsQuery.cs               |   81 +
 .../Apache.Ignite.Core/Cache/Query/SqlQuery.cs  |  119 +
 .../Apache.Ignite.Core/Cache/Query/TextQuery.cs |  104 +
 .../Store/CacheParallelLoadStoreAdapter.cs      |  205 ++
 .../Cache/Store/CacheStoreAdapter.cs            |  146 +
 .../Cache/Store/CacheStoreException.cs          |   66 +
 .../Cache/Store/ICacheStore.cs                  |  184 +
 .../Cache/Store/ICacheStoreSession.cs           |   42 +
 .../Cluster/ClusterGroupEmptyException.cs       |   70 +
 .../Cluster/ClusterTopologyException.cs         |   69 +
 .../Apache.Ignite.Core/Cluster/ICluster.cs      |   80 +
 .../Apache.Ignite.Core/Cluster/IClusterGroup.cs |  229 ++
 .../Cluster/IClusterMetrics.cs                  |  515 +++
 .../Apache.Ignite.Core/Cluster/IClusterNode.cs  |  138 +
 .../Cluster/IClusterNodeFilter.cs               |   32 +
 .../Common/AsyncSupportedAttribute.cs           |   33 +
 .../Apache.Ignite.Core/Common/IAsyncSupport.cs  |   52 +
 .../dotnet/Apache.Ignite.Core/Common/IFuture.cs |  115 +
 .../Common/IgniteException.cs                   |   66 +
 .../Apache.Ignite.Core/Common/IgniteGuid.cs     |  138 +
 .../ComputeExecutionRejectedException.cs        |   69 +
 .../Compute/ComputeJobAdapter.cs                |  122 +
 .../Compute/ComputeJobFailoverException.cs      |   72 +
 .../Compute/ComputeJobResultPolicy.cs           |   45 +
 .../Compute/ComputeTaskAdapter.cs               |   93 +
 .../Compute/ComputeTaskCancelledException.cs    |   69 +
 .../ComputeTaskNoResultCacheAttribute.cs        |   35 +
 .../Compute/ComputeTaskSplitAdapter.cs          |   95 +
 .../Compute/ComputeTaskTimeoutException.cs      |   67 +
 .../Compute/ComputeUserUndeclaredException.cs   |   70 +
 .../Apache.Ignite.Core/Compute/ICompute.cs      |  274 ++
 .../Apache.Ignite.Core/Compute/IComputeFunc.cs  |   55 +
 .../Apache.Ignite.Core/Compute/IComputeJob.cs   |   58 +
 .../Compute/IComputeJobResult.cs                |   73 +
 .../Compute/IComputeReducer.cs                  |   39 +
 .../Apache.Ignite.Core/Compute/IComputeTask.cs  |  132 +
 .../Datastream/IDataStreamer.cs                 |  206 ++
 .../Datastream/IStreamReceiver.cs               |   38 +
 .../Datastream/StreamTransformer.cs             |   73 +
 .../Datastream/StreamVisitor.cs                 |   55 +
 .../Apache.Ignite.Core/Events/CacheEvent.cs     |  176 +
 .../Events/CacheQueryExecutedEvent.cs           |   97 +
 .../Events/CacheQueryReadEvent.cs               |  134 +
 .../Events/CacheRebalancingEvent.cs             |   98 +
 .../Events/CheckpointEvent.cs                   |   50 +
 .../Apache.Ignite.Core/Events/DiscoveryEvent.cs |   80 +
 .../Apache.Ignite.Core/Events/EventBase.cs      |  160 +
 .../Apache.Ignite.Core/Events/EventReader.cs    |   72 +
 .../Apache.Ignite.Core/Events/EventType.cs      |  514 +++
 .../dotnet/Apache.Ignite.Core/Events/IEvent.cs  |   74 +
 .../Apache.Ignite.Core/Events/IEventFilter.cs   |   36 +
 .../dotnet/Apache.Ignite.Core/Events/IEvents.cs |  182 +
 .../Apache.Ignite.Core/Events/JobEvent.cs       |  100 +
 .../Apache.Ignite.Core/Events/SwapSpaceEvent.cs |   50 +
 .../Apache.Ignite.Core/Events/TaskEvent.cs      |   91 +
 .../main/dotnet/Apache.Ignite.Core/IIgnite.cs   |  168 +
 .../Apache.Ignite.Core/IgniteConfiguration.cs   |  140 +
 .../main/dotnet/Apache.Ignite.Core/Ignition.cs  |  657 ++++
 .../Impl/Cache/CacheAffinityImpl.cs             |  275 ++
 .../Apache.Ignite.Core/Impl/Cache/CacheEntry.cs |  126 +
 .../Impl/Cache/CacheEntryFilterHolder.cs        |  147 +
 .../Impl/Cache/CacheEntryProcessorHolder.cs     |  145 +
 .../Impl/Cache/CacheEntryProcessorResult.cs     |   65 +
 .../Cache/CacheEntryProcessorResultHolder.cs    |  127 +
 .../Impl/Cache/CacheEnumerable.cs               |   82 +
 .../Impl/Cache/CacheEnumerator.cs               |  117 +
 .../Impl/Cache/CacheEnumeratorProxy.cs          |  156 +
 .../Apache.Ignite.Core/Impl/Cache/CacheImpl.cs  |  940 +++++
 .../Apache.Ignite.Core/Impl/Cache/CacheLock.cs  |  171 +
 .../Impl/Cache/CacheMetricsImpl.cs              |  248 ++
 .../Apache.Ignite.Core/Impl/Cache/CacheOp.cs    |   63 +
 .../Impl/Cache/CacheProxyImpl.cs                |  499 +++
 .../Impl/Cache/Event/CacheEntryCreateEvent.cs   |   74 +
 .../Impl/Cache/Event/CacheEntryRemoveEvent.cs   |   74 +
 .../Impl/Cache/Event/CacheEntryUpdateEvent.cs   |   79 +
 .../Impl/Cache/MutableCacheEntry.cs             |  163 +
 .../Impl/Cache/Query/AbstractQueryCursor.cs     |  264 ++
 .../Query/Continuous/ContinuousQueryFilter.cs   |  125 +
 .../Continuous/ContinuousQueryFilterHolder.cs   |  118 +
 .../Continuous/ContinuousQueryHandleImpl.cs     |  216 ++
 .../Query/Continuous/ContinuousQueryUtils.cs    |  115 +
 .../Impl/Cache/Query/FieldsQueryCursor.cs       |   54 +
 .../Impl/Cache/Query/QueryCursor.cs             |   50 +
 .../Impl/Cache/Store/CacheStore.cs              |  263 ++
 .../Impl/Cache/Store/CacheStoreSession.cs       |   53 +
 .../Impl/Cache/Store/CacheStoreSessionProxy.cs  |   63 +
 .../Impl/Cluster/ClusterGroupImpl.cs            |  577 ++++
 .../Impl/Cluster/ClusterMetricsImpl.cs          |  292 ++
 .../Impl/Cluster/ClusterNodeImpl.cs             |  221 ++
 .../Impl/Cluster/IClusterGroupEx.cs             |   35 +
 .../Impl/Collections/CollectionExtensions.cs    |   45 +
 .../Impl/Collections/MultiValueDictionary.cs    |  143 +
 .../Impl/Collections/ReadOnlyCollection.cs      |  102 +
 .../Impl/Collections/ReadOnlyDictionary.cs      |  149 +
 .../Impl/Common/AsyncResult.cs                  |   71 +
 .../Impl/Common/CompletedAsyncResult.cs         |   70 +
 .../Common/CopyOnWriteConcurrentDictionary.cs   |   70 +
 .../Impl/Common/DelegateConverter.cs            |  253 ++
 .../Impl/Common/DelegateTypeDescriptor.cs       |  314 ++
 .../Apache.Ignite.Core/Impl/Common/Future.cs    |  286 ++
 .../Impl/Common/FutureConverter.cs              |   62 +
 .../Impl/Common/FutureType.cs                   |   52 +
 .../Impl/Common/IFutureConverter.cs             |   34 +
 .../Impl/Common/IFutureInternal.cs              |   46 +
 .../Impl/Common/IgniteArgumentCheck.cs          |   76 +
 .../Impl/Common/LoadedAssembliesResolver.cs     |   96 +
 .../Impl/Common/PortableResultWrapper.cs        |   68 +
 .../Impl/Common/TypeCaster.cs                   |   72 +
 .../Closure/ComputeAbstractClosureTask.cs       |  101 +
 .../Impl/Compute/Closure/ComputeActionJob.cs    |   83 +
 .../Impl/Compute/Closure/ComputeFuncJob.cs      |   89 +
 .../Compute/Closure/ComputeMultiClosureTask.cs  |   56 +
 .../Impl/Compute/Closure/ComputeOutFuncJob.cs   |   76 +
 .../Closure/ComputeReducingClosureTask.cs       |   61 +
 .../Compute/Closure/ComputeSingleClosureTask.cs |   48 +
 .../Compute/Closure/IComputeResourceInjector.cs |   31 +
 .../Apache.Ignite.Core/Impl/Compute/Compute.cs  |  213 ++
 .../Impl/Compute/ComputeAsync.cs                |  261 ++
 .../Impl/Compute/ComputeFunc.cs                 |  119 +
 .../Impl/Compute/ComputeImpl.cs                 |  645 ++++
 .../Impl/Compute/ComputeJob.cs                  |  163 +
 .../Impl/Compute/ComputeJobHolder.cs            |  246 ++
 .../Compute/ComputeJobResultGenericWrapper.cs   |   70 +
 .../Impl/Compute/ComputeJobResultImpl.cs        |   96 +
 .../Impl/Compute/ComputeOutFunc.cs              |  123 +
 .../Impl/Compute/ComputeTaskHolder.cs           |  484 +++
 .../Impl/Datastream/DataStreamerBatch.cs        |  269 ++
 .../Impl/Datastream/DataStreamerEntry.cs        |   64 +
 .../Impl/Datastream/DataStreamerImpl.cs         |  832 +++++
 .../Impl/Datastream/DataStreamerRemoveEntry.cs  |   48 +
 .../Impl/Datastream/StreamReceiverHolder.cs     |  144 +
 .../Apache.Ignite.Core/Impl/Events/Events.cs    |  498 +++
 .../Impl/Events/EventsAsync.cs                  |  158 +
 .../Impl/Events/RemoteListenEventFilter.cs      |   85 +
 .../Apache.Ignite.Core/Impl/ExceptionUtils.cs   |  204 ++
 .../Apache.Ignite.Core/Impl/Handle/Handle.cs    |   69 +
 .../Impl/Handle/HandleRegistry.cs               |  340 ++
 .../Apache.Ignite.Core/Impl/Handle/IHandle.cs   |   35 +
 .../Apache.Ignite.Core/Impl/IInteropCallback.cs |   34 +
 .../dotnet/Apache.Ignite.Core/Impl/Ignite.cs    |  549 +++
 .../Impl/IgniteConfigurationEx.cs               |   57 +
 .../Apache.Ignite.Core/Impl/IgniteManager.cs    |  492 +++
 .../Apache.Ignite.Core/Impl/IgniteProxy.cs      |  351 ++
 .../Apache.Ignite.Core/Impl/IgniteUtils.cs      |  438 +++
 .../Impl/Interop/InteropDotNetConfiguration.cs  |   62 +
 .../InteropDotNetPortableConfiguration.cs       |  127 +
 .../InteropDotNetPortableTypeConfiguration.cs   |  151 +
 .../Impl/InteropExceptionHolder.cs              |   85 +
 .../Impl/LifecycleBeanHolder.cs                 |   66 +
 .../Impl/Memory/IPlatformMemory.cs              |   65 +
 .../Impl/Memory/InteropExternalMemory.cs        |   46 +
 .../Impl/Memory/InteropMemoryUtils.cs           |   38 +
 .../Memory/PlatformBigEndianMemoryStream.cs     |  483 +++
 .../Impl/Memory/PlatformMemory.cs               |   78 +
 .../Impl/Memory/PlatformMemoryManager.cs        |  107 +
 .../Impl/Memory/PlatformMemoryPool.cs           |  106 +
 .../Impl/Memory/PlatformMemoryStream.cs         |  677 ++++
 .../Impl/Memory/PlatformMemoryUtils.cs          |  463 +++
 .../Impl/Memory/PlatformPooledMemory.cs         |   70 +
 .../Impl/Memory/PlatformRawMemory.cs            |   89 +
 .../Impl/Memory/PlatformUnpooledMemory.cs       |   52 +
 .../Impl/Messaging/MessageFilterHolder.cs       |  179 +
 .../Impl/Messaging/Messaging.cs                 |  262 ++
 .../Impl/Messaging/MessagingAsync.cs            |   68 +
 .../Apache.Ignite.Core/Impl/NativeMethods.cs    |   47 +
 .../Apache.Ignite.Core/Impl/PlatformTarget.cs   |  715 ++++
 .../Portable/IPortableSystemTypeSerializer.cs   |   34 +
 .../Impl/Portable/IPortableTypeDescriptor.cs    |  124 +
 .../Impl/Portable/IPortableWriteAware.cs        |   34 +
 .../Impl/Portable/Io/IPortableStream.cs         |  320 ++
 .../Impl/Portable/Io/PortableAbstractStream.cs  | 1298 +++++++
 .../Impl/Portable/Io/PortableHeapStream.cs      |  447 +++
 .../Impl/Portable/Io/PortableStreamAdapter.cs   |  114 +
 .../Metadata/IPortableMetadataHandler.cs        |   41 +
 .../Metadata/PortableHashsetMetadataHandler.cs  |   69 +
 .../Portable/Metadata/PortableMetadataHolder.cs |  149 +
 .../Portable/Metadata/PortableMetadataImpl.cs   |  200 ++
 .../Impl/Portable/PortableBuilderField.cs       |   73 +
 .../Impl/Portable/PortableBuilderImpl.cs        |  923 +++++
 .../Impl/Portable/PortableCollectionInfo.cs     |  251 ++
 .../Impl/Portable/PortableFullTypeDescriptor.cs |  203 ++
 .../Impl/Portable/PortableHandleDictionary.cs   |  187 +
 .../Portable/PortableMarshalAwareSerializer.cs  |   45 +
 .../Impl/Portable/PortableMarshaller.cs         |  603 ++++
 .../Impl/Portable/PortableMode.cs               |   40 +
 .../Impl/Portable/PortableObjectHandle.cs       |   59 +
 .../PortableOrSerializableObjectHolder.cs       |   66 +
 .../Portable/PortableReaderHandleDictionary.cs  |   42 +
 .../Impl/Portable/PortableReaderImpl.cs         | 1013 ++++++
 .../Impl/Portable/PortableReflectiveRoutines.cs |  483 +++
 .../Portable/PortableReflectiveSerializer.cs    |  218 ++
 .../Portable/PortableSurrogateTypeDescriptor.cs |  133 +
 .../Impl/Portable/PortableSystemHandlers.cs     | 1336 +++++++
 .../Portable/PortableSystemTypeSerializer.cs    |   62 +
 .../Impl/Portable/PortableUserObject.cs         |  385 +++
 .../Impl/Portable/PortableUtils.cs              | 2039 +++++++++++
 .../Impl/Portable/PortableWriterImpl.cs         | 1305 +++++++
 .../Impl/Portable/PortablesImpl.cs              |  205 ++
 .../Impl/Portable/SerializableObjectHolder.cs   |   66 +
 .../Impl/Portable/TypeResolver.cs               |  227 ++
 .../Impl/Resource/IResourceInjector.cs          |   27 +
 .../Impl/Resource/ResourceFieldInjector.cs      |   47 +
 .../Impl/Resource/ResourceMethodInjector.cs     |   48 +
 .../Impl/Resource/ResourceProcessor.cs          |  105 +
 .../Impl/Resource/ResourcePropertyInjector.cs   |   47 +
 .../Impl/Resource/ResourceTypeDescriptor.cs     |  291 ++
 .../Impl/Services/ServiceContext.cs             |   60 +
 .../Impl/Services/ServiceDescriptor.cs          |  106 +
 .../Impl/Services/ServiceProxy.cs               |   71 +
 .../Impl/Services/ServiceProxyInvoker.cs        |  136 +
 .../Impl/Services/ServiceProxySerializer.cs     |  140 +
 .../Impl/Services/Services.cs                   |  316 ++
 .../Impl/Services/ServicesAsync.cs              |   89 +
 .../Impl/Transactions/AsyncTransaction.cs       |   78 +
 .../Impl/Transactions/Transaction.cs            |  155 +
 .../Impl/Transactions/TransactionImpl.cs        |  489 +++
 .../Impl/Transactions/TransactionMetricsImpl.cs |   62 +
 .../Impl/Transactions/TransactionsImpl.cs       |  201 ++
 .../Impl/Unmanaged/IUnmanagedTarget.cs          |   42 +
 .../Impl/Unmanaged/UnmanagedCallbackHandlers.cs |   99 +
 .../Impl/Unmanaged/UnmanagedCallbacks.cs        | 1154 +++++++
 .../Impl/Unmanaged/UnmanagedContext.cs          |   53 +
 .../Unmanaged/UnmanagedNonReleaseableTarget.cs  |   68 +
 .../Impl/Unmanaged/UnmanagedTarget.cs           |   77 +
 .../Impl/Unmanaged/UnmanagedUtils.cs            | 1263 +++++++
 .../Lifecycle/ILifecycleBean.cs                 |   64 +
 .../Lifecycle/LifecycleEventType.cs             |   49 +
 .../Messaging/IMessageFilter.cs                 |   35 +
 .../Apache.Ignite.Core/Messaging/IMessaging.cs  |  105 +
 .../Portable/IPortableBuilder.cs                |   78 +
 .../Portable/IPortableIdMapper.cs               |   40 +
 .../Portable/IPortableMarshalAware.cs           |   39 +
 .../Portable/IPortableMetadata.cs               |   61 +
 .../Portable/IPortableNameMapper.cs             |   39 +
 .../Portable/IPortableObject.cs                 |   44 +
 .../Portable/IPortableRawReader.cs              |  264 ++
 .../Portable/IPortableRawWriter.cs              |  221 ++
 .../Portable/IPortableReader.cs                 |  340 ++
 .../Portable/IPortableSerializer.cs             |   39 +
 .../Portable/IPortableWriter.cs                 |  259 ++
 .../Apache.Ignite.Core/Portable/IPortables.cs   |  120 +
 .../Portable/PortableConfiguration.cs           |  122 +
 .../Portable/PortableException.cs               |   64 +
 .../Portable/PortableTypeConfiguration.cs       |  162 +
 .../Portable/PortableTypeNames.cs               |  115 +
 .../Properties/AssemblyInfo.cs                  |   46 +
 .../Resource/InstanceResourceAttribute.cs       |   35 +
 .../Resource/StoreSessionResourceAttribute.cs   |   32 +
 .../Apache.Ignite.Core/Services/IService.cs     |   51 +
 .../Services/IServiceContext.cs                 |   69 +
 .../Services/IServiceDescriptor.cs              |   96 +
 .../Apache.Ignite.Core/Services/IServices.cs    |  181 +
 .../Services/ServiceConfiguration.cs            |   62 +
 .../Services/ServiceInvocationException.cs      |  101 +
 .../Transactions/ITransaction.cs                |  230 ++
 .../Transactions/ITransactionMetrics.cs         |   47 +
 .../Transactions/ITransactions.cs               |   73 +
 .../Transactions/TransactionConcurrency.cs      |   36 +
 .../TransactionHeuristicException.cs            |   72 +
 .../Transactions/TransactionIsolation.cs        |   41 +
 .../TransactionOptimisticException.cs           |   69 +
 .../TransactionRollbackException.cs             |   68 +
 .../Transactions/TransactionState.cs            |   70 +
 .../Transactions/TransactionTimeoutException.cs |   69 +
 .../platform/src/main/dotnet/Apache.Ignite.sln  |   68 +
 .../main/dotnet/Apache.Ignite.sln.DotSettings   |    4 +
 .../dotnet/Apache.Ignite/Apache.Ignite.csproj   |   76 +
 .../src/main/dotnet/Apache.Ignite/App.config    |   56 +
 .../Config/AppSettingsConfigurator.cs           |  113 +
 .../Apache.Ignite/Config/ArgsConfigurator.cs    |  164 +
 .../Apache.Ignite/Config/ConfigValueParser.cs   |   42 +
 .../Apache.Ignite/Config/IConfigurator.cs       |   34 +
 .../main/dotnet/Apache.Ignite/IgniteRunner.cs   |  171 +
 .../Apache.Ignite/Properties/AssemblyInfo.cs    |   35 +
 .../Apache.Ignite/Service/IgniteService.cs      |  219 ++
 .../Apache.Ignite/Service/NativeMethods.cs      |   57 +
 .../Apache.Ignite/Service/ServiceDescription.cs |   32 +
 ...processors.platform.PlatformBootstrapFactory |    2 +
 .../platform/PlatformAbstractBootstrap.java     |   48 +
 .../PlatformAbstractConfigurationClosure.java   |   61 +
 .../platform/PlatformAbstractPredicate.java     |   67 +
 .../platform/PlatformAbstractTarget.java        |  320 ++
 .../processors/platform/PlatformBootstrap.java  |   35 +
 .../platform/PlatformBootstrapFactory.java      |   37 +
 .../platform/PlatformConfigurationEx.java       |   48 +
 .../platform/PlatformContextImpl.java           |  621 ++++
 .../processors/platform/PlatformIgnition.java   |  189 +
 .../platform/PlatformProcessorImpl.java         |  359 ++
 .../platform/cache/PlatformCache.java           | 1090 ++++++
 .../cache/PlatformCacheEntryFilterImpl.java     |  106 +
 .../cache/PlatformCacheEntryProcessorImpl.java  |  220 ++
 .../platform/cache/PlatformCacheIterator.java   |   72 +
 .../PlatformCachePartialUpdateException.java    |   59 +
 .../cache/affinity/PlatformAffinity.java        |  296 ++
 .../query/PlatformAbstractQueryCursor.java      |  192 ++
 .../query/PlatformContinuousQueryImpl.java      |  235 ++
 .../PlatformContinuousQueryRemoteFilter.java    |  188 +
 .../cache/query/PlatformFieldsQueryCursor.java  |   49 +
 .../cache/query/PlatformQueryCursor.java        |   45 +
 .../cache/store/PlatformCacheStoreCallback.java |   61 +
 .../platform/cluster/PlatformClusterGroup.java  |  335 ++
 .../cluster/PlatformClusterNodeFilterImpl.java  |   78 +
 .../platform/compute/PlatformAbstractJob.java   |  156 +
 .../platform/compute/PlatformAbstractTask.java  |  206 ++
 .../PlatformBalancingMultiClosureTask.java      |   83 +
 ...tformBalancingSingleClosureAffinityTask.java |   88 +
 .../PlatformBalancingSingleClosureTask.java     |   81 +
 .../PlatformBroadcastingMultiClosureTask.java   |   87 +
 .../PlatformBroadcastingSingleClosureTask.java  |   84 +
 .../platform/compute/PlatformClosureJob.java    |  104 +
 .../platform/compute/PlatformCompute.java       |  332 ++
 .../platform/compute/PlatformFullJob.java       |  220 ++
 .../platform/compute/PlatformFullTask.java      |  192 ++
 .../platform/cpp/PlatformCppBootstrap.java      |   31 +
 .../cpp/PlatformCppBootstrapFactory.java        |   39 +
 .../cpp/PlatformCppConfigurationClosure.java    |   99 +
 .../cpp/PlatformCppConfigurationEx.java         |   82 +
 .../datastreamer/PlatformDataStreamer.java      |  226 ++
 .../PlatformStreamReceiverImpl.java             |  119 +
 .../dotnet/PlatformDotNetBootstrap.java         |   31 +
 .../dotnet/PlatformDotNetBootstrapFactory.java  |   39 +
 .../dotnet/PlatformDotNetCacheStore.java        |  497 +++
 .../PlatformDotNetConfigurationClosure.java     |  254 ++
 .../dotnet/PlatformDotNetConfigurationEx.java   |   90 +
 .../platform/dotnet/PlatformDotNetService.java  |   27 +
 .../dotnet/PlatformDotNetServiceImpl.java       |   47 +
 .../events/PlatformEventFilterListenerImpl.java |  163 +
 .../platform/events/PlatformEvents.java         |  396 +++
 .../lifecycle/PlatformLifecycleBean.java        |   75 +
 .../platform/memory/PlatformAbstractMemory.java |  121 +
 .../PlatformBigEndianInputStreamImpl.java       |  126 +
 .../PlatformBigEndianOutputStreamImpl.java      |  161 +
 .../platform/memory/PlatformExternalMemory.java |   55 +
 .../memory/PlatformInputStreamImpl.java         |  331 ++
 .../memory/PlatformMemoryManagerImpl.java       |   85 +
 .../platform/memory/PlatformMemoryPool.java     |  140 +
 .../platform/memory/PlatformMemoryUtils.java    |  467 +++
 .../memory/PlatformOutputStreamImpl.java        |  267 ++
 .../platform/memory/PlatformPooledMemory.java   |   64 +
 .../platform/memory/PlatformUnpooledMemory.java |   51 +
 .../messaging/PlatformMessageFilterImpl.java    |  110 +
 .../messaging/PlatformMessageLocalFilter.java   |  102 +
 .../platform/messaging/PlatformMessaging.java   |  166 +
 .../services/PlatformAbstractService.java       |  230 ++
 .../platform/services/PlatformServices.java     |  275 ++
 .../transactions/PlatformTransactions.java      |  259 ++
 .../platform/utils/PlatformFutureUtils.java     |  397 +++
 .../platform/utils/PlatformReaderBiClosure.java |   34 +
 .../platform/utils/PlatformReaderClosure.java   |   34 +
 .../platform/utils/PlatformUtils.java           |  768 +++++
 .../platform/utils/PlatformWriterBiClosure.java |   34 +
 .../platform/utils/PlatformWriterClosure.java   |   33 +
 .../platform/cpp/PlatformCppConfiguration.java  |   47 +
 .../ignite/platform/cpp/package-info.java       |   22 +
 .../dotnet/PlatformDotNetCacheStoreFactory.java |  139 +
 .../dotnet/PlatformDotNetLifecycleBean.java     |  109 +
 .../ignite/platform/dotnet/package-info.java    |   22 +
 .../apache/ignite/platform/package-info.java    |   22 +
 .../Apache.Ignite.Core.Tests.TestDll.csproj     |   52 +
 .../Properties/AssemblyInfo.cs                  |   49 +
 .../TestClass.cs                                |   35 +
 .../Apache.Ignite.Core.Tests.csproj             |  225 ++
 .../Cache/CacheAbstractTest.cs                  | 3252 ++++++++++++++++++
 .../Cache/CacheAffinityTest.cs                  |  139 +
 .../Cache/CacheDynamicStartTest.cs              |  281 ++
 .../Cache/CacheEntryTest.cs                     |   69 +
 .../Cache/CacheForkedTest.cs                    |   81 +
 .../Cache/CacheLocalAtomicTest.cs               |   57 +
 .../Cache/CacheLocalTest.cs                     |   56 +
 .../CachePartitionedAtomicNearEnabledTest.cs    |   50 +
 .../Cache/CachePartitionedAtomicTest.cs         |   50 +
 .../Cache/CachePartitionedNearEnabledTest.cs    |   50 +
 .../Cache/CachePartitionedTest.cs               |   50 +
 .../Cache/CacheReplicatedAtomicTest.cs          |   60 +
 .../Cache/CacheReplicatedTest.cs                |   60 +
 .../Cache/CacheTestAsyncWrapper.cs              |  436 +++
 .../Cache/Query/CacheQueriesTest.cs             |  928 +++++
 .../Continuous/ContinuousQueryAbstractTest.cs   | 1181 +++++++
 .../ContinuousQueryAtomicBackupTest.cs          |   33 +
 .../ContinuousQueryAtomicNoBackupTest.cs        |   34 +
 .../ContinuousQueryNoBackupAbstractTest.cs      |   72 +
 .../ContinuousQueryTransactionalBackupTest.cs   |   34 +
 .../ContinuousQueryTransactionalNoBackupTest.cs |   33 +
 .../Cache/Store/CacheParallelLoadStoreTest.cs   |  110 +
 .../Cache/Store/CacheStoreSessionTest.cs        |  285 ++
 .../Cache/Store/CacheStoreTest.cs               |  510 +++
 .../Cache/Store/CacheTestParallelLoadStore.cs   |   91 +
 .../Cache/Store/CacheTestStore.cs               |  155 +
 .../Compute/AbstractTaskTest.cs                 |  217 ++
 .../Compute/ClosureTaskTest.cs                  |  390 +++
 .../Compute/ComputeApiTest.cs                   | 1281 +++++++
 .../Compute/ComputeMultithreadedTest.cs         |  269 ++
 .../Compute/FailoverTaskSelfTest.cs             |  246 ++
 .../Forked/ForkedPortableClosureTaskTest.cs     |   30 +
 .../Compute/Forked/ForkedResourceTaskTest.cs    |   33 +
 .../Forked/ForkedSerializableClosureTaskTest.cs |   33 +
 .../Compute/Forked/ForkedTaskAdapterTest.cs     |   30 +
 .../Compute/IgniteExceptionTaskSelfTest.cs      |  753 ++++
 .../Compute/PortableClosureTaskTest.cs          |  217 ++
 .../Compute/PortableTaskTest.cs                 |  253 ++
 .../Compute/ResourceTaskTest.cs                 |  568 +++
 .../Compute/SerializableClosureTaskTest.cs      |  217 ++
 .../Compute/TaskAdapterTest.cs                  |  274 ++
 .../Compute/TaskResultTest.cs                   |  437 +++
 .../Config/Apache.Ignite.exe.config.test        |   41 +
 .../Config/Cache/Store/cache-store-session.xml  |   80 +
 .../Config/Compute/compute-grid1.xml            |   90 +
 .../Config/Compute/compute-grid2.xml            |   63 +
 .../Config/Compute/compute-grid3.xml            |   52 +
 .../Config/Compute/compute-standalone.xml       |   87 +
 .../Config/Dynamic/dynamic-client.xml           |   51 +
 .../Config/Dynamic/dynamic-data-no-cfg.xml      |   47 +
 .../Config/Dynamic/dynamic-data.xml             |   65 +
 .../Config/Lifecycle/lifecycle-beans.xml        |   66 +
 .../Config/Lifecycle/lifecycle-no-beans.xml     |   44 +
 .../Config/cache-portables.xml                  |   78 +
 .../Config/cache-query-continuous.xml           |  171 +
 .../Config/cache-query.xml                      |  100 +
 .../Config/marshaller-default.xml               |   43 +
 .../Config/marshaller-invalid.xml               |   46 +
 .../Config/marshaller-portable.xml              |   43 +
 .../native-client-test-cache-affinity.xml       |   70 +
 .../native-client-test-cache-parallel-store.xml |   69 +
 .../Config/native-client-test-cache-store.xml   |  125 +
 .../Config/native-client-test-cache.xml         |  194 ++
 .../Config/portable.xml                         |   56 +
 .../Config/start-test-grid1.xml                 |   54 +
 .../Config/start-test-grid2.xml                 |   45 +
 .../Config/start-test-grid3.xml                 |   43 +
 .../Dataload/DataStreamerTest.cs                |  592 ++++
 .../Apache.Ignite.Core.Tests/EventsTest.cs      |  961 ++++++
 .../Apache.Ignite.Core.Tests/ExceptionsTest.cs  |  352 ++
 .../Apache.Ignite.Core.Tests/ExecutableTest.cs  |  443 +++
 .../Apache.Ignite.Core.Tests/FutureTest.cs      |  278 ++
 .../IgniteManagerTest.cs                        |   51 +
 .../IgniteStartStopTest.cs                      |  422 +++
 .../Apache.Ignite.Core.Tests/LifecycleTest.cs   |  288 ++
 .../Apache.Ignite.Core.Tests/LoadDllTest.cs     |  243 ++
 .../Apache.Ignite.Core.Tests/MarshallerTest.cs  |   71 +
 .../Memory/InteropMemoryTest.cs                 |  213 ++
 .../Apache.Ignite.Core.Tests/MessagingTest.cs   |  646 ++++
 .../Portable/PortableApiSelfTest.cs             | 1787 ++++++++++
 .../Portable/PortableSelfTest.cs                | 2078 +++++++++++
 .../PortableConfigurationTest.cs                |  173 +
 .../Process/IIgniteProcessOutputReader.cs       |   35 +
 .../Process/IgniteProcess.cs                    |  283 ++
 .../Process/IgniteProcessConsoleOutputReader.cs |   40 +
 .../Properties/AssemblyInfo.cs                  |   35 +
 .../Query/ImplicitPortablePerson.cs             |   46 +
 .../Query/NoDefPortablePerson.cs                |   35 +
 .../Query/PortablePerson.cs                     |   69 +
 .../SerializationTest.cs                        |  240 ++
 .../Services/ServiceProxyTest.cs                |  741 ++++
 .../Services/ServicesAsyncWrapper.cs            |  174 +
 .../Services/ServicesTest.cs                    |  823 +++++
 .../Services/ServicesTestAsync.cs               |   33 +
 .../Apache.Ignite.Core.Tests/TestRunner.cs      |   71 +
 .../Apache.Ignite.Core.Tests/TestUtils.cs       |  292 ++
 .../TypeResolverTest.cs                         |  107 +
 .../platform/PlatformComputeBroadcastTask.java  |   73 +
 .../platform/PlatformComputeDecimalTask.java    |  106 +
 .../platform/PlatformComputeEchoTask.java       |  188 +
 .../ignite/platform/PlatformComputeEnum.java    |   28 +
 .../platform/PlatformComputeJavaPortable.java   |   39 +
 .../platform/PlatformComputePortable.java       |   42 +
 .../PlatformComputePortableArgTask.java         |  121 +
 .../platform/PlatformEventsWriteEventTask.java  |  146 +
 .../ignite/platform/PlatformMaxMemoryTask.java  |   57 +
 .../ignite/platform/PlatformMinMemoryTask.java  |   57 +
 .../lifecycle/PlatformJavaLifecycleBean.java    |   47 +
 .../lifecycle/PlatformJavaLifecycleTask.java    |   65 +
 modules/rest-http/pom.xml                       |    3 +-
 .../http/jetty/GridJettyJsonConfig.java         |    9 +-
 .../http/jetty/GridJettyRestHandler.java        |   72 +-
 .../http/jetty/GridJettyRestProtocol.java       |   59 +-
 .../rest/protocols/http/jetty/rest.html         |    4 +-
 modules/scalar-2.10/pom.xml                     |    5 +-
 modules/scalar/pom.xml                          |    5 +-
 modules/schedule/pom.xml                        |    3 +-
 .../schedule/IgniteScheduleProcessor.java       |   26 +-
 .../processors/schedule/ScheduleFutureImpl.java |   56 +-
 .../schedule/GridScheduleSelfTest.java          |   31 +-
 .../testsuites/IgniteSchedulerTestSuite.java    |    6 +-
 modules/schema-import/pom.xml                   |    3 +-
 .../ignite/schema/generator/CodeGenerator.java  |   36 +-
 .../ignite/schema/generator/XmlGenerator.java   |   51 +-
 .../apache/ignite/schema/model/IndexItem.java   |    2 +-
 .../ignite/schema/model/PojoDescriptor.java     |   60 +-
 .../apache/ignite/schema/model/PojoField.java   |   58 +-
 .../ignite/schema/model/SchemaDescriptor.java   |   62 +
 .../schema/parser/DatabaseMetadataParser.java   |   83 +-
 .../apache/ignite/schema/parser/DbColumn.java   |    2 +-
 .../apache/ignite/schema/parser/DbTable.java    |    6 +-
 .../parser/dialect/DB2MetadataDialect.java      |    9 +-
 .../parser/dialect/DatabaseMetadataDialect.java |   29 +-
 .../parser/dialect/JdbcMetadataDialect.java     |  148 +-
 .../parser/dialect/MySQLMetadataDialect.java    |   61 +
 .../parser/dialect/OracleMetadataDialect.java   |  255 +-
 .../ignite/schema/ui/ConfirmCallable.java       |   14 +-
 .../org/apache/ignite/schema/ui/Controls.java   |   88 +-
 .../org/apache/ignite/schema/ui/GridPaneEx.java |   13 +-
 .../org/apache/ignite/schema/ui/MessageBox.java |   35 +-
 .../apache/ignite/schema/ui/ModalDialog.java    |    4 +-
 .../ignite/schema/ui/SchemaImportApp.java       |  276 +-
 .../ignite/schema/ui/TextColumnValidator.java   |    2 +-
 .../schema/test/AbstractSchemaImportTest.java   |   30 +-
 .../test/generator/CodeGeneratorTest.java       |    7 +-
 .../schema/test/generator/XmlGeneratorTest.java |   14 +-
 .../test/parser/DbMetadataParserTest.java       |   17 +-
 .../testsuites/IgniteSchemaImportTestSuite.java |    9 +-
 modules/slf4j/pom.xml                           |    3 +-
 .../apache/ignite/logger/slf4j/Slf4jLogger.java |    9 +-
 modules/spark-2.10/pom.xml                      |    3 +-
 modules/spark/pom.xml                           |    3 +-
 .../ignite/spark/JavaIgniteRDDSelfTest.java     |   36 +-
 modules/spring/pom.xml                          |    3 +-
 .../java/org/apache/ignite/IgniteSpring.java    |   23 +-
 .../org/apache/ignite/IgniteSpringBean.java     |   40 +-
 .../apache/ignite/cache/spring/SpringCache.java |    8 +-
 .../ignite/cache/spring/SpringCacheManager.java |   21 +-
 .../spring/CacheSpringStoreSessionListener.java |   33 +-
 .../GridResourceSpringBeanInjector.java         |   17 +-
 .../resource/GridSpringResourceContextImpl.java |   10 +-
 .../util/spring/IgniteSpringHelperImpl.java     |   63 +-
 .../jdbc/CacheJdbcBlobStoreFactorySelfTest.java |   27 +-
 .../jdbc/CacheJdbcPojoStoreFactorySelfTest.java |   40 +-
 ...CacheSpringStoreSessionListenerSelfTest.java |   36 +-
 .../ignite/internal/GridFactorySelfTest.java    |   92 +-
 .../GridSpringBeanSerializationSelfTest.java    |   21 +-
 .../internal/IgniteDynamicCacheConfigTest.java  |   20 +-
 .../apache/ignite/internal/TestNodeFilter.java  |    6 +-
 .../resource/GridLoggerInjectionSelfTest.java   |   21 +-
 .../resource/GridResourceProcessorSelfTest.java |   41 +-
 .../resource/GridResourceTestUtils.java         |    4 +-
 .../resource/GridServiceInjectionSelfTest.java  |   16 +-
 .../processors/resource/TestClosure.java        |   12 +-
 .../p2p/GridP2PUserVersionChangeSelfTest.java   |   50 +-
 .../spring/GridSpringCacheManagerSelfTest.java  |   19 +-
 .../ignite/spring/GridSpringCacheTestKey.java   |    4 +-
 .../spring/GridSpringCacheTestKeyGenerator.java |    7 +-
 .../spring/GridSpringCacheTestService.java      |    9 +-
 .../GridSpringDynamicCacheTestService.java      |    9 +-
 .../IgniteExcludeInConfigurationTest.java       |   21 +-
 .../IgniteStartFromStreamConfigurationTest.java |   18 +-
 .../testsuites/IgniteResourceSelfTestSuite.java |    8 +-
 .../testsuites/IgniteSpringTestSuite.java       |   19 +-
 modules/ssh/pom.xml                             |    3 +-
 .../util/nodestart/IgniteSshHelperImpl.java     |    2 +-
 .../util/nodestart/StartNodeCallableImpl.java   |   39 +-
 .../internal/IgniteNodeStartUtilsSelfTest.java  |   25 +-
 ...gniteProjectionStartStopRestartSelfTest.java |   62 +-
 .../IgniteStartStopRestartTestSuite.java        |    3 +-
 modules/tools/pom.xml                           |    3 +-
 .../ant/beautifier/GridJavadocAntTask.java      |   29 +-
 .../GridJavadocCharArrayLexReader.java          |    2 +-
 .../tools/ant/beautifier/GridJavadocToken.java  |    2 +-
 .../ant/beautifier/GridJavadocTokenType.java    |    2 +-
 .../ignite/tools/classgen/ClassesGenerator.java |   27 +-
 .../ignite/tools/javadoc/IgniteLinkTaglet.java  |   11 +-
 .../util/antgar/IgniteDeploymentGarAntTask.java |   15 +-
 modules/urideploy/pom.xml                       |    3 +-
 .../uri/GridUriDeploymentClassLoader.java       |   15 +-
 .../GridUriDeploymentClassLoaderFactory.java    |   16 +-
 .../uri/GridUriDeploymentDiscovery.java         |   18 +-
 .../uri/GridUriDeploymentFileProcessor.java     |   42 +-
 .../GridUriDeploymentFileProcessorResult.java   |   11 +-
 .../GridUriDeploymentFileResourceLoader.java    |    9 +-
 .../uri/GridUriDeploymentJarVerifier.java       |   26 +-
 .../uri/GridUriDeploymentSpringDocument.java    |   17 +-
 .../uri/GridUriDeploymentSpringParser.java      |   20 +-
 .../uri/GridUriDeploymentUnitDescriptor.java    |   28 +-
 .../uri/GridUriDeploymentUriParser.java         |    6 +-
 .../spi/deployment/uri/UriDeploymentSpi.java    |   68 +-
 .../deployment/uri/UriDeploymentSpiMBean.java   |    9 +-
 .../uri/scanners/GridDeploymentFileHandler.java |    4 +-
 .../GridDeploymentFolderScannerHelper.java      |    5 +-
 .../GridUriDeploymentScannerListener.java       |    7 +-
 .../uri/scanners/UriDeploymentScanner.java      |    4 +-
 .../scanners/UriDeploymentScannerContext.java   |   12 +-
 .../scanners/UriDeploymentScannerManager.java   |   18 +-
 .../scanners/file/UriDeploymentFileScanner.java |   32 +-
 .../scanners/http/UriDeploymentHttpScanner.java |   58 +-
 .../GridTaskUriDeploymentDeadlockSelfTest.java  |   32 +-
 .../ignite/p2p/GridP2PDisabledSelfTest.java     |   24 +-
 .../uri/GridUriDeploymentAbstractSelfTest.java  |   10 +-
 ...loymentClassLoaderMultiThreadedSelfTest.java |   15 +-
 .../GridUriDeploymentClassLoaderSelfTest.java   |   17 +-
 ...riDeploymentClassloaderRegisterSelfTest.java |   29 +-
 .../uri/GridUriDeploymentConfigSelfTest.java    |    8 +-
 .../GridUriDeploymentFileProcessorSelfTest.java |   19 +-
 .../uri/GridUriDeploymentMd5CheckSelfTest.java  |   21 +-
 ...loymentMultiScannersErrorThrottlingTest.java |   11 +-
 .../GridUriDeploymentMultiScannersSelfTest.java |   13 +-
 .../uri/GridUriDeploymentSimpleSelfTest.java    |   27 +-
 .../file/GridFileDeploymentSelfTest.java        |   14 +-
 .../GridFileDeploymentUndeploySelfTest.java     |   19 +-
 .../http/GridHttpDeploymentSelfTest.java        |   29 +-
 .../testsuites/IgniteToolsSelfTestSuite.java    |    6 +-
 .../IgniteUriDeploymentTestSuite.java           |   20 +-
 .../apache/ignite/tools/GridToolsSelfTest.java  |   23 +-
 modules/visor-console-2.10/pom.xml              |    3 +-
 modules/visor-console/pom.xml                   |    3 +-
 modules/visor-plugins/pom.xml                   |    3 +-
 .../visor/plugin/VisorExtensionPoint.java       |    4 +-
 .../ignite/visor/plugin/VisorPluggableTab.java  |    5 +-
 .../apache/ignite/visor/plugin/VisorPlugin.java |    6 +-
 .../visor/plugin/VisorPluginComponent.java      |    4 +-
 .../plugin/VisorPluginComponentsFactory.java    |    6 +-
 .../visor/plugin/VisorPluginLogPanel.java       |    2 +-
 .../ignite/visor/plugin/VisorPluginModel.java   |   12 +-
 .../visor/plugin/VisorPluginNodesPanel.java     |    5 +-
 .../visor/plugin/VisorSelectionListener.java    |    4 +-
 .../visor/plugin/VisorTopologyListener.java     |    4 +-
 modules/web/pom.xml                             |    3 +-
 .../ignite/cache/websession/WebSession.java     |   30 +-
 .../cache/websession/WebSessionFilter.java      |   64 +-
 .../cache/websession/WebSessionListener.java    |   36 +-
 .../servlet/ServletContextListenerStartup.java  |   32 +-
 .../ignite/startup/servlet/ServletStartup.java  |   28 +-
 .../IgniteWebSessionSelfTestSuite.java          |    8 +-
 .../internal/websession/WebSessionSelfTest.java |   46 +-
 .../websession/WebSessionServerStart.java       |   22 +-
 .../internal/websession/WebSessionTest.java     |   11 +-
 modules/yardstick/README.txt                    |    2 +-
 .../yardstick/config/benchmark-query.properties |    5 +-
 modules/yardstick/config/ignite-base-config.xml |    2 +-
 modules/yardstick/config/ignite-jdbc-config.xml |   55 +
 modules/yardstick/pom.xml                       |    3 +-
 .../yardstick/IgniteAbstractBenchmark.java      |   25 +-
 .../yardstick/IgniteBenchmarkArguments.java     |   32 +-
 .../org/apache/ignite/yardstick/IgniteNode.java |   48 +-
 .../cache/IgniteCacheAbstractBenchmark.java     |    8 +-
 .../yardstick/cache/IgniteGetBenchmark.java     |    7 +-
 .../cache/IgniteGetOffHeapBenchmark.java        |    6 +-
 .../cache/IgniteGetOffHeapValuesBenchmark.java  |    6 +-
 .../cache/IgniteJdbcSqlQueryBenchmark.java      |   21 +-
 .../yardstick/cache/IgnitePutAllBenchmark.java  |   15 +-
 .../cache/IgnitePutAllTxBenchmark.java          |   15 +-
 .../yardstick/cache/IgnitePutBenchmark.java     |    9 +-
 .../yardstick/cache/IgnitePutGetBenchmark.java  |    9 +-
 .../cache/IgnitePutGetOffHeapBenchmark.java     |    6 +-
 .../IgnitePutGetOffHeapValuesBenchmark.java     |    6 +-
 .../cache/IgnitePutGetTxBenchmark.java          |   11 +-
 .../cache/IgnitePutGetTxOffHeapBenchmark.java   |    6 +-
 .../IgnitePutGetTxOffHeapValuesBenchmark.java   |    6 +-
 .../cache/IgnitePutIndexedValue1Benchmark.java  |    9 +-
 .../cache/IgnitePutIndexedValue2Benchmark.java  |    9 +-
 .../cache/IgnitePutIndexedValue8Benchmark.java  |    9 +-
 .../cache/IgnitePutOffHeapBenchmark.java        |    6 +-
 .../cache/IgnitePutOffHeapValuesBenchmark.java  |    6 +-
 .../yardstick/cache/IgnitePutTxBenchmark.java   |    9 +-
 .../cache/IgnitePutTxOffHeapBenchmark.java      |    6 +-
 .../IgnitePutTxOffHeapValuesBenchmark.java      |    6 +-
 .../cache/IgniteSqlQueryBenchmark.java          |   23 +-
 .../cache/IgniteSqlQueryJoinBenchmark.java      |   23 +-
 .../IgniteSqlQueryJoinOffHeapBenchmark.java     |    6 +-
 .../cache/IgniteSqlQueryOffHeapBenchmark.java   |    6 +-
 .../cache/IgniteSqlQueryPutBenchmark.java       |   18 +-
 .../IgniteSqlQueryPutOffHeapBenchmark.java      |    6 +-
 .../yardstick/cache/model/Organization.java     |   10 +-
 .../ignite/yardstick/cache/model/Person.java    |   10 +-
 .../ignite/yardstick/cache/model/Person1.java   |    7 +-
 .../ignite/yardstick/cache/model/Person2.java   |    7 +-
 .../ignite/yardstick/cache/model/Person8.java   |    7 +-
 .../ignite/yardstick/cache/model/SampleKey.java |    7 +-
 .../yardstick/cache/model/SampleValue.java      |    7 +-
 .../jdbc/IgniteJdbcStoreAbstractBenchmark.java  |   20 +-
 .../store/jdbc/IgniteJdbcStoreGetBenchmark.java |    9 +-
 .../jdbc/IgniteJdbcStoreGetTxBenchmark.java     |    9 +-
 .../store/jdbc/IgniteJdbcStorePutBenchmark.java |   10 +-
 .../jdbc/IgniteJdbcStorePutGetBenchmark.java    |   10 +-
 .../jdbc/IgniteJdbcStorePutGetTxBenchmark.java  |   10 +-
 .../jdbc/IgniteJdbcStorePutTxBenchmark.java     |   10 +-
 .../compute/IgniteAffinityCallBenchmark.java    |   11 +-
 .../yardstick/compute/IgniteApplyBenchmark.java |   18 +-
 .../compute/IgniteBroadcastBenchmark.java       |    9 +-
 .../compute/IgniteExecuteBenchmark.java         |    9 +-
 .../yardstick/compute/IgniteRunBenchmark.java   |   18 +-
 .../yardstick/compute/model/NoopCallable.java   |   10 +-
 .../yardstick/compute/model/NoopTask.java       |   21 +-
 modules/yarn/pom.xml                            |    3 +-
 .../apache/ignite/yarn/ApplicationMaster.java   |   45 +-
 .../apache/ignite/yarn/ClusterProperties.java   |   15 +-
 .../org/apache/ignite/yarn/IgniteContainer.java |    5 +-
 .../org/apache/ignite/yarn/IgniteProvider.java  |   24 +-
 .../apache/ignite/yarn/IgniteYarnClient.java    |   39 +-
 .../ignite/yarn/utils/IgniteYarnUtils.java      |   15 +-
 .../org/apache/ignite/IgniteYarnTestSuite.java  |    6 +-
 .../yarn/IgniteApplicationMasterSelfTest.java   |   52 +-
 modules/zookeeper/pom.xml                       |   90 +
 .../zk/TcpDiscoveryZookeeperIpFinder.java       |  356 ++
 .../tcp/ipfinder/zk/ZookeeperIpFinderTest.java  |  395 +++
 parent/pom.xml                                  |   44 +-
 pom.xml                                         |  138 +-
 scripts/apply-pull-request.sh                   |  220 ++
 scripts/git-format-patch.sh                     |    2 +-
 5171 files changed, 230400 insertions(+), 41549 deletions(-)
----------------------------------------------------------------------



[52/55] [abbrv] ignite git commit: ignite-1171 Test for reproducing added

Posted by ag...@apache.org.
ignite-1171 Test for reproducing added


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

Branch: refs/heads/ignite-1171
Commit: e75917d47aaa34d8ff6219245668959fb68f0227
Parents: 5f5220a
Author: agura <ag...@gridgain.com>
Authored: Wed Sep 16 18:23:16 2015 +0300
Committer: agura <ag...@gridgain.com>
Committed: Wed Sep 16 18:23:16 2015 +0300

----------------------------------------------------------------------
 .../distributed/CacheAffEarlySelfTest.java      | 245 +++++++++++++++++++
 .../ignite/testsuites/IgniteBasicTestSuite.java |   5 +
 2 files changed, 250 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/e75917d4/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/CacheAffEarlySelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/CacheAffEarlySelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/CacheAffEarlySelfTest.java
new file mode 100644
index 0000000..7f0ca11
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/CacheAffEarlySelfTest.java
@@ -0,0 +1,245 @@
+/*
+ * 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 java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Random;
+import java.util.UUID;
+import org.apache.ignite.Ignite;
+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.IgniteConfiguration;
+import org.apache.ignite.internal.IgniteFutureTimeoutCheckedException;
+import org.apache.ignite.internal.IgniteInternalFuture;
+import org.apache.ignite.internal.IgniteKernal;
+import org.apache.ignite.internal.util.typedef.G;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.lang.IgniteFuture;
+import org.apache.ignite.lang.IgniteFutureTimeoutException;
+import org.apache.ignite.lang.IgniteInClosure;
+import org.apache.ignite.marshaller.optimized.OptimizedMarshaller;
+import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi;
+import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+
+/**
+ *
+ */
+public class CacheAffEarlySelfTest extends GridCommonAbstractTest {
+    /** Grid count. */
+    private static int GRID_CNT = 8;
+
+    /** Operation timeout. */
+    private static long OP_TIMEOUT = 5000;
+
+    /** Always dump threads or only once per operation. */
+    private static boolean ALWAYS_DUMP_THREADS = false;
+
+    /** Stopped. */
+    private volatile boolean stopped;
+
+    /** Iteration. */
+    private int iters = 10;
+
+    /** Concurrent. */
+    private boolean concurrent = true;
+
+    /** Futs. */
+    private Collection<IgniteInternalFuture<?>> futs = new ArrayList<>(GRID_CNT);
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(gridName);
+
+        TcpDiscoveryVmIpFinder finder = new TcpDiscoveryVmIpFinder(true);
+        finder.setAddresses(Collections.singletonList("127.0.0.1:47500..47510"));
+
+        TcpDiscoverySpi discoSpi = new TcpDiscoverySpi();
+        discoSpi.setIpFinder(finder);
+
+        cfg.setDiscoverySpi(discoSpi);
+
+        OptimizedMarshaller marsh = new OptimizedMarshaller();
+        marsh.setRequireSerializable(false);
+
+        cfg.setMarshaller(marsh);
+
+        return cfg;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected long getTestTimeout() {
+        return 6 * 60 * 1000L;
+    }
+
+    /**
+     *
+     */
+    public void testStartNodes() throws Exception {
+        for (int i = 0; i < iters; i++) {
+            try {
+                System.out.println("*** Iteration " + (i + 1) + '/' + iters);
+
+                IgniteInternalFuture<?> fut = multithreadedAsync(new Runnable() {
+                    @Override public void run() {
+                        try {
+                            doTest();
+                        }
+                        catch (Exception e) {
+                            e.printStackTrace();
+                        }
+                    }
+                }, 1);
+
+                fut.get(30000);
+            }
+            catch (IgniteFutureTimeoutCheckedException e) {
+                // No-op.
+            }
+            finally {
+                stopAllGrids(true);
+            }
+        }
+    }
+
+    /**
+     *
+     */
+    public void doTest() throws Exception {
+        for (int i = 0; i < GRID_CNT; i++) {
+            final int idx = i;
+
+            final Ignite grid = concurrent ? null : startGrid(idx);
+
+            IgniteInternalFuture<?> fut = multithreadedAsync(new Runnable() {
+                @Override public void run() {
+                    Random rnd = new Random();
+
+                    try {
+                        final Ignite ignite = grid == null ? startGrid(idx) : grid;
+
+                        final IgniteCache<UUID, UUID> cache = getCache(ignite).withAsync();
+
+                        CacheAffEarlySelfTest.this.execute(cache, new IgniteInClosure<IgniteCache<UUID,UUID>>() {
+                            @Override public void apply(IgniteCache<UUID, UUID> entries) {
+                                cache.put(ignite.cluster().localNode().id(), UUID.randomUUID());
+                            }
+                        });
+
+                        while (!stopped) {
+                            int val = Math.abs(rnd.nextInt(100));
+                            if (val >= 0 && val < 40)
+                                execute(cache, new IgniteInClosure<IgniteCache<UUID, UUID>>() {
+                                    @Override public void apply(IgniteCache<UUID, UUID> entries) {
+                                        cache.containsKey(ignite.cluster().localNode().id());
+                                    }
+                                });
+                            else if (val >= 40 && val < 80)
+                                execute(cache, new IgniteInClosure<IgniteCache<UUID, UUID>>() {
+                                    @Override public void apply(IgniteCache<UUID, UUID> entries) {
+                                        cache.get(ignite.cluster().localNode().id());
+                                    }
+                                });
+                            else
+                                execute(cache, new IgniteInClosure<IgniteCache<UUID, UUID>>() {
+                                    @Override public void apply(IgniteCache<UUID, UUID> entries) {
+                                        cache.put(ignite.cluster().localNode().id(), UUID.randomUUID());
+                                    }
+                                });
+
+                            Thread.sleep(50);
+                        }
+                    }
+                    catch (Exception e) {
+                        e.printStackTrace();
+                    }
+                }
+            }, 1);
+
+            futs.add(fut);
+        }
+
+        Thread.sleep(10000);
+
+        stopped = true;
+
+        for (IgniteInternalFuture<?> fut : futs)
+            fut.get();
+    }
+
+    /**
+     * @param cache Cache.
+     * @param c Closure.
+     */
+    private void execute(IgniteCache<UUID, UUID> cache, IgniteInClosure<IgniteCache<UUID, UUID>> c) {
+        c.apply(cache);
+
+        IgniteFuture<Object> fut = cache.future();
+
+        boolean success = false;
+
+        int iter = 0;
+
+        while (!success) {
+            try {
+                fut.get(OP_TIMEOUT);
+
+                success = true;
+            }
+            catch (IgniteFutureTimeoutException e) {
+                debug(iter == 0 || ALWAYS_DUMP_THREADS);
+            }
+
+            iter++;
+        }
+    }
+
+    /**
+     *
+     */
+    private void debug(boolean dumpThreads) {
+        log.info("DUMPING DEBUG INFO:");
+
+        for (Ignite ignite : G.allGrids())
+            ((IgniteKernal)ignite).dumpDebugInfo();
+
+        if (dumpThreads) {
+            U.dumpThreads(null);
+
+            U.dumpThreads(log);
+        }
+    }
+
+    /**
+     * @param grid Grid.
+     */
+    private IgniteCache<UUID, UUID> getCache(Ignite grid) {
+        CacheConfiguration<UUID, UUID> ccfg = defaultCacheConfiguration();
+
+        ccfg.setCacheMode(CacheMode.PARTITIONED);
+        ccfg.setAtomicityMode(CacheAtomicityMode.ATOMIC);
+        ccfg.setBackups(1);
+        ccfg.setNearConfiguration(null);
+
+        return grid.getOrCreateCache(ccfg);
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/e75917d4/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteBasicTestSuite.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteBasicTestSuite.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteBasicTestSuite.java
index 1a9913f..390807c 100644
--- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteBasicTestSuite.java
+++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteBasicTestSuite.java
@@ -36,6 +36,7 @@ import org.apache.ignite.internal.processors.affinity.GridAffinityProcessorRende
 import org.apache.ignite.internal.processors.cache.GridProjectionForCachesOnDaemonNodeSelfTest;
 import org.apache.ignite.internal.processors.cache.IgniteDaemonNodeMarshallerCacheTest;
 import org.apache.ignite.internal.processors.cache.OffHeapTieredTransactionSelfTest;
+import org.apache.ignite.internal.processors.cache.distributed.CacheAffEarlySelfTest;
 import org.apache.ignite.internal.processors.closure.GridClosureProcessorSelfTest;
 import org.apache.ignite.internal.processors.continuous.GridEventConsumeSelfTest;
 import org.apache.ignite.internal.processors.continuous.GridMessageListenSelfTest;
@@ -68,6 +69,9 @@ public class IgniteBasicTestSuite extends TestSuite {
     public static TestSuite suite(Set<Class> ignoredTests) throws Exception {
         TestSuite suite = new TestSuite("Ignite Basic Test Suite");
 
+        suite.addTest(new TestSuite(CacheAffEarlySelfTest.class));
+
+/*
         suite.addTest(IgniteLangSelfTestSuite.suite());
         suite.addTest(IgniteUtilSelfTestSuite.suite(ignoredTests));
         suite.addTest(IgniteMarshallerSelfTestSuite.suite(ignoredTests));
@@ -110,6 +114,7 @@ public class IgniteBasicTestSuite extends TestSuite {
         suite.addTestSuite(OffHeapTieredTransactionSelfTest.class);
         suite.addTestSuite(IgniteSlowClientDetectionSelfTest.class);
         suite.addTestSuite(IgniteDaemonNodeMarshallerCacheTest.class);
+*/
 
         return suite;
     }


[09/55] [abbrv] ignite git commit: ignite-1462: hid portable API in 1.4 release

Posted by ag...@apache.org.
http://git-wip-us.apache.org/repos/asf/ignite/blob/71379a80/modules/core/src/main/java/org/apache/ignite/internal/portable/api/PortableMarshaller.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/api/PortableMarshaller.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/api/PortableMarshaller.java
new file mode 100644
index 0000000..de0df8d
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/portable/api/PortableMarshaller.java
@@ -0,0 +1,358 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.portable.api;
+
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.sql.Timestamp;
+import java.util.ArrayList;
+import java.util.Collection;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.internal.portable.GridPortableMarshaller;
+import org.apache.ignite.internal.portable.PortableContext;
+import org.apache.ignite.marshaller.AbstractMarshaller;
+import org.apache.ignite.marshaller.MarshallerContext;
+import org.apache.ignite.internal.portable.api.PortableException;
+import org.apache.ignite.internal.portable.api.PortableIdMapper;
+import org.apache.ignite.internal.portable.api.PortableObject;
+import org.apache.ignite.internal.portable.api.PortableProtocolVersion;
+import org.apache.ignite.internal.portable.api.PortableSerializer;
+import org.apache.ignite.internal.portable.api.PortableTypeConfiguration;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Implementation of {@link org.apache.ignite.marshaller.Marshaller} that lets to serialize and deserialize all objects
+ * in the portable format.
+ * <p>
+ * {@code PortableMarshaller} is tested only on Java HotSpot VM on other VMs it could yield unexpected results.
+ * <p>
+ * <h1 class="header">Configuration</h1>
+ * <h2 class="header">Mandatory</h2>
+ * This marshaller has no mandatory configuration parameters.
+ * <h2 class="header">Java Example</h2>
+ * <pre name="code" class="java">
+ * PortableMarshaller marshaller = new PortableMarshaller();
+ *
+ * IgniteConfiguration cfg = new IgniteConfiguration();
+ *
+ * // Override marshaller.
+ * cfg.setMarshaller(marshaller);
+ *
+ * // Starts grid.
+ * G.start(cfg);
+ * </pre>
+ * <h2 class="header">Spring Example</h2>
+ * PortableMarshaller can be configured from Spring XML configuration file:
+ * <pre name="code" class="xml">
+ * &lt;bean id="grid.custom.cfg" class="org.apache.ignite.configuration.IgniteConfiguration" singleton="true"&gt;
+ *     ...
+ *     &lt;property name="marshaller"&gt;
+ *         &lt;bean class="org.apache.ignite.internal.portable.api.PortableMarshaller"&gt;
+ *            ...
+ *         &lt;/bean&gt;
+ *     &lt;/property&gt;
+ *     ...
+ * &lt;/bean&gt;
+ * </pre>
+ * <p>
+ * <img src="http://ignite.apache.org/images/spring-small.png">
+ * <br>
+ * For information about Spring framework visit <a href="http://www.springframework.org/">www.springframework.org</a>
+ */
+public class PortableMarshaller extends AbstractMarshaller {
+    /** Default portable protocol version. */
+    public static final PortableProtocolVersion DFLT_PORTABLE_PROTO_VER = PortableProtocolVersion.VER_1_4_0;
+
+    /** Class names. */
+    private Collection<String> clsNames;
+
+    /** ID mapper. */
+    private PortableIdMapper idMapper;
+
+    /** Serializer. */
+    private PortableSerializer serializer;
+
+    /** Types. */
+    private Collection<PortableTypeConfiguration> typeCfgs;
+
+    /** Use timestamp flag. */
+    private boolean useTs = true;
+
+    /** Whether to convert string to bytes using UTF-8 encoding. */
+    private boolean convertString = true;
+
+    /** Meta data enabled flag. */
+    private boolean metaDataEnabled = true;
+
+    /** Keep deserialized flag. */
+    private boolean keepDeserialized = true;
+
+    /** Protocol version. */
+    private PortableProtocolVersion protoVer = DFLT_PORTABLE_PROTO_VER;
+
+    /** */
+    private GridPortableMarshaller impl;
+
+    /**
+     * Gets class names.
+     *
+     * @return Class names.
+     */
+    public Collection<String> getClassNames() {
+        return clsNames;
+    }
+
+    /**
+     * Sets class names of portable objects explicitly.
+     *
+     * @param clsNames Class names.
+     */
+    public void setClassNames(Collection<String> clsNames) {
+        this.clsNames = new ArrayList<>(clsNames.size());
+
+        for (String clsName : clsNames)
+            this.clsNames.add(clsName.trim());
+    }
+
+    /**
+     * Gets ID mapper.
+     *
+     * @return ID mapper.
+     */
+    public PortableIdMapper getIdMapper() {
+        return idMapper;
+    }
+
+    /**
+     * Sets ID mapper.
+     *
+     * @param idMapper ID mapper.
+     */
+    public void setIdMapper(PortableIdMapper idMapper) {
+        this.idMapper = idMapper;
+    }
+
+    /**
+     * Gets serializer.
+     *
+     * @return Serializer.
+     */
+    public PortableSerializer getSerializer() {
+        return serializer;
+    }
+
+    /**
+     * Sets serializer.
+     *
+     * @param serializer Serializer.
+     */
+    public void setSerializer(PortableSerializer serializer) {
+        this.serializer = serializer;
+    }
+
+    /**
+     * Gets types configuration.
+     *
+     * @return Types configuration.
+     */
+    public Collection<PortableTypeConfiguration> getTypeConfigurations() {
+        return typeCfgs;
+    }
+
+    /**
+     * Sets type configurations.
+     *
+     * @param typeCfgs Type configurations.
+     */
+    public void setTypeConfigurations(Collection<PortableTypeConfiguration> typeCfgs) {
+        this.typeCfgs = typeCfgs;
+    }
+
+    /**
+     * If {@code true} then date values converted to {@link Timestamp} on deserialization.
+     * <p>
+     * Default value is {@code true}.
+     *
+     * @return Flag indicating whether date values converted to {@link Timestamp} during unmarshalling.
+     */
+    public boolean isUseTimestamp() {
+        return useTs;
+    }
+
+    /**
+     * @param useTs Flag indicating whether date values converted to {@link Timestamp} during unmarshalling.
+     */
+    public void setUseTimestamp(boolean useTs) {
+        this.useTs = useTs;
+    }
+
+    /**
+     * Gets strings must be converted to or from bytes using UTF-8 encoding.
+     * <p>
+     * Default value is {@code true}.
+     *
+     * @return Flag indicating whether string must be converted to byte array using UTF-8 encoding.
+     */
+    public boolean isConvertStringToBytes() {
+        return convertString;
+    }
+
+    /**
+     * Sets strings must be converted to or from bytes using UTF-8 encoding.
+     * <p>
+     * Default value is {@code true}.
+     *
+     * @param convertString Flag indicating whether string must be converted to byte array using UTF-8 encoding.
+     */
+    public void setConvertStringToBytes(boolean convertString) {
+        this.convertString = convertString;
+    }
+
+    /**
+     * If {@code true}, meta data will be collected or all types. If you need to override this behaviour for
+     * some specific type, use {@link PortableTypeConfiguration#setMetaDataEnabled(Boolean)} method.
+     * <p>
+     * Default value if {@code true}.
+     *
+     * @return Whether meta data is collected.
+     */
+    public boolean isMetaDataEnabled() {
+        return metaDataEnabled;
+    }
+
+    /**
+     * @param metaDataEnabled Whether meta data is collected.
+     */
+    public void setMetaDataEnabled(boolean metaDataEnabled) {
+        this.metaDataEnabled = metaDataEnabled;
+    }
+
+    /**
+     * If {@code true}, {@link PortableObject} will cache deserialized instance after
+     * {@link PortableObject#deserialize()} is called. All consequent calls of this
+     * method on the same instance of {@link PortableObject} will return that cached
+     * value without actually deserializing portable object. If you need to override this
+     * behaviour for some specific type, use {@link PortableTypeConfiguration#setKeepDeserialized(Boolean)}
+     * method.
+     * <p>
+     * Default value if {@code true}.
+     *
+     * @return Whether deserialized value is kept.
+     */
+    public boolean isKeepDeserialized() {
+        return keepDeserialized;
+    }
+
+    /**
+     * @param keepDeserialized Whether deserialized value is kept.
+     */
+    public void setKeepDeserialized(boolean keepDeserialized) {
+        this.keepDeserialized = keepDeserialized;
+    }
+
+    /**
+     * Gets portable protocol version.
+     * <p>
+     * Defaults to {@link #DFLT_PORTABLE_PROTO_VER}.
+     *
+     * @return Portable protocol version.
+     */
+    public PortableProtocolVersion getProtocolVersion() {
+        return protoVer;
+    }
+
+    /**
+     * Sets portable protocol version.
+     * <p>
+     * Defaults to {@link #DFLT_PORTABLE_PROTO_VER}.
+     *
+     * @param protoVer Portable protocol version.
+     */
+    public void setProtocolVersion(PortableProtocolVersion protoVer) {
+        if (protoVer == null)
+            throw new IllegalArgumentException("Wrong portable protocol version: " + protoVer);
+
+        this.protoVer = protoVer;
+    }
+
+    /**
+     * Returns currently set {@link MarshallerContext}.
+     *
+     * @return Marshaller context.
+     */
+    public MarshallerContext getContext() {
+        return ctx;
+    }
+
+    /**
+     * Sets {@link PortableContext}.
+     * <p/>
+     * @param ctx Portable context.
+     */
+    private void setPortableContext(PortableContext ctx) {
+        ctx.configure(this);
+
+        impl = new GridPortableMarshaller(ctx);
+    }
+
+    /** {@inheritDoc} */
+    @Override public byte[] marshal(@Nullable Object obj) throws IgniteCheckedException {
+        return impl.marshal(obj, 0);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void marshal(@Nullable Object obj, OutputStream out) throws IgniteCheckedException {
+        byte[] arr = marshal(obj);
+
+        try {
+            out.write(arr);
+        }
+        catch (IOException e) {
+            throw new PortableException("Failed to marshal the object: " + obj, e);
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public <T> T unmarshal(byte[] bytes, @Nullable ClassLoader clsLdr) throws IgniteCheckedException {
+        return impl.deserialize(bytes, clsLdr);
+    }
+
+    /** {@inheritDoc} */
+    @Override public <T> T unmarshal(InputStream in, @Nullable ClassLoader clsLdr) throws IgniteCheckedException {
+        ByteArrayOutputStream buffer = new ByteArrayOutputStream();
+
+        byte[] arr = new byte[4096];
+        int cnt;
+
+        // we have to fully read the InputStream because GridPortableMarshaller requires support of a method that
+        // returns number of bytes remaining.
+        try {
+            while ((cnt = in.read(arr)) != -1)
+                buffer.write(arr, 0, cnt);
+
+            buffer.flush();
+
+            return impl.deserialize(buffer.toByteArray(), clsLdr);
+        }
+        catch (IOException e) {
+            throw new PortableException("Failed to unmarshal the object from InputStream", e);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/71379a80/modules/core/src/main/java/org/apache/ignite/internal/portable/api/PortableMetadata.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/api/PortableMetadata.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/api/PortableMetadata.java
new file mode 100644
index 0000000..a90bdca
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/portable/api/PortableMetadata.java
@@ -0,0 +1,60 @@
+/*
+ * 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.api;
+
+import java.util.Collection;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Portable type meta data. Metadata for portable types can be accessed from any of the
+ * {@link IgnitePortables#metadata(String)} methods.
+ * Having metadata also allows for proper formatting of {@code PortableObject#toString()} method,
+ * even when portable objects are kept in binary format only, which may be necessary for audit reasons.
+ */
+public interface PortableMetadata {
+    /**
+     * Gets portable type name.
+     *
+     * @return Portable type name.
+     */
+    public String typeName();
+
+    /**
+     * Gets collection of all field names for this portable type.
+     *
+     * @return Collection of all field names for this portable type.
+     */
+    public Collection<String> fields();
+
+    /**
+     * Gets name of the field type for a given field.
+     *
+     * @param fieldName Field name.
+     * @return Field type name.
+     */
+    @Nullable public String fieldTypeName(String fieldName);
+
+    /**
+     * Portable objects can optionally specify custom key-affinity mapping in the
+     * configuration. This method returns the name of the field which should be
+     * used for the key-affinity mapping.
+     *
+     * @return Affinity key field name.
+     */
+    @Nullable public String affinityKeyFieldName();
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/71379a80/modules/core/src/main/java/org/apache/ignite/internal/portable/api/PortableObject.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/api/PortableObject.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/api/PortableObject.java
new file mode 100644
index 0000000..ec965c6
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/portable/api/PortableObject.java
@@ -0,0 +1,152 @@
+/*
+ * 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.api;
+
+import java.io.Serializable;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.LinkedList;
+import java.util.TreeMap;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Wrapper for portable object in portable binary format. Once an object is defined as portable,
+ * Ignite will always store it in memory in the portable (i.e. binary) format.
+ * User can choose to work either with the portable format or with the deserialized form
+ * (assuming that class definitions are present in the classpath).
+ * <p>
+ * <b>NOTE:</b> user does not need to (and should not) implement this interface directly.
+ * <p>
+ * To work with the portable format directly, user should create a cache projection
+ * over {@code PortableObject} class and then retrieve individual fields as needed:
+ * <pre name=code class=java>
+ * IgniteCache&lt;PortableObject, PortableObject&gt; prj = cache.withKeepPortable();
+ *
+ * // Convert instance of MyKey to portable format.
+ * // We could also use GridPortableBuilder to create the key in portable format directly.
+ * PortableObject key = grid.portables().toPortable(new MyKey());
+ *
+ * PortableObject val = prj.get(key);
+ *
+ * String field = val.field("myFieldName");
+ * </pre>
+ * Alternatively, if we have class definitions in the classpath, we may choose to work with deserialized
+ * typed objects at all times. In this case we do incur the deserialization cost. However, if
+ * {@link PortableMarshaller#isKeepDeserialized()} is {@code true} then Ignite will only deserialize on the first access
+ * and will cache the deserialized object, so it does not have to be deserialized again:
+ * <pre name=code class=java>
+ * IgniteCache&lt;MyKey.class, MyValue.class&gt; cache = grid.cache(null);
+ *
+ * MyValue val = cache.get(new MyKey());
+ *
+ * // Normal java getter.
+ * String fieldVal = val.getMyFieldName();
+ * </pre>
+ * <h1 class="header">Working With Maps and Collections</h1>
+ * All maps and collections in the portable objects are serialized automatically. When working
+ * with different platforms, e.g. C++ or .NET, Ignite will automatically pick the most
+ * adequate collection or map in either language. For example, {@link ArrayList} in Java will become
+ * {@code List} in C#, {@link LinkedList} in Java is {@link LinkedList} in C#, {@link HashMap}
+ * in Java is {@code Dictionary} in C#, and {@link TreeMap} in Java becomes {@code SortedDictionary}
+ * in C#, etc.
+ * <h1 class="header">Dynamic Structure Changes</h1>
+ * Since objects are always cached in the portable binary format, server does not need to
+ * be aware of the class definitions. Moreover, if class definitions are not present or not
+ * used on the server, then clients can continuously change the structure of the portable
+ * objects without having to restart the cluster. For example, if one client stores a
+ * certain class with fields A and B, and another client stores the same class with
+ * fields B and C, then the server-side portable object will have the fields A, B, and C.
+ * As the structure of a portable object changes, the new fields become available for SQL queries
+ * automatically.
+ * <h1 class="header">Building Portable Objects</h1>
+ * Ignite comes with {@link PortableBuilder} which allows to build portable objects dynamically:
+ * <pre name=code class=java>
+ * PortableBuilder builder = Ignition.ignite().portables().builder("org.project.MyObject");
+ *
+ * builder.setField("fieldA", "A");
+ * builder.setField("fieldB", "B");
+ *
+ * PortableObject portableObj = builder.build();
+ * </pre>
+ * For the cases when class definition is present
+ * in the class path, it is also possible to populate a standard POJO and then
+ * convert it to portable format, like so:
+ * <pre name=code class=java>
+ * MyObject obj = new MyObject();
+ *
+ * obj.setFieldA("A");
+ * obj.setFieldB(123);
+ *
+ * PortableObject portableObj = Ignition.ignite().portables().toPortable(obj);
+ * </pre>
+ * <h1 class="header">Portable Metadata</h1>
+ * Even though Ignite portable protocol only works with hash codes for type and field names
+ * to achieve better performance, Ignite provides metadata for all portable types which
+ * can be queried ar runtime via any of the {@link IgnitePortables#metadata(Class)}
+ * methods. Having metadata also allows for proper formatting of {@code PortableObject.toString()} method,
+ * even when portable objects are kept in binary format only, which may be necessary for audit reasons.
+ */
+public interface PortableObject extends Serializable, Cloneable {
+    /**
+     * Gets portable object type ID.
+     *
+     * @return Type ID.
+     */
+    public int typeId();
+
+    /**
+     * Gets meta data for this portable object.
+     *
+     * @return Meta data.
+     * @throws PortableException In case of error.
+     */
+    @Nullable public PortableMetadata metaData() throws PortableException;
+
+    /**
+     * Gets field value.
+     *
+     * @param fieldName Field name.
+     * @return Field value.
+     * @throws PortableException In case of any other error.
+     */
+    @Nullable public <F> F field(String fieldName) throws PortableException;
+
+    /**
+     * Checks whether field is set.
+     *
+     * @param fieldName Field name.
+     * @return {@code true} if field is set.
+     */
+    public boolean hasField(String fieldName);
+
+    /**
+     * Gets fully deserialized instance of portable object.
+     *
+     * @return Fully deserialized instance of portable object.
+     * @throws PortableInvalidClassException If class doesn't exist.
+     * @throws PortableException In case of any other error.
+     */
+    @Nullable public <T> T deserialize() throws PortableException;
+
+    /**
+     * Copies this portable object.
+     *
+     * @return Copy of this portable object.
+     */
+    public PortableObject clone() throws CloneNotSupportedException;
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/71379a80/modules/core/src/main/java/org/apache/ignite/internal/portable/api/PortableProtocolVersion.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/api/PortableProtocolVersion.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/api/PortableProtocolVersion.java
new file mode 100644
index 0000000..741c2a5
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/portable/api/PortableProtocolVersion.java
@@ -0,0 +1,41 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.portable.api;
+
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Portable protocol version.
+ */
+public enum PortableProtocolVersion {
+    /** Ignite 1.4.0 release. */
+    VER_1_4_0;
+
+    /** Enumerated values. */
+    private static final PortableProtocolVersion[] VALS = values();
+
+    /**
+     * Efficiently gets enumerated value from its ordinal.
+     *
+     * @param ord Ordinal value.
+     * @return Enumerated value or {@code null} if ordinal out of range.
+     */
+    @Nullable public static PortableProtocolVersion fromOrdinal(int ord) {
+        return ord >= 0 && ord < VALS.length ? VALS[ord] : null;
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/71379a80/modules/core/src/main/java/org/apache/ignite/internal/portable/api/PortableRawReader.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/api/PortableRawReader.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/api/PortableRawReader.java
new file mode 100644
index 0000000..c12aa1a
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/portable/api/PortableRawReader.java
@@ -0,0 +1,234 @@
+/*
+ * 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.api;
+
+import java.math.BigDecimal;
+import java.sql.Timestamp;
+import java.util.Collection;
+import java.util.Date;
+import java.util.Map;
+import java.util.UUID;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Raw reader for portable objects. Raw reader does not use field name hash codes, therefore,
+ * making the format even more compact. However, if the raw reader is used,
+ * dynamic structure changes to the portable objects are not supported.
+ */
+public interface PortableRawReader {
+    /**
+     * @return Byte value.
+     * @throws PortableException In case of error.
+     */
+    public byte readByte() throws PortableException;
+
+    /**
+     * @return Short value.
+     * @throws PortableException In case of error.
+     */
+    public short readShort() throws PortableException;
+
+    /**
+     * @return Integer value.
+     * @throws PortableException In case of error.
+     */
+    public int readInt() throws PortableException;
+
+    /**
+     * @return Long value.
+     * @throws PortableException In case of error.
+     */
+    public long readLong() throws PortableException;
+
+    /**
+     * @return Float value.
+     * @throws PortableException In case of error.
+     */
+    public float readFloat() throws PortableException;
+
+    /**
+     * @return Double value.
+     * @throws PortableException In case of error.
+     */
+    public double readDouble() throws PortableException;
+
+    /**
+     * @return Char value.
+     * @throws PortableException In case of error.
+     */
+    public char readChar() throws PortableException;
+
+    /**
+     * @return Boolean value.
+     * @throws PortableException In case of error.
+     */
+    public boolean readBoolean() throws PortableException;
+
+    /**
+     * @return Decimal value.
+     * @throws PortableException In case of error.
+     */
+    @Nullable public BigDecimal readDecimal() throws PortableException;
+
+    /**
+     * @return String value.
+     * @throws PortableException In case of error.
+     */
+    @Nullable public String readString() throws PortableException;
+
+    /**
+     * @return UUID.
+     * @throws PortableException In case of error.
+     */
+    @Nullable public UUID readUuid() throws PortableException;
+
+    /**
+     * @return Date.
+     * @throws PortableException In case of error.
+     */
+    @Nullable public Date readDate() throws PortableException;
+
+    /**
+     * @return Timestamp.
+     * @throws PortableException In case of error.
+     */
+    @Nullable public Timestamp readTimestamp() throws PortableException;
+
+    /**
+     * @return Object.
+     * @throws PortableException In case of error.
+     */
+    @Nullable public <T> T readObject() throws PortableException;
+
+    /**
+     * @return Byte array.
+     * @throws PortableException In case of error.
+     */
+    @Nullable public byte[] readByteArray() throws PortableException;
+
+    /**
+     * @return Short array.
+     * @throws PortableException In case of error.
+     */
+    @Nullable public short[] readShortArray() throws PortableException;
+
+    /**
+     * @return Integer array.
+     * @throws PortableException In case of error.
+     */
+    @Nullable public int[] readIntArray() throws PortableException;
+
+    /**
+     * @return Long array.
+     * @throws PortableException In case of error.
+     */
+    @Nullable public long[] readLongArray() throws PortableException;
+
+    /**
+     * @return Float array.
+     * @throws PortableException In case of error.
+     */
+    @Nullable public float[] readFloatArray() throws PortableException;
+
+    /**
+     * @return Byte array.
+     * @throws PortableException In case of error.
+     */
+    @Nullable public double[] readDoubleArray() throws PortableException;
+
+    /**
+     * @return Char array.
+     * @throws PortableException In case of error.
+     */
+    @Nullable public char[] readCharArray() throws PortableException;
+
+    /**
+     * @return Boolean array.
+     * @throws PortableException In case of error.
+     */
+    @Nullable public boolean[] readBooleanArray() throws PortableException;
+
+    /**
+     * @return Decimal array.
+     * @throws PortableException In case of error.
+     */
+    @Nullable public BigDecimal[] readDecimalArray() throws PortableException;
+
+    /**
+     * @return String array.
+     * @throws PortableException In case of error.
+     */
+    @Nullable public String[] readStringArray() throws PortableException;
+
+    /**
+     * @return UUID array.
+     * @throws PortableException In case of error.
+     */
+    @Nullable public UUID[] readUuidArray() throws PortableException;
+
+    /**
+     * @return Date array.
+     * @throws PortableException In case of error.
+     */
+    @Nullable public Date[] readDateArray() throws PortableException;
+
+    /**
+     * @return Object array.
+     * @throws PortableException In case of error.
+     */
+    @Nullable public Object[] readObjectArray() throws PortableException;
+
+    /**
+     * @return Collection.
+     * @throws PortableException In case of error.
+     */
+    @Nullable public <T> Collection<T> readCollection() throws PortableException;
+
+    /**
+     * @param colCls Collection class.
+     * @return Collection.
+     * @throws PortableException In case of error.
+     */
+    @Nullable public <T> Collection<T> readCollection(Class<? extends Collection<T>> colCls)
+        throws PortableException;
+
+    /**
+     * @return Map.
+     * @throws PortableException In case of error.
+     */
+    @Nullable public <K, V> Map<K, V> readMap() throws PortableException;
+
+    /**
+     * @param mapCls Map class.
+     * @return Map.
+     * @throws PortableException In case of error.
+     */
+    @Nullable public <K, V> Map<K, V> readMap(Class<? extends Map<K, V>> mapCls) throws PortableException;
+
+    /**
+     * @return Value.
+     * @throws PortableException In case of error.
+     */
+    @Nullable public <T extends Enum<?>> T readEnum() throws PortableException;
+
+    /**
+     * @return Value.
+     * @throws PortableException In case of error.
+     */
+    @Nullable public <T extends Enum<?>> T[] readEnumArray() throws PortableException;
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/71379a80/modules/core/src/main/java/org/apache/ignite/internal/portable/api/PortableRawWriter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/api/PortableRawWriter.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/api/PortableRawWriter.java
new file mode 100644
index 0000000..91f0e3b
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/portable/api/PortableRawWriter.java
@@ -0,0 +1,219 @@
+/*
+ * 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.api;
+
+import java.math.BigDecimal;
+import java.sql.Timestamp;
+import java.util.Collection;
+import java.util.Date;
+import java.util.Map;
+import java.util.UUID;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Raw writer for portable object. Raw writer does not write field name hash codes, therefore,
+ * making the format even more compact. However, if the raw writer is used,
+ * dynamic structure changes to the portable objects are not supported.
+ */
+public interface PortableRawWriter {
+    /**
+     * @param val Value to write.
+     * @throws PortableException In case of error.
+     */
+    public void writeByte(byte val) throws PortableException;
+
+    /**
+     * @param val Value to write.
+     * @throws PortableException In case of error.
+     */
+    public void writeShort(short val) throws PortableException;
+
+    /**
+     * @param val Value to write.
+     * @throws PortableException In case of error.
+     */
+    public void writeInt(int val) throws PortableException;
+
+    /**
+     * @param val Value to write.
+     * @throws PortableException In case of error.
+     */
+    public void writeLong(long val) throws PortableException;
+
+    /**
+     * @param val Value to write.
+     * @throws PortableException In case of error.
+     */
+    public void writeFloat(float val) throws PortableException;
+
+    /**
+     * @param val Value to write.
+     * @throws PortableException In case of error.
+     */
+    public void writeDouble(double val) throws PortableException;
+
+    /**
+     * @param val Value to write.
+     * @throws PortableException In case of error.
+     */
+    public void writeChar(char val) throws PortableException;
+
+    /**
+     * @param val Value to write.
+     * @throws PortableException In case of error.
+     */
+    public void writeBoolean(boolean val) throws PortableException;
+
+    /**
+     * @param val Value to write.
+     * @throws PortableException In case of error.
+     */
+    public void writeDecimal(@Nullable BigDecimal val) throws PortableException;
+
+    /**
+     * @param val Value to write.
+     * @throws PortableException In case of error.
+     */
+    public void writeString(@Nullable String val) throws PortableException;
+
+    /**
+     * @param val UUID to write.
+     * @throws PortableException In case of error.
+     */
+    public void writeUuid(@Nullable UUID val) throws PortableException;
+
+    /**
+     * @param val Date to write.
+     * @throws PortableException In case of error.
+     */
+    public void writeDate(@Nullable Date val) throws PortableException;
+
+    /**
+     * @param val Timestamp to write.
+     * @throws PortableException In case of error.
+     */
+    public void writeTimestamp(@Nullable Timestamp val) throws PortableException;
+
+    /**
+     * @param obj Value to write.
+     * @throws PortableException In case of error.
+     */
+    public void writeObject(@Nullable Object obj) throws PortableException;
+
+    /**
+     * @param val Value to write.
+     * @throws PortableException In case of error.
+     */
+    public void writeByteArray(@Nullable byte[] val) throws PortableException;
+
+    /**
+     * @param val Value to write.
+     * @throws PortableException In case of error.
+     */
+    public void writeShortArray(@Nullable short[] val) throws PortableException;
+
+    /**
+     * @param val Value to write.
+     * @throws PortableException In case of error.
+     */
+    public void writeIntArray(@Nullable int[] val) throws PortableException;
+
+    /**
+     * @param val Value to write.
+     * @throws PortableException In case of error.
+     */
+    public void writeLongArray(@Nullable long[] val) throws PortableException;
+
+    /**
+     * @param val Value to write.
+     * @throws PortableException In case of error.
+     */
+    public void writeFloatArray(@Nullable float[] val) throws PortableException;
+
+    /**
+     * @param val Value to write.
+     * @throws PortableException In case of error.
+     */
+    public void writeDoubleArray(@Nullable double[] val) throws PortableException;
+
+    /**
+     * @param val Value to write.
+     * @throws PortableException In case of error.
+     */
+    public void writeCharArray(@Nullable char[] val) throws PortableException;
+
+    /**
+     * @param val Value to write.
+     * @throws PortableException In case of error.
+     */
+    public void writeBooleanArray(@Nullable boolean[] val) throws PortableException;
+
+    /**
+     * @param val Value to write.
+     * @throws PortableException In case of error.
+     */
+    public void writeDecimalArray(@Nullable BigDecimal[] val) throws PortableException;
+
+    /**
+     * @param val Value to write.
+     * @throws PortableException In case of error.
+     */
+    public void writeStringArray(@Nullable String[] val) throws PortableException;
+
+    /**
+     * @param val Value to write.
+     * @throws PortableException In case of error.
+     */
+    public void writeUuidArray(@Nullable UUID[] val) throws PortableException;
+
+    /**
+     * @param val Value to write.
+     * @throws PortableException In case of error.
+     */
+    public void writeDateArray(@Nullable Date[] val) throws PortableException;
+
+    /**
+     * @param val Value to write.
+     * @throws PortableException In case of error.
+     */
+    public void writeObjectArray(@Nullable Object[] val) throws PortableException;
+
+    /**
+     * @param col Collection to write.
+     * @throws PortableException In case of error.
+     */
+    public <T> void writeCollection(@Nullable Collection<T> col) throws PortableException;
+
+    /**
+     * @param map Map to write.
+     * @throws PortableException In case of error.
+     */
+    public <K, V> void writeMap(@Nullable Map<K, V> map) throws PortableException;
+
+    /**
+     * @param val Value to write.
+     * @throws PortableException In case of error.
+     */
+    public <T extends Enum<?>> void writeEnum(T val) throws PortableException;
+
+    /**
+     * @param val Value to write.
+     * @throws PortableException In case of error.
+     */
+    public <T extends Enum<?>> void writeEnumArray(T[] val) throws PortableException;
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/71379a80/modules/core/src/main/java/org/apache/ignite/internal/portable/api/PortableReader.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/api/PortableReader.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/api/PortableReader.java
new file mode 100644
index 0000000..ca322e7
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/portable/api/PortableReader.java
@@ -0,0 +1,284 @@
+/*
+ * 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.api;
+
+import java.math.BigDecimal;
+import java.sql.Timestamp;
+import java.util.Collection;
+import java.util.Date;
+import java.util.Map;
+import java.util.UUID;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Reader for portable objects used in {@link PortableMarshalAware} implementations.
+ * Useful for the cases when user wants a fine-grained control over serialization.
+ * <p>
+ * Note that Ignite never writes full strings for field or type names. Instead,
+ * for performance reasons, Ignite writes integer hash codes for type and field names.
+ * It has been tested that hash code conflicts for the type names or the field names
+ * within the same type are virtually non-existent and, to gain performance, it is safe
+ * to work with hash codes. For the cases when hash codes for different types or fields
+ * actually do collide, Ignite provides {@link PortableIdMapper} which
+ * allows to override the automatically generated hash code IDs for the type and field names.
+ */
+public interface PortableReader {
+    /**
+     * @param fieldName Field name.
+     * @return Byte value.
+     * @throws PortableException In case of error.
+     */
+    public byte readByte(String fieldName) throws PortableException;
+
+    /**
+     * @param fieldName Field name.
+     * @return Short value.
+     * @throws PortableException In case of error.
+     */
+    public short readShort(String fieldName) throws PortableException;
+
+    /**
+     * @param fieldName Field name.
+     * @return Integer value.
+     * @throws PortableException In case of error.
+     */
+    public int readInt(String fieldName) throws PortableException;
+
+    /**
+     * @param fieldName Field name.
+     * @return Long value.
+     * @throws PortableException In case of error.
+     */
+    public long readLong(String fieldName) throws PortableException;
+
+    /**
+     * @param fieldName Field name.
+     * @throws PortableException In case of error.
+     * @return Float value.
+     */
+    public float readFloat(String fieldName) throws PortableException;
+
+    /**
+     * @param fieldName Field name.
+     * @return Double value.
+     * @throws PortableException In case of error.
+     */
+    public double readDouble(String fieldName) throws PortableException;
+
+    /**
+     * @param fieldName Field name.
+     * @return Char value.
+     * @throws PortableException In case of error.
+     */
+    public char readChar(String fieldName) throws PortableException;
+
+    /**
+     * @param fieldName Field name.
+     * @return Boolean value.
+     * @throws PortableException In case of error.
+     */
+    public boolean readBoolean(String fieldName) throws PortableException;
+
+    /**
+     * @param fieldName Field name.
+     * @return Decimal value.
+     * @throws PortableException In case of error.
+     */
+    @Nullable public BigDecimal readDecimal(String fieldName) throws PortableException;
+
+    /**
+     * @param fieldName Field name.
+     * @return String value.
+     * @throws PortableException In case of error.
+     */
+    @Nullable public String readString(String fieldName) throws PortableException;
+
+    /**
+     * @param fieldName Field name.
+     * @return UUID.
+     * @throws PortableException In case of error.
+     */
+    @Nullable public UUID readUuid(String fieldName) throws PortableException;
+
+    /**
+     * @param fieldName Field name.
+     * @return Date.
+     * @throws PortableException In case of error.
+     */
+    @Nullable public Date readDate(String fieldName) throws PortableException;
+
+    /**
+     * @param fieldName Field name.
+     * @return Timestamp.
+     * @throws PortableException In case of error.
+     */
+    @Nullable public Timestamp readTimestamp(String fieldName) throws PortableException;
+
+    /**
+     * @param fieldName Field name.
+     * @return Object.
+     * @throws PortableException In case of error.
+     */
+    @Nullable public <T> T readObject(String fieldName) throws PortableException;
+
+    /**
+     * @param fieldName Field name.
+     * @return Byte array.
+     * @throws PortableException In case of error.
+     */
+    @Nullable public byte[] readByteArray(String fieldName) throws PortableException;
+
+    /**
+     * @param fieldName Field name.
+     * @return Short array.
+     * @throws PortableException In case of error.
+     */
+    @Nullable public short[] readShortArray(String fieldName) throws PortableException;
+
+    /**
+     * @param fieldName Field name.
+     * @return Integer array.
+     * @throws PortableException In case of error.
+     */
+    @Nullable public int[] readIntArray(String fieldName) throws PortableException;
+
+    /**
+     * @param fieldName Field name.
+     * @return Long array.
+     * @throws PortableException In case of error.
+     */
+    @Nullable public long[] readLongArray(String fieldName) throws PortableException;
+
+    /**
+     * @param fieldName Field name.
+     * @return Float array.
+     * @throws PortableException In case of error.
+     */
+    @Nullable public float[] readFloatArray(String fieldName) throws PortableException;
+
+    /**
+     * @param fieldName Field name.
+     * @return Byte array.
+     * @throws PortableException In case of error.
+     */
+    @Nullable public double[] readDoubleArray(String fieldName) throws PortableException;
+
+    /**
+     * @param fieldName Field name.
+     * @return Char array.
+     * @throws PortableException In case of error.
+     */
+    @Nullable public char[] readCharArray(String fieldName) throws PortableException;
+
+    /**
+     * @param fieldName Field name.
+     * @return Boolean array.
+     * @throws PortableException In case of error.
+     */
+    @Nullable public boolean[] readBooleanArray(String fieldName) throws PortableException;
+
+    /**
+     * @param fieldName Field name.
+     * @return Decimal array.
+     * @throws PortableException In case of error.
+     */
+    @Nullable public BigDecimal[] readDecimalArray(String fieldName) throws PortableException;
+
+    /**
+     * @param fieldName Field name.
+     * @return String array.
+     * @throws PortableException In case of error.
+     */
+    @Nullable public String[] readStringArray(String fieldName) throws PortableException;
+
+    /**
+     * @param fieldName Field name.
+     * @return UUID array.
+     * @throws PortableException In case of error.
+     */
+    @Nullable public UUID[] readUuidArray(String fieldName) throws PortableException;
+
+    /**
+     * @param fieldName Field name.
+     * @return Date array.
+     * @throws PortableException In case of error.
+     */
+    @Nullable public Date[] readDateArray(String fieldName) throws PortableException;
+
+    /**
+     * @param fieldName Field name.
+     * @return Object array.
+     * @throws PortableException In case of error.
+     */
+    @Nullable public Object[] readObjectArray(String fieldName) throws PortableException;
+
+    /**
+     * @param fieldName Field name.
+     * @return Collection.
+     * @throws PortableException In case of error.
+     */
+    @Nullable public <T> Collection<T> readCollection(String fieldName) throws PortableException;
+
+    /**
+     * @param fieldName Field name.
+     * @param colCls Collection class.
+     * @return Collection.
+     * @throws PortableException In case of error.
+     */
+    @Nullable public <T> Collection<T> readCollection(String fieldName, Class<? extends Collection<T>> colCls)
+        throws PortableException;
+
+    /**
+     * @param fieldName Field name.
+     * @return Map.
+     * @throws PortableException In case of error.
+     */
+    @Nullable public <K, V> Map<K, V> readMap(String fieldName) throws PortableException;
+
+    /**
+     * @param fieldName Field name.
+     * @param mapCls Map class.
+     * @return Map.
+     * @throws PortableException In case of error.
+     */
+    @Nullable public <K, V> Map<K, V> readMap(String fieldName, Class<? extends Map<K, V>> mapCls)
+        throws PortableException;
+
+    /**
+     * @param fieldName Field name.
+     * @return Value.
+     * @throws PortableException In case of error.
+     */
+    @Nullable public <T extends Enum<?>> T readEnum(String fieldName) throws PortableException;
+
+    /**
+     * @param fieldName Field name.
+     * @return Value.
+     * @throws PortableException In case of error.
+     */
+    @Nullable public <T extends Enum<?>> T[] readEnumArray(String fieldName) throws PortableException;
+
+    /**
+     * Gets raw reader. Raw reader does not use field name hash codes, therefore,
+     * making the format even more compact. However, if the raw reader is used,
+     * dynamic structure changes to the portable objects are not supported.
+     *
+     * @return Raw reader.
+     */
+    public PortableRawReader rawReader();
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/71379a80/modules/core/src/main/java/org/apache/ignite/internal/portable/api/PortableSerializer.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/api/PortableSerializer.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/api/PortableSerializer.java
new file mode 100644
index 0000000..b9e835f
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/portable/api/PortableSerializer.java
@@ -0,0 +1,47 @@
+/*
+ * 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.api;
+
+/**
+ * Interface that allows to implement custom serialization logic for portable objects.
+ * Can be used instead of {@link PortableMarshalAware} in case if the class
+ * cannot be changed directly.
+ * <p>
+ * Portable serializer can be configured for all portable objects via
+ * {@link PortableMarshaller#getSerializer()} method, or for a specific
+ * portable type via {@link PortableTypeConfiguration#getSerializer()} method.
+ */
+public interface PortableSerializer {
+    /**
+     * Writes fields to provided writer.
+     *
+     * @param obj Empty object.
+     * @param writer Portable object writer.
+     * @throws PortableException In case of error.
+     */
+    public void writePortable(Object obj, PortableWriter writer) throws PortableException;
+
+    /**
+     * Reads fields from provided reader.
+     *
+     * @param obj Empty object
+     * @param reader Portable object reader.
+     * @throws PortableException In case of error.
+     */
+    public void readPortable(Object obj, PortableReader reader) throws PortableException;
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/71379a80/modules/core/src/main/java/org/apache/ignite/internal/portable/api/PortableTypeConfiguration.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/api/PortableTypeConfiguration.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/api/PortableTypeConfiguration.java
new file mode 100644
index 0000000..80a043e
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/portable/api/PortableTypeConfiguration.java
@@ -0,0 +1,195 @@
+/*
+ * 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.api;
+
+import java.sql.Timestamp;
+import java.util.Collection;
+import org.apache.ignite.internal.util.typedef.internal.S;
+
+/**
+ * Defines configuration properties for a specific portable type. Providing per-type
+ * configuration is optional, as it is generally enough, and also optional, to provide global portable
+ * configuration using {@link PortableMarshaller#setClassNames(Collection)}.
+ * However, this class allows you to change configuration properties for a specific
+ * portable type without affecting configuration for other portable types.
+ * <p>
+ * Per-type portable configuration can be specified in {@link PortableMarshaller#getTypeConfigurations()} method.
+ */
+public class PortableTypeConfiguration {
+    /** Class name. */
+    private String clsName;
+
+    /** ID mapper. */
+    private PortableIdMapper idMapper;
+
+    /** Serializer. */
+    private PortableSerializer serializer;
+
+    /** Use timestamp flag. */
+    private Boolean useTs;
+
+    /** Meta data enabled flag. */
+    private Boolean metaDataEnabled;
+
+    /** Keep deserialized flag. */
+    private Boolean keepDeserialized;
+
+    /** Affinity key field name. */
+    private String affKeyFieldName;
+
+    /**
+     */
+    public PortableTypeConfiguration() {
+        // No-op.
+    }
+
+    /**
+     * @param clsName Class name.
+     */
+    public PortableTypeConfiguration(String clsName) {
+        this.clsName = clsName;
+    }
+
+    /**
+     * Gets type name.
+     *
+     * @return Type name.
+     */
+    public String getClassName() {
+        return clsName;
+    }
+
+    /**
+     * Sets type name.
+     *
+     * @param clsName Type name.
+     */
+    public void setClassName(String clsName) {
+        this.clsName = clsName;
+    }
+
+    /**
+     * Gets ID mapper.
+     *
+     * @return ID mapper.
+     */
+    public PortableIdMapper getIdMapper() {
+        return idMapper;
+    }
+
+    /**
+     * Sets ID mapper.
+     *
+     * @param idMapper ID mapper.
+     */
+    public void setIdMapper(PortableIdMapper idMapper) {
+        this.idMapper = idMapper;
+    }
+
+    /**
+     * Gets serializer.
+     *
+     * @return Serializer.
+     */
+    public PortableSerializer getSerializer() {
+        return serializer;
+    }
+
+    /**
+     * Sets serializer.
+     *
+     * @param serializer Serializer.
+     */
+    public void setSerializer(PortableSerializer serializer) {
+        this.serializer = serializer;
+    }
+
+    /**
+     * If {@code true} then date values converted to {@link Timestamp} during unmarshalling.
+     *
+     * @return Flag indicating whether date values converted to {@link Timestamp} during unmarshalling.
+     */
+    public Boolean isUseTimestamp() {
+        return useTs;
+    }
+
+    /**
+     * @param useTs Flag indicating whether date values converted to {@link Timestamp} during unmarshalling.
+     */
+    public void setUseTimestamp(Boolean useTs) {
+        this.useTs = useTs;
+    }
+
+    /**
+     * Defines whether meta data is collected for this type. If provided, this value will override
+     * {@link PortableMarshaller#isMetaDataEnabled()} property.
+     *
+     * @return Whether meta data is collected.
+     */
+    public Boolean isMetaDataEnabled() {
+        return metaDataEnabled;
+    }
+
+    /**
+     * @param metaDataEnabled Whether meta data is collected.
+     */
+    public void setMetaDataEnabled(Boolean metaDataEnabled) {
+        this.metaDataEnabled = metaDataEnabled;
+    }
+
+    /**
+     * Defines whether {@link PortableObject} should cache deserialized instance. If provided,
+     * this value will override {@link PortableMarshaller#isKeepDeserialized()}
+     * property.
+     *
+     * @return Whether deserialized value is kept.
+     */
+    public Boolean isKeepDeserialized() {
+        return keepDeserialized;
+    }
+
+    /**
+     * @param keepDeserialized Whether deserialized value is kept.
+     */
+    public void setKeepDeserialized(Boolean keepDeserialized) {
+        this.keepDeserialized = keepDeserialized;
+    }
+
+    /**
+     * Gets affinity key field name.
+     *
+     * @return Affinity key field name.
+     */
+    public String getAffinityKeyFieldName() {
+        return affKeyFieldName;
+    }
+
+    /**
+     * Sets affinity key field name.
+     *
+     * @param affKeyFieldName Affinity key field name.
+     */
+    public void setAffinityKeyFieldName(String affKeyFieldName) {
+        this.affKeyFieldName = affKeyFieldName;
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return S.toString(PortableTypeConfiguration.class, this, super.toString());
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/71379a80/modules/core/src/main/java/org/apache/ignite/internal/portable/api/PortableWriter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/api/PortableWriter.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/api/PortableWriter.java
new file mode 100644
index 0000000..8af04a3
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/portable/api/PortableWriter.java
@@ -0,0 +1,266 @@
+/*
+ * 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.api;
+
+import java.math.BigDecimal;
+import java.sql.Timestamp;
+import java.util.Collection;
+import java.util.Date;
+import java.util.Map;
+import java.util.UUID;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Writer for portable object used in {@link PortableMarshalAware} implementations.
+ * Useful for the cases when user wants a fine-grained control over serialization.
+ * <p>
+ * Note that Ignite never writes full strings for field or type names. Instead,
+ * for performance reasons, Ignite writes integer hash codes for type and field names.
+ * It has been tested that hash code conflicts for the type names or the field names
+ * within the same type are virtually non-existent and, to gain performance, it is safe
+ * to work with hash codes. For the cases when hash codes for different types or fields
+ * actually do collide, Ignite provides {@link PortableIdMapper} which
+ * allows to override the automatically generated hash code IDs for the type and field names.
+ */
+public interface PortableWriter {
+    /**
+     * @param fieldName Field name.
+     * @param val Value to write.
+     * @throws PortableException In case of error.
+     */
+    public void writeByte(String fieldName, byte val) throws PortableException;
+
+    /**
+     * @param fieldName Field name.
+     * @param val Value to write.
+     * @throws PortableException In case of error.
+     */
+    public void writeShort(String fieldName, short val) throws PortableException;
+
+    /**
+     * @param fieldName Field name.
+     * @param val Value to write.
+     * @throws PortableException In case of error.
+     */
+    public void writeInt(String fieldName, int val) throws PortableException;
+
+    /**
+     * @param fieldName Field name.
+     * @param val Value to write.
+     * @throws PortableException In case of error.
+     */
+    public void writeLong(String fieldName, long val) throws PortableException;
+
+    /**
+     * @param fieldName Field name.
+     * @param val Value to write.
+     * @throws PortableException In case of error.
+     */
+    public void writeFloat(String fieldName, float val) throws PortableException;
+
+    /**
+     * @param fieldName Field name.
+     * @param val Value to write.
+     * @throws PortableException In case of error.
+     */
+    public void writeDouble(String fieldName, double val) throws PortableException;
+
+    /**
+     * @param fieldName Field name.
+     * @param val Value to write.
+     * @throws PortableException In case of error.
+     */
+    public void writeChar(String fieldName, char val) throws PortableException;
+
+    /**
+     * @param fieldName Field name.
+     * @param val Value to write.
+     * @throws PortableException In case of error.
+     */
+    public void writeBoolean(String fieldName, boolean val) throws PortableException;
+
+    /**
+     * @param fieldName Field name.
+     * @param val Value to write.
+     * @throws PortableException In case of error.
+     */
+    public void writeDecimal(String fieldName, @Nullable BigDecimal val) throws PortableException;
+
+    /**
+     * @param fieldName Field name.
+     * @param val Value to write.
+     * @throws PortableException In case of error.
+     */
+    public void writeString(String fieldName, @Nullable String val) throws PortableException;
+
+    /**
+     * @param fieldName Field name.
+     * @param val UUID to write.
+     * @throws PortableException In case of error.
+     */
+    public void writeUuid(String fieldName, @Nullable UUID val) throws PortableException;
+
+    /**
+     * @param fieldName Field name.
+     * @param val Date to write.
+     * @throws PortableException In case of error.
+     */
+    public void writeDate(String fieldName, @Nullable Date val) throws PortableException;
+
+    /**
+     * @param fieldName Field name.
+     * @param val Timestamp to write.
+     * @throws PortableException In case of error.
+     */
+    public void writeTimestamp(String fieldName, @Nullable Timestamp val) throws PortableException;
+
+    /**
+     * @param fieldName Field name.
+     * @param obj Value to write.
+     * @throws PortableException In case of error.
+     */
+    public void writeObject(String fieldName, @Nullable Object obj) throws PortableException;
+
+    /**
+     * @param fieldName Field name.
+     * @param val Value to write.
+     * @throws PortableException In case of error.
+     */
+    public void writeByteArray(String fieldName, @Nullable byte[] val) throws PortableException;
+
+    /**
+     * @param fieldName Field name.
+     * @param val Value to write.
+     * @throws PortableException In case of error.
+     */
+    public void writeShortArray(String fieldName, @Nullable short[] val) throws PortableException;
+
+    /**
+     * @param fieldName Field name.
+     * @param val Value to write.
+     * @throws PortableException In case of error.
+     */
+    public void writeIntArray(String fieldName, @Nullable int[] val) throws PortableException;
+
+    /**
+     * @param fieldName Field name.
+     * @param val Value to write.
+     * @throws PortableException In case of error.
+     */
+    public void writeLongArray(String fieldName, @Nullable long[] val) throws PortableException;
+
+    /**
+     * @param fieldName Field name.
+     * @param val Value to write.
+     * @throws PortableException In case of error.
+     */
+    public void writeFloatArray(String fieldName, @Nullable float[] val) throws PortableException;
+
+    /**
+     * @param fieldName Field name.
+     * @param val Value to write.
+     * @throws PortableException In case of error.
+     */
+    public void writeDoubleArray(String fieldName, @Nullable double[] val) throws PortableException;
+
+    /**
+     * @param fieldName Field name.
+     * @param val Value to write.
+     * @throws PortableException In case of error.
+     */
+    public void writeCharArray(String fieldName, @Nullable char[] val) throws PortableException;
+
+    /**
+     * @param fieldName Field name.
+     * @param val Value to write.
+     * @throws PortableException In case of error.
+     */
+    public void writeBooleanArray(String fieldName, @Nullable boolean[] val) throws PortableException;
+
+    /**
+     * @param fieldName Field name.
+     * @param val Value to write.
+     * @throws PortableException In case of error.
+     */
+    public void writeDecimalArray(String fieldName, @Nullable BigDecimal[] val) throws PortableException;
+
+    /**
+     * @param fieldName Field name.
+     * @param val Value to write.
+     * @throws PortableException In case of error.
+     */
+    public void writeStringArray(String fieldName, @Nullable String[] val) throws PortableException;
+
+    /**
+     * @param fieldName Field name.
+     * @param val Value to write.
+     * @throws PortableException In case of error.
+     */
+    public void writeUuidArray(String fieldName, @Nullable UUID[] val) throws PortableException;
+
+    /**
+     * @param fieldName Field name.
+     * @param val Value to write.
+     * @throws PortableException In case of error.
+     */
+    public void writeDateArray(String fieldName, @Nullable Date[] val) throws PortableException;
+
+    /**
+     * @param fieldName Field name.
+     * @param val Value to write.
+     * @throws PortableException In case of error.
+     */
+    public void writeObjectArray(String fieldName, @Nullable Object[] val) throws PortableException;
+
+    /**
+     * @param fieldName Field name.
+     * @param col Collection to write.
+     * @throws PortableException In case of error.
+     */
+    public <T> void writeCollection(String fieldName, @Nullable Collection<T> col) throws PortableException;
+
+    /**
+     * @param fieldName Field name.
+     * @param map Map to write.
+     * @throws PortableException In case of error.
+     */
+    public <K, V> void writeMap(String fieldName, @Nullable Map<K, V> map) throws PortableException;
+
+    /**
+     * @param fieldName Field name.
+     * @param val Value to write.
+     * @throws PortableException In case of error.
+     */
+    public <T extends Enum<?>> void writeEnum(String fieldName, T val) throws PortableException;
+
+    /**
+     * @param fieldName Field name.
+     * @param val Value to write.
+     * @throws PortableException In case of error.
+     */
+    public <T extends Enum<?>> void writeEnumArray(String fieldName, T[] val) throws PortableException;
+
+    /**
+     * Gets raw writer. Raw writer does not write field name hash codes, therefore,
+     * making the format even more compact. However, if the raw writer is used,
+     * dynamic structure changes to the portable objects are not supported.
+     *
+     * @return Raw writer.
+     */
+    public PortableRawWriter rawWriter();
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/71379a80/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
index 1472d56..8673b70 100644
--- 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
@@ -20,7 +20,7 @@ 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;
+import org.apache.ignite.internal.portable.api.PortableInvalidClassException;
 
 /**
  *

http://git-wip-us.apache.org/repos/asf/ignite/blob/71379a80/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
index b2e4c0d..96d10a2 100644
--- 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
@@ -25,17 +25,13 @@ 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.apache.ignite.internal.portable.api.PortableBuilder;
+import org.apache.ignite.internal.portable.api.PortableException;
+import org.apache.ignite.internal.portable.api.PortableInvalidClassException;
+import org.apache.ignite.internal.portable.api.PortableMetadata;
+import org.apache.ignite.internal.portable.api.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;

http://git-wip-us.apache.org/repos/asf/ignite/blob/71379a80/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/PortableBuilderReader.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/PortableBuilderReader.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/PortableBuilderReader.java
index 45355d7..e93f860 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/PortableBuilderReader.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/PortableBuilderReader.java
@@ -28,7 +28,7 @@ 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 org.apache.ignite.internal.portable.api.PortableException;
 
 import static java.nio.charset.StandardCharsets.UTF_8;
 import static org.apache.ignite.internal.portable.GridPortableMarshaller.NULL;

http://git-wip-us.apache.org/repos/asf/ignite/blob/71379a80/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/PortableBuilderSerializer.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/PortableBuilderSerializer.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/PortableBuilderSerializer.java
index 2d9c961..6fe8875 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/PortableBuilderSerializer.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/PortableBuilderSerializer.java
@@ -21,8 +21,8 @@ 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.portable.api.PortableObject;
 import org.apache.ignite.internal.util.*;
-import org.apache.ignite.portable.*;
 
 import java.util.*;
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/71379a80/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
index d864a6e..15c52e0 100644
--- 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
@@ -20,8 +20,8 @@ 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;
+import org.apache.ignite.internal.portable.api.PortableException;
+import org.apache.ignite.internal.portable.api.PortableInvalidClassException;
 
 /**
  *

http://git-wip-us.apache.org/repos/asf/ignite/blob/71379a80/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
index 1126a3c..96f4944 100644
--- 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
@@ -20,7 +20,7 @@ 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;
+import org.apache.ignite.internal.portable.api.PortableInvalidClassException;
 
 /**
  *

http://git-wip-us.apache.org/repos/asf/ignite/blob/71379a80/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
index 8743fbe..300c4ad 100644
--- 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
@@ -20,7 +20,7 @@ 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;
+import org.apache.ignite.internal.portable.api.PortableObject;
 
 /**
  *

http://git-wip-us.apache.org/repos/asf/ignite/blob/71379a80/modules/core/src/main/java/org/apache/ignite/internal/portable/streams/PortableAbstractInputStream.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/streams/PortableAbstractInputStream.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/streams/PortableAbstractInputStream.java
index 107b02e..80f91be 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/portable/streams/PortableAbstractInputStream.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/portable/streams/PortableAbstractInputStream.java
@@ -17,7 +17,7 @@
 
 package org.apache.ignite.internal.portable.streams;
 
-import org.apache.ignite.portable.PortableException;
+import org.apache.ignite.internal.portable.api.PortableException;
 
 /**
  * Portable abstract input stream.

http://git-wip-us.apache.org/repos/asf/ignite/blob/71379a80/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 75d4c43..59bb5f7 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
@@ -115,7 +115,7 @@ import org.apache.ignite.lang.IgniteUuid;
 import org.apache.ignite.lifecycle.LifecycleAware;
 import org.apache.ignite.marshaller.Marshaller;
 import org.apache.ignite.marshaller.jdk.JdkMarshaller;
-import org.apache.ignite.marshaller.portable.PortableMarshaller;
+import org.apache.ignite.internal.portable.api.PortableMarshaller;
 import org.apache.ignite.spi.IgniteNodeValidationResult;
 import org.jetbrains.annotations.Nullable;
 
@@ -1063,12 +1063,6 @@ public class GridCacheProcessor extends GridProcessorAdapter {
 
         CacheConfiguration cfg = cacheCtx.config();
 
-        // Intentionally compare Boolean references using '!=' below to check if the flag has been explicitly set.
-        if (cfg.isKeepPortableInStore() && cfg.isKeepPortableInStore() != CacheConfiguration.DFLT_KEEP_PORTABLE_IN_STORE
-            && !(ctx.config().getMarshaller() instanceof PortableMarshaller))
-            U.warn(log, "CacheConfiguration.isKeepPortableInStore() configuration property will be ignored because " +
-                "PortableMarshaller is not used");
-
         // Start managers.
         for (GridCacheManager mgr : F.view(cacheCtx.managers(), F.notContains(dhtExcludes(cacheCtx))))
             mgr.start(cacheCtx);

http://git-wip-us.apache.org/repos/asf/ignite/blob/71379a80/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheProxy.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheProxy.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheProxy.java
index ce0cdd7..cc6c19a 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheProxy.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheProxy.java
@@ -311,11 +311,6 @@ public class IgniteCacheProxy<K, V> extends AsyncSupportAdapter<IgniteCache<K, V
     }
 
     /** {@inheritDoc} */
-    @Override public <K1, V1> IgniteCache<K1, V1> withKeepPortable() {
-        return keepPortable();
-    }
-
-    /** {@inheritDoc} */
     @Override public IgniteCache<K, V> withNoRetries() {
         GridCacheGateway<K, V> gate = this.gate;
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/71379a80/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/portable/CacheDefaultPortableAffinityKeyMapper.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/portable/CacheDefaultPortableAffinityKeyMapper.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/portable/CacheDefaultPortableAffinityKeyMapper.java
index 23edd9e..0dbf71d 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/portable/CacheDefaultPortableAffinityKeyMapper.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/portable/CacheDefaultPortableAffinityKeyMapper.java
@@ -21,7 +21,7 @@ import org.apache.ignite.IgniteException;
 import org.apache.ignite.internal.IgniteKernal;
 import org.apache.ignite.internal.processors.cache.GridCacheDefaultAffinityKeyMapper;
 import org.apache.ignite.internal.util.typedef.internal.U;
-import org.apache.ignite.portable.PortableObject;
+import org.apache.ignite.internal.portable.api.PortableObject;
 
 /**
  *

http://git-wip-us.apache.org/repos/asf/ignite/blob/71379a80/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/portable/CacheObjectPortableContext.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/portable/CacheObjectPortableContext.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/portable/CacheObjectPortableContext.java
index 2e0d37d..d064601 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/portable/CacheObjectPortableContext.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/portable/CacheObjectPortableContext.java
@@ -27,7 +27,7 @@ import org.apache.ignite.internal.portable.PortableUtils;
 import org.apache.ignite.internal.processors.cache.CacheObjectContext;
 import org.apache.ignite.internal.processors.cache.GridCacheDefaultAffinityKeyMapper;
 import org.apache.ignite.internal.util.typedef.F;
-import org.apache.ignite.portable.PortableObject;
+import org.apache.ignite.internal.portable.api.PortableObject;
 
 /**
  *

http://git-wip-us.apache.org/repos/asf/ignite/blob/71379a80/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/portable/CacheObjectPortableProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/portable/CacheObjectPortableProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/portable/CacheObjectPortableProcessor.java
index fcd73d2..7f6512b 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/portable/CacheObjectPortableProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/portable/CacheObjectPortableProcessor.java
@@ -20,11 +20,11 @@ package org.apache.ignite.internal.processors.cache.portable;
 import java.util.Collection;
 import java.util.Map;
 import org.apache.ignite.IgniteException;
-import org.apache.ignite.IgnitePortables;
+import org.apache.ignite.internal.portable.api.IgnitePortables;
 import org.apache.ignite.internal.processors.cacheobject.IgniteCacheObjectProcessor;
-import org.apache.ignite.portable.PortableBuilder;
-import org.apache.ignite.portable.PortableMetadata;
-import org.apache.ignite.portable.PortableObject;
+import org.apache.ignite.internal.portable.api.PortableBuilder;
+import org.apache.ignite.internal.portable.api.PortableMetadata;
+import org.apache.ignite.internal.portable.api.PortableObject;
 import org.jetbrains.annotations.Nullable;
 
 /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/71379a80/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 1be5aea..4cab3db 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
@@ -39,7 +39,7 @@ import javax.cache.processor.EntryProcessor;
 import javax.cache.processor.MutableEntry;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.IgniteException;
-import org.apache.ignite.IgnitePortables;
+import org.apache.ignite.internal.portable.api.IgnitePortables;
 import org.apache.ignite.cache.CacheEntryEventSerializableFilter;
 import org.apache.ignite.cluster.ClusterNode;
 import org.apache.ignite.configuration.CacheConfiguration;
@@ -80,11 +80,11 @@ import org.apache.ignite.internal.util.typedef.internal.S;
 import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.lang.IgniteBiPredicate;
 import org.apache.ignite.marshaller.Marshaller;
-import org.apache.ignite.marshaller.portable.PortableMarshaller;
-import org.apache.ignite.portable.PortableBuilder;
-import org.apache.ignite.portable.PortableException;
-import org.apache.ignite.portable.PortableMetadata;
-import org.apache.ignite.portable.PortableObject;
+import org.apache.ignite.internal.portable.api.PortableMarshaller;
+import org.apache.ignite.internal.portable.api.PortableBuilder;
+import org.apache.ignite.internal.portable.api.PortableException;
+import org.apache.ignite.internal.portable.api.PortableMetadata;
+import org.apache.ignite.internal.portable.api.PortableObject;
 import org.jetbrains.annotations.Nullable;
 import org.jsr166.ConcurrentHashMap8;
 import sun.misc.Unsafe;

http://git-wip-us.apache.org/repos/asf/ignite/blob/71379a80/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/portable/IgnitePortablesImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/portable/IgnitePortablesImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/portable/IgnitePortablesImpl.java
index 5ed6505..40c3b70 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/portable/IgnitePortablesImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/portable/IgnitePortablesImpl.java
@@ -18,13 +18,13 @@
 package org.apache.ignite.internal.processors.cache.portable;
 
 import java.util.Collection;
-import org.apache.ignite.IgnitePortables;
+import org.apache.ignite.internal.portable.api.IgnitePortables;
 import org.apache.ignite.internal.GridKernalContext;
 import org.apache.ignite.internal.processors.cacheobject.IgniteCacheObjectProcessor;
-import org.apache.ignite.portable.PortableBuilder;
-import org.apache.ignite.portable.PortableException;
-import org.apache.ignite.portable.PortableMetadata;
-import org.apache.ignite.portable.PortableObject;
+import org.apache.ignite.internal.portable.api.PortableBuilder;
+import org.apache.ignite.internal.portable.api.PortableException;
+import org.apache.ignite.internal.portable.api.PortableMetadata;
+import org.apache.ignite.internal.portable.api.PortableObject;
 import org.jetbrains.annotations.Nullable;
 
 /**


[12/55] [abbrv] ignite git commit: Minor docs improvement.

Posted by ag...@apache.org.
Minor docs improvement.


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

Branch: refs/heads/ignite-1171
Commit: c01313d4609bb9a4209e2026b24b09df12935d75
Parents: a0cd9af
Author: vozerov-gridgain <vo...@gridgain.com>
Authored: Mon Sep 14 17:33:37 2015 +0300
Committer: vozerov-gridgain <vo...@gridgain.com>
Committed: Mon Sep 14 17:33:37 2015 +0300

----------------------------------------------------------------------
 .../processors/platform/services/PlatformAbstractService.java   | 5 ++---
 1 file changed, 2 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/c01313d4/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/services/PlatformAbstractService.java
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/services/PlatformAbstractService.java b/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/services/PlatformAbstractService.java
index dd5c28a..0b9ee53 100644
--- a/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/services/PlatformAbstractService.java
+++ b/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/services/PlatformAbstractService.java
@@ -212,9 +212,8 @@ public abstract class PlatformAbstractService implements PlatformService, Extern
     @SuppressWarnings("UnusedDeclaration")
     @IgniteInstanceResource
     public void setIgniteInstance(Ignite ignite) {
-        platformCtx = ignite != null
-            ? PlatformUtils.platformContext(ignite)
-            : null;
+        // Ignite instance can be null here because service processor invokes "cleanup" on resource manager.
+        platformCtx = ignite != null ? PlatformUtils.platformContext(ignite) : null;
     }
 
     /** {@inheritDoc} */


[07/55] [abbrv] ignite git commit: ignite-1462: hid portable API in 1.4 release

Posted by ag...@apache.org.
http://git-wip-us.apache.org/repos/asf/ignite/blob/71379a80/modules/core/src/main/java/org/apache/ignite/portable/PortableWriter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/portable/PortableWriter.java b/modules/core/src/main/java/org/apache/ignite/portable/PortableWriter.java
deleted file mode 100644
index 0d7160f..0000000
--- a/modules/core/src/main/java/org/apache/ignite/portable/PortableWriter.java
+++ /dev/null
@@ -1,266 +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.portable;
-
-import java.math.BigDecimal;
-import java.sql.Timestamp;
-import java.util.Collection;
-import java.util.Date;
-import java.util.Map;
-import java.util.UUID;
-import org.jetbrains.annotations.Nullable;
-
-/**
- * Writer for portable object used in {@link PortableMarshalAware} implementations.
- * Useful for the cases when user wants a fine-grained control over serialization.
- * <p>
- * Note that Ignite never writes full strings for field or type names. Instead,
- * for performance reasons, Ignite writes integer hash codes for type and field names.
- * It has been tested that hash code conflicts for the type names or the field names
- * within the same type are virtually non-existent and, to gain performance, it is safe
- * to work with hash codes. For the cases when hash codes for different types or fields
- * actually do collide, Ignite provides {@link PortableIdMapper} which
- * allows to override the automatically generated hash code IDs for the type and field names.
- */
-public interface PortableWriter {
-    /**
-     * @param fieldName Field name.
-     * @param val Value to write.
-     * @throws PortableException In case of error.
-     */
-    public void writeByte(String fieldName, byte val) throws PortableException;
-
-    /**
-     * @param fieldName Field name.
-     * @param val Value to write.
-     * @throws PortableException In case of error.
-     */
-    public void writeShort(String fieldName, short val) throws PortableException;
-
-    /**
-     * @param fieldName Field name.
-     * @param val Value to write.
-     * @throws PortableException In case of error.
-     */
-    public void writeInt(String fieldName, int val) throws PortableException;
-
-    /**
-     * @param fieldName Field name.
-     * @param val Value to write.
-     * @throws PortableException In case of error.
-     */
-    public void writeLong(String fieldName, long val) throws PortableException;
-
-    /**
-     * @param fieldName Field name.
-     * @param val Value to write.
-     * @throws PortableException In case of error.
-     */
-    public void writeFloat(String fieldName, float val) throws PortableException;
-
-    /**
-     * @param fieldName Field name.
-     * @param val Value to write.
-     * @throws PortableException In case of error.
-     */
-    public void writeDouble(String fieldName, double val) throws PortableException;
-
-    /**
-     * @param fieldName Field name.
-     * @param val Value to write.
-     * @throws PortableException In case of error.
-     */
-    public void writeChar(String fieldName, char val) throws PortableException;
-
-    /**
-     * @param fieldName Field name.
-     * @param val Value to write.
-     * @throws PortableException In case of error.
-     */
-    public void writeBoolean(String fieldName, boolean val) throws PortableException;
-
-    /**
-     * @param fieldName Field name.
-     * @param val Value to write.
-     * @throws PortableException In case of error.
-     */
-    public void writeDecimal(String fieldName, @Nullable BigDecimal val) throws PortableException;
-
-    /**
-     * @param fieldName Field name.
-     * @param val Value to write.
-     * @throws PortableException In case of error.
-     */
-    public void writeString(String fieldName, @Nullable String val) throws PortableException;
-
-    /**
-     * @param fieldName Field name.
-     * @param val UUID to write.
-     * @throws PortableException In case of error.
-     */
-    public void writeUuid(String fieldName, @Nullable UUID val) throws PortableException;
-
-    /**
-     * @param fieldName Field name.
-     * @param val Date to write.
-     * @throws PortableException In case of error.
-     */
-    public void writeDate(String fieldName, @Nullable Date val) throws PortableException;
-
-    /**
-     * @param fieldName Field name.
-     * @param val Timestamp to write.
-     * @throws PortableException In case of error.
-     */
-    public void writeTimestamp(String fieldName, @Nullable Timestamp val) throws PortableException;
-
-    /**
-     * @param fieldName Field name.
-     * @param obj Value to write.
-     * @throws PortableException In case of error.
-     */
-    public void writeObject(String fieldName, @Nullable Object obj) throws PortableException;
-
-    /**
-     * @param fieldName Field name.
-     * @param val Value to write.
-     * @throws PortableException In case of error.
-     */
-    public void writeByteArray(String fieldName, @Nullable byte[] val) throws PortableException;
-
-    /**
-     * @param fieldName Field name.
-     * @param val Value to write.
-     * @throws PortableException In case of error.
-     */
-    public void writeShortArray(String fieldName, @Nullable short[] val) throws PortableException;
-
-    /**
-     * @param fieldName Field name.
-     * @param val Value to write.
-     * @throws PortableException In case of error.
-     */
-    public void writeIntArray(String fieldName, @Nullable int[] val) throws PortableException;
-
-    /**
-     * @param fieldName Field name.
-     * @param val Value to write.
-     * @throws PortableException In case of error.
-     */
-    public void writeLongArray(String fieldName, @Nullable long[] val) throws PortableException;
-
-    /**
-     * @param fieldName Field name.
-     * @param val Value to write.
-     * @throws PortableException In case of error.
-     */
-    public void writeFloatArray(String fieldName, @Nullable float[] val) throws PortableException;
-
-    /**
-     * @param fieldName Field name.
-     * @param val Value to write.
-     * @throws PortableException In case of error.
-     */
-    public void writeDoubleArray(String fieldName, @Nullable double[] val) throws PortableException;
-
-    /**
-     * @param fieldName Field name.
-     * @param val Value to write.
-     * @throws PortableException In case of error.
-     */
-    public void writeCharArray(String fieldName, @Nullable char[] val) throws PortableException;
-
-    /**
-     * @param fieldName Field name.
-     * @param val Value to write.
-     * @throws PortableException In case of error.
-     */
-    public void writeBooleanArray(String fieldName, @Nullable boolean[] val) throws PortableException;
-
-    /**
-     * @param fieldName Field name.
-     * @param val Value to write.
-     * @throws PortableException In case of error.
-     */
-    public void writeDecimalArray(String fieldName, @Nullable BigDecimal[] val) throws PortableException;
-
-    /**
-     * @param fieldName Field name.
-     * @param val Value to write.
-     * @throws PortableException In case of error.
-     */
-    public void writeStringArray(String fieldName, @Nullable String[] val) throws PortableException;
-
-    /**
-     * @param fieldName Field name.
-     * @param val Value to write.
-     * @throws PortableException In case of error.
-     */
-    public void writeUuidArray(String fieldName, @Nullable UUID[] val) throws PortableException;
-
-    /**
-     * @param fieldName Field name.
-     * @param val Value to write.
-     * @throws PortableException In case of error.
-     */
-    public void writeDateArray(String fieldName, @Nullable Date[] val) throws PortableException;
-
-    /**
-     * @param fieldName Field name.
-     * @param val Value to write.
-     * @throws PortableException In case of error.
-     */
-    public void writeObjectArray(String fieldName, @Nullable Object[] val) throws PortableException;
-
-    /**
-     * @param fieldName Field name.
-     * @param col Collection to write.
-     * @throws PortableException In case of error.
-     */
-    public <T> void writeCollection(String fieldName, @Nullable Collection<T> col) throws PortableException;
-
-    /**
-     * @param fieldName Field name.
-     * @param map Map to write.
-     * @throws PortableException In case of error.
-     */
-    public <K, V> void writeMap(String fieldName, @Nullable Map<K, V> map) throws PortableException;
-
-    /**
-     * @param fieldName Field name.
-     * @param val Value to write.
-     * @throws PortableException In case of error.
-     */
-    public <T extends Enum<?>> void writeEnum(String fieldName, T val) throws PortableException;
-
-    /**
-     * @param fieldName Field name.
-     * @param val Value to write.
-     * @throws PortableException In case of error.
-     */
-    public <T extends Enum<?>> void writeEnumArray(String fieldName, T[] val) throws PortableException;
-
-    /**
-     * Gets raw writer. Raw writer does not write field name hash codes, therefore,
-     * making the format even more compact. However, if the raw writer is used,
-     * dynamic structure changes to the portable objects are not supported.
-     *
-     * @return Raw writer.
-     */
-    public PortableRawWriter rawWriter();
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/71379a80/modules/core/src/main/java/org/apache/ignite/portable/package-info.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/portable/package-info.java b/modules/core/src/main/java/org/apache/ignite/portable/package-info.java
deleted file mode 100644
index 0105b15..0000000
--- a/modules/core/src/main/java/org/apache/ignite/portable/package-info.java
+++ /dev/null
@@ -1,22 +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 description. -->
- * Contains portable objects API classes.
- */
-package org.apache.ignite.portable;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/71379a80/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 70c32e5..36ac156 100644
--- a/modules/core/src/main/resources/META-INF/classnames.properties
+++ b/modules/core/src/main/resources/META-INF/classnames.properties
@@ -1572,10 +1572,10 @@ org.apache.ignite.plugin.security.SecuritySubject
 org.apache.ignite.plugin.security.SecuritySubjectType
 org.apache.ignite.plugin.segmentation.SegmentationPolicy
 org.apache.ignite.plugin.segmentation.SegmentationResolver
-org.apache.ignite.portable.PortableException
-org.apache.ignite.portable.PortableInvalidClassException
-org.apache.ignite.portable.PortableObject
-org.apache.ignite.portable.PortableProtocolVersion
+org.apache.ignite.internal.portable.api.PortableException
+org.apache.ignite.internal.portable.api.PortableInvalidClassException
+org.apache.ignite.internal.portable.api.PortableObject
+org.apache.ignite.internal.portable.api.PortableProtocolVersion
 org.apache.ignite.services.Service
 org.apache.ignite.services.ServiceConfiguration
 org.apache.ignite.services.ServiceContext

http://git-wip-us.apache.org/repos/asf/ignite/blob/71379a80/modules/core/src/test/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManagerAttributesSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManagerAttributesSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManagerAttributesSelfTest.java
index 82da10f..dc73cff 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManagerAttributesSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManagerAttributesSelfTest.java
@@ -159,51 +159,6 @@ public abstract class GridDiscoveryManagerAttributesSelfTest extends GridCommonA
     }
 
     /**
-     * @throws Exception If failed.
-     */
-    public void testDifferentPortableProtocolVersions() throws Exception {
-        startGridWithPortableProtocolVer("VER_99_99_99");
-
-        try {
-            startGrid(1);
-
-            fail();
-        }
-        catch (IgniteCheckedException e) {
-            if (!e.getCause().getMessage().startsWith("Remote node has portable protocol version different from local"))
-                throw e;
-        }
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testNullPortableProtocolVersion() throws Exception {
-        startGridWithPortableProtocolVer(null);
-
-        // Must not fail in order to preserve backward compatibility with the nodes that don't have this property yet.
-        startGrid(1);
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    private void startGridWithPortableProtocolVer(String ver) throws Exception {
-        Ignite ignite = startGrid(0);
-
-        ClusterNode clusterNode = ignite.cluster().localNode();
-
-        Field f = clusterNode.getClass().getDeclaredField("attrs");
-        f.setAccessible(true);
-
-        Map<String, Object> attrs = new HashMap<>((Map<String, Object>)f.get(clusterNode));
-
-        attrs.put(IgniteNodeAttributes.ATTR_PORTABLE_PROTO_VER, ver);
-
-        f.set(clusterNode, attrs);
-    }
-
-    /**
      * @param preferIpV4 {@code java.net.preferIPv4Stack} system property value.
      * @throws Exception If failed.
      */

http://git-wip-us.apache.org/repos/asf/ignite/blob/71379a80/modules/core/src/test/java/org/apache/ignite/internal/portable/GridPortableAffinityKeySelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/portable/GridPortableAffinityKeySelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/portable/GridPortableAffinityKeySelfTest.java
deleted file mode 100644
index 59084db..0000000
--- a/modules/core/src/test/java/org/apache/ignite/internal/portable/GridPortableAffinityKeySelfTest.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.Collections;
-import java.util.UUID;
-import java.util.concurrent.atomic.AtomicReference;
-import org.apache.ignite.Ignite;
-import org.apache.ignite.cache.affinity.Affinity;
-import org.apache.ignite.configuration.CacheConfiguration;
-import org.apache.ignite.configuration.IgniteConfiguration;
-import org.apache.ignite.internal.IgniteKernal;
-import org.apache.ignite.internal.processors.affinity.GridAffinityProcessor;
-import org.apache.ignite.internal.processors.cache.CacheObject;
-import org.apache.ignite.internal.processors.cache.CacheObjectContext;
-import org.apache.ignite.internal.processors.cacheobject.IgniteCacheObjectProcessor;
-import org.apache.ignite.lang.IgniteCallable;
-import org.apache.ignite.lang.IgniteRunnable;
-import org.apache.ignite.marshaller.portable.PortableMarshaller;
-import org.apache.ignite.portable.PortableTypeConfiguration;
-import org.apache.ignite.resources.IgniteInstanceResource;
-import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi;
-import org.apache.ignite.spi.discovery.tcp.ipfinder.TcpDiscoveryIpFinder;
-import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder;
-import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
-
-import static org.apache.ignite.cache.CacheMode.PARTITIONED;
-
-/**
- * Test for portable object affinity key.
- */
-public class GridPortableAffinityKeySelfTest extends GridCommonAbstractTest {
-    /** */
-    private static final AtomicReference<UUID> nodeId = new AtomicReference<>();
-
-    /** VM ip finder for TCP discovery. */
-    private static TcpDiscoveryIpFinder ipFinder = new TcpDiscoveryVmIpFinder(true);
-
-    /** */
-    private static int GRID_CNT = 5;
-
-    /** {@inheritDoc} */
-    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
-        IgniteConfiguration cfg = super.getConfiguration(gridName);
-
-        PortableTypeConfiguration typeCfg = new PortableTypeConfiguration();
-
-        typeCfg.setClassName(TestObject.class.getName());
-        typeCfg.setAffinityKeyFieldName("affKey");
-
-        PortableMarshaller marsh = new PortableMarshaller();
-
-        marsh.setTypeConfigurations(Collections.singleton(typeCfg));
-
-        cfg.setMarshaller(marsh);
-
-        if (!gridName.equals(getTestGridName(GRID_CNT))) {
-            CacheConfiguration cacheCfg = new CacheConfiguration();
-
-            cacheCfg.setCacheMode(PARTITIONED);
-
-            cfg.setCacheConfiguration(cacheCfg);
-        }
-
-        ((TcpDiscoverySpi)cfg.getDiscoverySpi()).setIpFinder(ipFinder);
-
-        return cfg;
-    }
-
-    /** {@inheritDoc} */
-    @Override protected void beforeTestsStarted() throws Exception {
-        startGridsMultiThreaded(GRID_CNT);
-    }
-
-    /** {@inheritDoc} */
-    @Override protected void afterTestsStopped() throws Exception {
-        stopAllGrids();
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testAffinity() throws Exception {
-        checkAffinity(grid(0));
-
-        try (Ignite igniteNoCache = startGrid(GRID_CNT)) {
-            try {
-                igniteNoCache.cache(null);
-            }
-            catch (IllegalArgumentException ignore) {
-                // Expected error.
-            }
-
-            checkAffinity(igniteNoCache);
-        }
-    }
-
-    /**
-     * @param ignite Ignite.
-     * @throws Exception If failed.
-     */
-    private void checkAffinity(Ignite ignite) throws Exception {
-        Affinity<Object> aff = ignite.affinity(null);
-
-        GridAffinityProcessor affProc = ((IgniteKernal)ignite).context().affinity();
-
-        IgniteCacheObjectProcessor cacheObjProc = ((IgniteKernal)ignite).context().cacheObjects();
-
-        CacheObjectContext cacheObjCtx = cacheObjProc.contextForCache(
-            ignite.cache(null).getConfiguration(CacheConfiguration.class));
-
-        for (int i = 0; i < 1000; i++) {
-            assertEquals(i, aff.affinityKey(i));
-
-            assertEquals(i, aff.affinityKey(new TestObject(i)));
-
-            CacheObject cacheObj = cacheObjProc.toCacheObject(cacheObjCtx, new TestObject(i), true);
-
-            assertEquals(i, aff.affinityKey(cacheObj));
-
-            assertEquals(aff.mapKeyToNode(i), aff.mapKeyToNode(new TestObject(i)));
-
-            assertEquals(aff.mapKeyToNode(i), aff.mapKeyToNode(cacheObj));
-
-            assertEquals(i, affProc.affinityKey(null, i));
-
-            assertEquals(i, affProc.affinityKey(null, new TestObject(i)));
-
-            assertEquals(i, affProc.affinityKey(null, cacheObj));
-
-            assertEquals(affProc.mapKeyToNode(null, i), affProc.mapKeyToNode(null, new TestObject(i)));
-
-            assertEquals(affProc.mapKeyToNode(null, i), affProc.mapKeyToNode(null, cacheObj));
-        }
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testAffinityRun() throws Exception {
-        Affinity<Object> aff = grid(0).affinity(null);
-
-        for (int i = 0; i < 1000; i++) {
-            nodeId.set(null);
-
-            grid(0).compute().affinityRun(null, new TestObject(i), new IgniteRunnable() {
-                @IgniteInstanceResource
-                private Ignite ignite;
-
-                @Override public void run() {
-                    nodeId.set(ignite.configuration().getNodeId());
-                }
-            });
-
-            assertEquals(aff.mapKeyToNode(i).id(), nodeId.get());
-        }
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testAffinityCall() throws Exception {
-        Affinity<Object> aff = grid(0).affinity(null);
-
-        for (int i = 0; i < 1000; i++) {
-            nodeId.set(null);
-
-            grid(0).compute().affinityCall(null, new TestObject(i), new IgniteCallable<Object>() {
-                @IgniteInstanceResource
-                private Ignite ignite;
-
-                @Override public Object call() {
-                    nodeId.set(ignite.configuration().getNodeId());
-
-                    return null;
-                }
-            });
-
-            assertEquals(aff.mapKeyToNode(i).id(), nodeId.get());
-        }
-    }
-
-    /**
-     */
-    private static class TestObject {
-        /** */
-        @SuppressWarnings("UnusedDeclaration")
-        private int affKey;
-
-        /**
-         */
-        private TestObject() {
-            // No-op.
-        }
-
-        /**
-         * @param affKey Affinity key.
-         */
-        private TestObject(int affKey) {
-            this.affKey = affKey;
-        }
-    }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/71379a80/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
deleted file mode 100644
index 61ec714..0000000
--- a/modules/core/src/test/java/org/apache/ignite/internal/portable/GridPortableBuilderAdditionalSelfTest.java
+++ /dev/null
@@ -1,1226 +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 com.google.common.collect.ImmutableMap;
-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;
-import java.util.Collections;
-import java.util.Date;
-import java.util.HashSet;
-import java.util.LinkedList;
-import java.util.List;
-import java.util.Map;
-import java.util.Objects;
-import java.util.Set;
-import java.util.UUID;
-import org.apache.ignite.IgniteCheckedException;
-import org.apache.ignite.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;
-import org.apache.ignite.internal.util.lang.GridMapEntry;
-import org.apache.ignite.marshaller.portable.PortableMarshaller;
-import org.apache.ignite.portable.PortableBuilder;
-import org.apache.ignite.portable.PortableMetadata;
-import org.apache.ignite.portable.PortableObject;
-import org.apache.ignite.testframework.GridTestUtils;
-import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
-import org.junit.Assert;
-
-import static org.apache.ignite.cache.CacheMode.REPLICATED;
-import static org.apache.ignite.internal.portable.mutabletest.GridPortableTestClasses.Address;
-import static org.apache.ignite.internal.portable.mutabletest.GridPortableTestClasses.AddressBook;
-import static org.apache.ignite.internal.portable.mutabletest.GridPortableTestClasses.Company;
-import static org.apache.ignite.internal.portable.mutabletest.GridPortableTestClasses.TestObjectAllTypes;
-import static org.apache.ignite.internal.portable.mutabletest.GridPortableTestClasses.TestObjectArrayList;
-import static org.apache.ignite.internal.portable.mutabletest.GridPortableTestClasses.TestObjectContainer;
-import static org.apache.ignite.internal.portable.mutabletest.GridPortableTestClasses.TestObjectEnum;
-import static org.apache.ignite.internal.portable.mutabletest.GridPortableTestClasses.TestObjectInner;
-import static org.apache.ignite.internal.portable.mutabletest.GridPortableTestClasses.TestObjectOuter;
-
-/**
- *
- */
-public class GridPortableBuilderAdditionalSelfTest extends GridCommonAbstractTest {
-    /** {@inheritDoc} */
-    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
-        IgniteConfiguration cfg = super.getConfiguration(gridName);
-
-        CacheConfiguration cacheCfg = new CacheConfiguration();
-
-        cacheCfg.setCacheMode(REPLICATED);
-
-        cfg.setCacheConfiguration(cacheCfg);
-
-        PortableMarshaller marsh = new PortableMarshaller();
-
-        marsh.setClassNames(Arrays.asList("org.apache.ignite.internal.portable.mutabletest.*"));
-
-        marsh.setConvertStringToBytes(useUtf8());
-
-        cfg.setMarshaller(marsh);
-
-        return cfg;
-    }
-
-    /** {@inheritDoc} */
-    @Override protected void beforeTestsStarted() throws Exception {
-        startGrids(1);
-    }
-
-    /** {@inheritDoc} */
-    @Override protected void afterTestsStopped() throws Exception {
-        stopAllGrids();
-    }
-
-    /** {@inheritDoc} */
-    @Override protected void afterTest() throws Exception {
-        jcache(0).clear();
-    }
-
-    /**
-     * @return Whether to use UTF8 strings.
-     */
-    protected boolean useUtf8() {
-        return true;
-    }
-
-    /**
-     * @return Portables API.
-     */
-    protected IgnitePortables portables() {
-        return grid(0).portables();
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testSimpleTypeFieldRead() throws Exception {
-        TestObjectAllTypes exp = new TestObjectAllTypes();
-
-        exp.setDefaultData();
-
-        PortableBuilder mutPo = wrap(exp);
-
-        for (Field field : TestObjectAllTypes.class.getDeclaredFields()) {
-            Object expVal = field.get(exp);
-            Object actVal = mutPo.getField(field.getName());
-
-            switch (field.getName()) {
-                case "anEnum":
-                    assertEquals(((PortableBuilderEnum)actVal).getOrdinal(), ((Enum)expVal).ordinal());
-                    break;
-
-                case "enumArr": {
-                    PortableBuilderEnum[] actArr = (PortableBuilderEnum[])actVal;
-                    Enum[] expArr = (Enum[])expVal;
-
-                    assertEquals(expArr.length, actArr.length);
-
-                    for (int i = 0; i < actArr.length; i++)
-                        assertEquals(expArr[i].ordinal(), actArr[i].getOrdinal());
-
-                    break;
-                }
-
-                case "entry":
-                    assertEquals(((Map.Entry)expVal).getKey(), ((Map.Entry)actVal).getKey());
-                    assertEquals(((Map.Entry)expVal).getValue(), ((Map.Entry)actVal).getValue());
-                    break;
-
-                default:
-                    assertTrue(field.getName(), Objects.deepEquals(expVal, actVal));
-                    break;
-            }
-        }
-    }
-
-    /**
-     *
-     */
-    public void testSimpleTypeFieldSerialize() {
-        TestObjectAllTypes exp = new TestObjectAllTypes();
-
-        exp.setDefaultData();
-
-        PortableBuilderImpl mutPo = wrap(exp);
-
-        TestObjectAllTypes res = mutPo.build().deserialize();
-
-        GridTestUtils.deepEquals(exp, res);
-    }
-
-    /**
-     * @throws Exception If any error occurs.
-     */
-    public void testSimpleTypeFieldOverride() throws Exception {
-        TestObjectAllTypes exp = new TestObjectAllTypes();
-
-        exp.setDefaultData();
-
-        PortableBuilderImpl mutPo = wrap(new TestObjectAllTypes());
-
-        for (Field field : TestObjectAllTypes.class.getDeclaredFields())
-            mutPo.setField(field.getName(), field.get(exp));
-
-        TestObjectAllTypes res = mutPo.build().deserialize();
-
-        GridTestUtils.deepEquals(exp, res);
-    }
-
-    /**
-     * @throws Exception If any error occurs.
-     */
-    public void testSimpleTypeFieldSetNull() throws Exception {
-        TestObjectAllTypes exp = new TestObjectAllTypes();
-
-        exp.setDefaultData();
-
-        PortableBuilderImpl mutPo = wrap(exp);
-
-        for (Field field : TestObjectAllTypes.class.getDeclaredFields()) {
-            if (!field.getType().isPrimitive())
-                mutPo.setField(field.getName(), null);
-        }
-
-        TestObjectAllTypes res = mutPo.build().deserialize();
-
-        for (Field field : TestObjectAllTypes.class.getDeclaredFields()) {
-            if (!field.getType().isPrimitive())
-                assertNull(field.getName(), field.get(res));
-        }
-    }
-
-    /**
-     * @throws IgniteCheckedException If any error occurs.
-     */
-    public void testMakeCyclicDependency() throws IgniteCheckedException {
-        TestObjectOuter outer = new TestObjectOuter();
-        outer.inner = new TestObjectInner();
-
-        PortableBuilderImpl mutOuter = wrap(outer);
-
-        PortableBuilderImpl mutInner = mutOuter.getField("inner");
-
-        mutInner.setField("outer", mutOuter);
-        mutInner.setField("foo", mutInner);
-
-        TestObjectOuter res = mutOuter.build().deserialize();
-
-        assertEquals(res, res.inner.outer);
-        assertEquals(res.inner, res.inner.foo);
-    }
-
-    /**
-     *
-     */
-    public void testDateArrayModification() {
-        TestObjectAllTypes obj = new TestObjectAllTypes();
-
-        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);
-
-        String[] arr = mutObj.getField("strArr");
-        arr[0] = "b";
-
-        TestObjectAllTypes res = mutObj.build().deserialize();
-
-        Assert.assertArrayEquals(new String[] {"b", "a", "a"}, res.strArr);
-    }
-
-    /**
-     *
-     */
-    public void testModifyObjectArray() {
-        TestObjectContainer obj = new TestObjectContainer();
-        obj.foo = new Object[] {"a"};
-
-        PortableBuilderImpl mutObj = wrap(obj);
-
-        Object[] arr = mutObj.getField("foo");
-
-        Assert.assertArrayEquals(new Object[] {"a"}, arr);
-
-        arr[0] = "b";
-
-        TestObjectContainer res = mutObj.build().deserialize();
-
-        Assert.assertArrayEquals(new Object[] {"b"}, (Object[])res.foo);
-    }
-
-    /**
-     *
-     */
-    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)};
-
-        mutObj.setField("foo", createdArr.clone());
-
-        TestObjectContainer res = mutObj.build().deserialize();
-
-        createdArr[0] = res;
-
-        assertTrue(Objects.deepEquals(createdArr, res.foo));
-    }
-
-    /**
-     *
-     */
-    public void testDeepArray() {
-        TestObjectContainer obj = new TestObjectContainer();
-        obj.foo = new Object[] {new Object[] {"a", obj}};
-
-        PortableBuilderImpl mutObj = wrap(obj);
-
-        Object[] arr = (Object[])mutObj.<Object[]>getField("foo")[0];
-
-        assertEquals("a", arr[0]);
-        assertSame(mutObj, arr[1]);
-
-        arr[0] = mutObj;
-
-        TestObjectContainer res = mutObj.build().deserialize();
-
-        arr = (Object[])((Object[])res.foo)[0];
-
-        assertSame(arr[0], res);
-        assertSame(arr[0], arr[1]);
-    }
-
-    /**
-     *
-     */
-    public void testArrayListRead() {
-        TestObjectContainer obj = new TestObjectContainer();
-        obj.foo = Lists.newArrayList(obj, "a");
-
-        PortableBuilderImpl mutObj = wrap(obj);
-
-        List<Object> list = mutObj.getField("foo");
-
-        assert list.equals(Lists.newArrayList(mutObj, "a"));
-    }
-
-    /**
-     *
-     */
-    public void testArrayListOverride() {
-        TestObjectContainer obj = new TestObjectContainer();
-
-        PortableBuilderImpl mutObj = wrap(obj);
-
-        ArrayList<Object> list = Lists.newArrayList(mutObj, "a", Lists.newArrayList(1, 2));
-
-        mutObj.setField("foo", list);
-
-        TestObjectContainer res = mutObj.build().deserialize();
-
-        list.set(0, res);
-
-        assertNotSame(list, res.foo);
-        assertEquals(list, res.foo);
-    }
-
-    /**
-     *
-     */
-    public void testArrayListModification() {
-        TestObjectContainer obj = new TestObjectContainer();
-        obj.foo = Lists.newArrayList("a", "b", "c");
-
-        PortableBuilderImpl mutObj = wrap(obj);
-
-        List<String> list = mutObj.getField("foo");
-
-        list.add("!"); // "a", "b", "c", "!"
-        list.add(0, "_"); // "_", "a", "b", "c", "!"
-
-        String s = list.remove(1); // "_", "b", "c", "!"
-        assertEquals("a", s);
-
-        assertEquals(Arrays.asList("c", "!"), list.subList(2, 4));
-        assertEquals(1, list.indexOf("b"));
-        assertEquals(1, list.lastIndexOf("b"));
-
-        TestObjectContainer res = mutObj.build().deserialize();
-
-        assertTrue(res.foo instanceof ArrayList);
-        assertEquals(Arrays.asList("_", "b", "c", "!"), res.foo);
-    }
-
-    /**
-     *
-     */
-    public void testArrayListClear() {
-        TestObjectContainer obj = new TestObjectContainer();
-        obj.foo = Lists.newArrayList("a", "b", "c");
-
-        PortableBuilderImpl mutObj = wrap(obj);
-
-        List<String> list = mutObj.getField("foo");
-
-        list.clear();
-
-        assertEquals(Collections.emptyList(), mutObj.build().<TestObjectContainer>deserialize().foo);
-    }
-
-    /**
-     *
-     */
-    public void testArrayListWriteUnmodifiable() {
-        TestObjectContainer obj = new TestObjectContainer();
-
-        ArrayList<Object> src = Lists.newArrayList(obj, "a", "b", "c");
-
-        obj.foo = src;
-
-        PortableBuilderImpl mutObj = wrap(obj);
-
-        TestObjectContainer deserialized = mutObj.build().deserialize();
-
-        List<Object> res = (List<Object>)deserialized.foo;
-
-        src.set(0, deserialized);
-
-        assertEquals(src, res);
-    }
-
-    /**
-     *
-     */
-    public void testLinkedListRead() {
-        TestObjectContainer obj = new TestObjectContainer();
-        obj.foo = Lists.newLinkedList(Arrays.asList(obj, "a"));
-
-        PortableBuilderImpl mutObj = wrap(obj);
-
-        List<Object> list = mutObj.getField("foo");
-
-        assert list.equals(Lists.newLinkedList(Arrays.asList(mutObj, "a")));
-    }
-
-    /**
-     *
-     */
-    public void testLinkedListOverride() {
-        TestObjectContainer obj = new TestObjectContainer();
-
-        PortableBuilderImpl mutObj = wrap(obj);
-
-        List<Object> list = Lists.newLinkedList(Arrays.asList(mutObj, "a", Lists.newLinkedList(Arrays.asList(1, 2))));
-
-        mutObj.setField("foo", list);
-
-        TestObjectContainer res = mutObj.build().deserialize();
-
-        list.set(0, res);
-
-        assertNotSame(list, res.foo);
-        assertEquals(list, res.foo);
-    }
-
-    /**
-     *
-     */
-    public void testLinkedListModification() {
-        TestObjectContainer obj = new TestObjectContainer();
-
-        obj.foo = Lists.newLinkedList(Arrays.asList("a", "b", "c"));
-
-        PortableBuilderImpl mutObj = wrap(obj);
-
-        List<String> list = mutObj.getField("foo");
-
-        list.add("!"); // "a", "b", "c", "!"
-        list.add(0, "_"); // "_", "a", "b", "c", "!"
-
-        String s = list.remove(1); // "_", "b", "c", "!"
-        assertEquals("a", s);
-
-        assertEquals(Arrays.asList("c", "!"), list.subList(2, 4));
-        assertEquals(1, list.indexOf("b"));
-        assertEquals(1, list.lastIndexOf("b"));
-
-        TestObjectContainer res = mutObj.build().deserialize();
-
-        assertTrue(res.foo instanceof LinkedList);
-        assertEquals(Arrays.asList("_", "b", "c", "!"), res.foo);
-    }
-
-    /**
-     *
-     */
-    public void testLinkedListWriteUnmodifiable() {
-        TestObjectContainer obj = new TestObjectContainer();
-
-        LinkedList<Object> src = Lists.newLinkedList(Arrays.asList(obj, "a", "b", "c"));
-
-        obj.foo = src;
-
-        PortableBuilderImpl mutObj = wrap(obj);
-
-        TestObjectContainer deserialized = mutObj.build().deserialize();
-
-        List<Object> res = (List<Object>)deserialized.foo;
-
-        src.set(0, deserialized);
-
-        assertEquals(src, res);
-    }
-
-    /**
-     *
-     */
-    public void testHashSetRead() {
-        TestObjectContainer obj = new TestObjectContainer();
-        obj.foo = Sets.newHashSet(obj, "a");
-
-        PortableBuilderImpl mutObj = wrap(obj);
-
-        Set<Object> set = mutObj.getField("foo");
-
-        assert set.equals(Sets.newHashSet(mutObj, "a"));
-    }
-
-    /**
-     *
-     */
-    public void testHashSetOverride() {
-        TestObjectContainer obj = new TestObjectContainer();
-
-        PortableBuilderImpl mutObj = wrap(obj);
-
-        Set<Object> c = Sets.newHashSet(mutObj, "a", Sets.newHashSet(1, 2));
-
-        mutObj.setField("foo", c);
-
-        TestObjectContainer res = mutObj.build().deserialize();
-
-        c.remove(mutObj);
-        c.add(res);
-
-        assertNotSame(c, res.foo);
-        assertEquals(c, res.foo);
-    }
-
-    /**
-     *
-     */
-    public void testHashSetModification() {
-        TestObjectContainer obj = new TestObjectContainer();
-        obj.foo = Sets.newHashSet("a", "b", "c");
-
-        PortableBuilderImpl mutObj = wrap(obj);
-
-        Set<String> set = mutObj.getField("foo");
-
-        set.remove("b");
-        set.add("!");
-
-        assertEquals(Sets.newHashSet("a", "!", "c"), set);
-        assertTrue(set.contains("a"));
-        assertTrue(set.contains("!"));
-
-        TestObjectContainer res = mutObj.build().deserialize();
-
-        assertTrue(res.foo instanceof HashSet);
-        assertEquals(Sets.newHashSet("a", "!", "c"), res.foo);
-    }
-
-    /**
-     *
-     */
-    public void testHashSetWriteUnmodifiable() {
-        TestObjectContainer obj = new TestObjectContainer();
-
-        Set<Object> src = Sets.newHashSet(obj, "a", "b", "c");
-
-        obj.foo = src;
-
-        TestObjectContainer deserialized = wrap(obj).build().deserialize();
-
-        Set<Object> res = (Set<Object>)deserialized.foo;
-
-        src.remove(obj);
-        src.add(deserialized);
-
-        assertEquals(src, res);
-    }
-
-    /**
-     *
-     */
-    public void testMapRead() {
-        TestObjectContainer obj = new TestObjectContainer();
-        obj.foo = Maps.newHashMap(ImmutableMap.of(obj, "a", "b", obj));
-
-        PortableBuilderImpl mutObj = wrap(obj);
-
-        Map<Object, Object> map = mutObj.getField("foo");
-
-        assert map.equals(ImmutableMap.of(mutObj, "a", "b", mutObj));
-    }
-
-    /**
-     *
-     */
-    public void testMapOverride() {
-        TestObjectContainer obj = new TestObjectContainer();
-
-        PortableBuilderImpl mutObj = wrap(obj);
-
-        Map<Object, Object> map = Maps.newHashMap(ImmutableMap.of(mutObj, "a", "b", mutObj));
-
-        mutObj.setField("foo", map);
-
-        TestObjectContainer res = mutObj.build().deserialize();
-
-        assertEquals(ImmutableMap.of(res, "a", "b", res), res.foo);
-    }
-
-    /**
-     *
-     */
-    public void testMapModification() {
-        TestObjectContainer obj = new TestObjectContainer();
-        obj.foo = Maps.newHashMap(ImmutableMap.of(1, "a", 2, "b"));
-
-        PortableBuilderImpl mutObj = wrap(obj);
-
-        Map<Object, Object> map = mutObj.getField("foo");
-
-        map.put(3, mutObj);
-        Object rmv = map.remove(1);
-
-        assertEquals("a", rmv);
-
-        TestObjectContainer res = mutObj.build().deserialize();
-
-        assertEquals(ImmutableMap.of(2, "b", 3, res), res.foo);
-    }
-
-    /**
-     *
-     */
-    public void testEnumArrayModification() {
-        TestObjectAllTypes obj = new TestObjectAllTypes();
-
-        obj.enumArr = new TestObjectEnum[] {TestObjectEnum.A, TestObjectEnum.B};
-
-        PortableBuilderImpl mutObj = wrap(obj);
-
-        PortableBuilderEnum[] arr = mutObj.getField("enumArr");
-        arr[0] = new PortableBuilderEnum(mutObj.typeId(), TestObjectEnum.B);
-
-        TestObjectAllTypes res = mutObj.build().deserialize();
-
-        Assert.assertArrayEquals(new TestObjectEnum[] {TestObjectEnum.A, TestObjectEnum.B}, res.enumArr);
-    }
-
-    /**
-     *
-     */
-    public void testEditObjectWithRawData() {
-        GridPortableMarshalerAwareTestClass obj = new GridPortableMarshalerAwareTestClass();
-
-        obj.s = "a";
-        obj.sRaw = "aa";
-
-        PortableBuilderImpl mutableObj = wrap(obj);
-
-        mutableObj.setField("s", "z");
-
-        GridPortableMarshalerAwareTestClass res = mutableObj.build().deserialize();
-        assertEquals("z", res.s);
-        assertEquals("aa", res.sRaw);
-    }
-
-    /**
-     *
-     */
-    public void testHashCode() {
-        TestObjectContainer obj = new TestObjectContainer();
-
-        PortableBuilderImpl mutableObj = wrap(obj);
-
-        assertEquals(obj.hashCode(), mutableObj.build().hashCode());
-
-        mutableObj.hashCode(25);
-
-        assertEquals(25, mutableObj.build().hashCode());
-    }
-
-    /**
-     *
-     */
-    public void testCollectionsInCollection() {
-        TestObjectContainer obj = new TestObjectContainer();
-        obj.foo = Lists.newArrayList(
-            Lists.newArrayList(1, 2),
-            Lists.newLinkedList(Arrays.asList(1, 2)),
-            Sets.newHashSet("a", "b"),
-            Sets.newLinkedHashSet(Arrays.asList("a", "b")),
-            Maps.newHashMap(ImmutableMap.of(1, "a", 2, "b")));
-
-        TestObjectContainer deserialized = wrap(obj).build().deserialize();
-
-        assertEquals(obj.foo, deserialized.foo);
-    }
-
-    /**
-     *
-     */
-    public void testMapEntryModification() {
-        TestObjectContainer obj = new TestObjectContainer();
-        obj.foo = ImmutableMap.of(1, "a").entrySet().iterator().next();
-
-        PortableBuilderImpl mutableObj = wrap(obj);
-
-        Map.Entry<Object, Object> entry = mutableObj.getField("foo");
-
-        assertEquals(1, entry.getKey());
-        assertEquals("a", entry.getValue());
-
-        entry.setValue("b");
-
-        TestObjectContainer res = mutableObj.build().deserialize();
-
-        assertEquals(new GridMapEntry<>(1, "b"), res.foo);
-    }
-
-    /**
-     *
-     */
-    public void testMapEntryOverride() {
-        TestObjectContainer obj = new TestObjectContainer();
-
-        PortableBuilderImpl mutableObj = wrap(obj);
-
-        mutableObj.setField("foo", new GridMapEntry<>(1, "a"));
-
-        TestObjectContainer res = mutableObj.build().deserialize();
-
-        assertEquals(new GridMapEntry<>(1, "a"), res.foo);
-    }
-
-    /**
-     *
-     */
-    public void testMetadataChangingDoublePut() {
-        PortableBuilderImpl mutableObj = wrap(new TestObjectContainer());
-
-        mutableObj.setField("xx567", "a");
-        mutableObj.setField("xx567", "b");
-
-        mutableObj.build();
-
-        PortableMetadata metadata = portables().metadata(TestObjectContainer.class);
-
-        assertEquals("String", metadata.fieldTypeName("xx567"));
-    }
-
-    /**
-     *
-     */
-    public void testMetadataChangingDoublePut2() {
-        PortableBuilderImpl mutableObj = wrap(new TestObjectContainer());
-
-        mutableObj.setField("xx567", "a");
-        mutableObj.setField("xx567", "b");
-
-        mutableObj.build();
-
-        PortableMetadata metadata = portables().metadata(TestObjectContainer.class);
-
-        assertEquals("String", metadata.fieldTypeName("xx567"));
-    }
-
-    /**
-     *
-     */
-    public void testMetadataChanging() {
-        TestObjectContainer c = new TestObjectContainer();
-
-        PortableBuilderImpl mutableObj = wrap(c);
-
-        mutableObj.setField("intField", 1);
-        mutableObj.setField("intArrField", new int[] {1});
-        mutableObj.setField("arrField", new String[] {"1"});
-        mutableObj.setField("strField", "1");
-        mutableObj.setField("colField", Lists.newArrayList("1"));
-        mutableObj.setField("mapField", Maps.newHashMap(ImmutableMap.of(1, "1")));
-        mutableObj.setField("enumField", TestObjectEnum.A);
-        mutableObj.setField("enumArrField", new Enum[] {TestObjectEnum.A});
-
-        mutableObj.build();
-
-        PortableMetadata metadata = portables().metadata(c.getClass());
-
-        assertTrue(metadata.fields().containsAll(Arrays.asList("intField", "intArrField", "arrField", "strField",
-            "colField", "mapField", "enumField", "enumArrField")));
-
-        assertEquals("int", metadata.fieldTypeName("intField"));
-        assertEquals("int[]", metadata.fieldTypeName("intArrField"));
-        assertEquals("String[]", metadata.fieldTypeName("arrField"));
-        assertEquals("String", metadata.fieldTypeName("strField"));
-        assertEquals("Collection", metadata.fieldTypeName("colField"));
-        assertEquals("Map", metadata.fieldTypeName("mapField"));
-        assertEquals("Enum", metadata.fieldTypeName("enumField"));
-        assertEquals("Enum[]", metadata.fieldTypeName("enumArrField"));
-    }
-
-    /**
-     *
-     */
-    public void testDateInObjectField() {
-        TestObjectContainer obj = new TestObjectContainer();
-
-        obj.foo = new Date();
-
-        PortableBuilderImpl mutableObj = wrap(obj);
-
-        assertEquals(Timestamp.class, mutableObj.getField("foo").getClass());
-    }
-
-    /**
-     *
-     */
-    public void testDateInCollection() {
-        TestObjectContainer obj = new TestObjectContainer();
-
-        obj.foo = Lists.newArrayList(new Date());
-
-        PortableBuilderImpl mutableObj = wrap(obj);
-
-        assertEquals(Timestamp.class, ((List<?>)mutableObj.getField("foo")).get(0).getClass());
-    }
-
-    /**
-     *
-     */
-    @SuppressWarnings("AssertEqualsBetweenInconvertibleTypes")
-    public void testDateArrayOverride() {
-        TestObjectContainer obj = new TestObjectContainer();
-
-        PortableBuilderImpl mutableObj = wrap(obj);
-
-        Date[] arr = {new Date()};
-
-        mutableObj.setField("foo", arr);
-
-        TestObjectContainer res = mutableObj.build().deserialize();
-
-        assertEquals(Date[].class, res.foo.getClass());
-        assertTrue(Objects.deepEquals(arr, res.foo));
-    }
-
-    /**
-     *
-     */
-    public void testChangeMap() {
-        AddressBook addrBook = new AddressBook();
-
-        addrBook.addCompany(new Company(1, "Google inc", 100, new Address("Saint-Petersburg", "Torzhkovskya", 1, 53), "occupation"));
-        addrBook.addCompany(new Company(2, "Apple inc", 100, new Address("Saint-Petersburg", "Torzhkovskya", 1, 54), "occupation"));
-        addrBook.addCompany(new Company(3, "Microsoft", 100, new Address("Saint-Petersburg", "Torzhkovskya", 1, 55), "occupation"));
-        addrBook.addCompany(new Company(4, "Oracle", 100, new Address("Saint-Petersburg", "Nevskiy", 1, 1), "occupation"));
-
-        PortableBuilderImpl mutableObj = wrap(addrBook);
-
-        Map<String, List<PortableBuilderImpl>> map = mutableObj.getField("companyByStreet");
-
-        List<PortableBuilderImpl> list = map.get("Torzhkovskya");
-
-        PortableBuilderImpl company = list.get(0);
-
-        assert "Google inc".equals(company.<String>getField("name"));
-
-        list.remove(0);
-
-        AddressBook res = mutableObj.build().deserialize();
-
-        assertEquals(Arrays.asList("Nevskiy", "Torzhkovskya"), new ArrayList<>(res.getCompanyByStreet().keySet()));
-
-        List<Company> torzhkovskyaCompanies = res.getCompanyByStreet().get("Torzhkovskya");
-
-        assertEquals(2, torzhkovskyaCompanies.size());
-        assertEquals("Apple inc", torzhkovskyaCompanies.get(0).name);
-    }
-
-    /**
-     *
-     */
-    public void testSavingObjectWithNotZeroStart() {
-        TestObjectOuter out = new TestObjectOuter();
-        TestObjectInner inner = new TestObjectInner();
-
-        out.inner = inner;
-        inner.outer = out;
-
-        PortableBuilderImpl builder = wrap(out);
-
-        PortableBuilderImpl innerBuilder = builder.getField("inner");
-
-        TestObjectInner res = innerBuilder.build().deserialize();
-
-        assertSame(res, res.outer.inner);
-    }
-
-    /**
-     *
-     */
-    public void testPortableObjectField() {
-        TestObjectContainer container = new TestObjectContainer(toPortable(new TestObjectArrayList()));
-
-        PortableBuilderImpl wrapper = wrap(container);
-
-        assertTrue(wrapper.getField("foo") instanceof PortableObject);
-
-        TestObjectContainer deserialized = wrapper.build().deserialize();
-        assertTrue(deserialized.foo instanceof PortableObject);
-    }
-
-    /**
-     *
-     */
-    public void testAssignPortableObject() {
-        TestObjectContainer container = new TestObjectContainer();
-
-        PortableBuilderImpl wrapper = wrap(container);
-
-        wrapper.setField("foo", toPortable(new TestObjectArrayList()));
-
-        TestObjectContainer deserialized = wrapper.build().deserialize();
-        assertTrue(deserialized.foo instanceof TestObjectArrayList);
-    }
-
-    /**
-     *
-     */
-    public void testRemoveFromNewObject() {
-        PortableBuilderImpl wrapper = newWrapper(TestObjectAllTypes.class);
-
-        wrapper.setField("str", "a");
-
-        wrapper.removeField("str");
-
-        assertNull(wrapper.build().<TestObjectAllTypes>deserialize().str);
-    }
-
-    /**
-     *
-     */
-    public void testRemoveFromExistingObject() {
-        TestObjectAllTypes obj = new TestObjectAllTypes();
-        obj.setDefaultData();
-
-        PortableBuilderImpl wrapper = wrap(toPortable(obj));
-
-        wrapper.removeField("str");
-
-        assertNull(wrapper.build().<TestObjectAllTypes>deserialize().str);
-    }
-
-    /**
-     *
-     */
-    public void testCyclicArrays() {
-        TestObjectContainer obj = new TestObjectContainer();
-
-        Object[] arr1 = new Object[1];
-        Object[] arr2 = new Object[] {arr1};
-
-        arr1[0] = arr2;
-
-        obj.foo = arr1;
-
-        TestObjectContainer res = toPortable(obj).deserialize();
-
-        Object[] resArr = (Object[])res.foo;
-
-        assertSame(((Object[])resArr[0])[0], resArr);
-    }
-
-    /**
-     *
-     */
-    @SuppressWarnings("TypeMayBeWeakened")
-    public void testCyclicArrayList() {
-        TestObjectContainer obj = new TestObjectContainer();
-
-        List<Object> arr1 = new ArrayList<>();
-        List<Object> arr2 = new ArrayList<>();
-
-        arr1.add(arr2);
-        arr2.add(arr1);
-
-        obj.foo = arr1;
-
-        TestObjectContainer res = toPortable(obj).deserialize();
-
-        List<?> resArr = (List<?>)res.foo;
-
-        assertSame(((List<Object>)resArr.get(0)).get(0), resArr);
-    }
-
-    /**
-     * @param obj Object.
-     * @return Object in portable format.
-     */
-    private PortableObject toPortable(Object obj) {
-        return portables().toPortable(obj);
-    }
-
-    /**
-     * @param obj Object.
-     * @return GridMutablePortableObject.
-     */
-    private PortableBuilderImpl wrap(Object obj) {
-        return PortableBuilderImpl.wrap(toPortable(obj));
-    }
-
-    /**
-     * @param aCls Class.
-     * @return Wrapper.
-     */
-    private PortableBuilderImpl newWrapper(Class<?> aCls) {
-        CacheObjectPortableProcessorImpl processor = (CacheObjectPortableProcessorImpl)(
-            (IgnitePortablesImpl)portables()).processor();
-
-        return new PortableBuilderImpl(processor.portableContext(), processor.typeId(aCls.getName()),
-            aCls.getSimpleName());
-    }
-}
\ No newline at end of file


[36/55] [abbrv] ignite git commit: IGNITE-1400 On node stop prevent exchange worker hang on topology lock

Posted by ag...@apache.org.
IGNITE-1400 On node stop prevent exchange worker hang on topology lock


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

Branch: refs/heads/ignite-1171
Commit: a81cce7214c966de0281ef82da0b1fe042842911
Parents: 367d805
Author: sboikov <sb...@gridgain.com>
Authored: Tue Sep 15 15:46:16 2015 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Tue Sep 15 15:49:42 2015 +0300

----------------------------------------------------------------------
 .../dht/GridClientPartitionTopology.java        | 104 +++++++++++--------
 .../dht/GridDhtPartitionTopology.java           |   4 +-
 .../dht/GridDhtPartitionTopologyImpl.java       |   7 +-
 .../ignite/internal/util/IgniteUtils.java       |  16 +++
 4 files changed, 83 insertions(+), 48 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/a81cce72/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridClientPartitionTopology.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridClientPartitionTopology.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridClientPartitionTopology.java
index 21a7b3b..5e3cc0b 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridClientPartitionTopology.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridClientPartitionTopology.java
@@ -30,6 +30,7 @@ import java.util.concurrent.locks.ReentrantReadWriteLock;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.IgniteLogger;
 import org.apache.ignite.cluster.ClusterNode;
+import org.apache.ignite.internal.IgniteInterruptedCheckedException;
 import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
 import org.apache.ignite.internal.processors.cache.GridCacheSharedContext;
 import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionExchangeId;
@@ -110,7 +111,14 @@ public class GridClientPartitionTopology implements GridDhtPartitionTopology {
 
         log = cctx.logger(getClass());
 
-        beforeExchange(exchFut);
+        lock.writeLock().lock();
+
+        try {
+            beforeExchange0(cctx.localNode(), exchFut);
+        }
+        finally {
+            lock.writeLock().unlock();
+        }
     }
 
     /**
@@ -154,8 +162,8 @@ public class GridClientPartitionTopology implements GridDhtPartitionTopology {
         GridDhtPartitionsExchangeFuture exchFut,
         long updSeq,
         boolean stopping
-    ) {
-        lock.writeLock().lock();
+    ) throws IgniteInterruptedCheckedException {
+        U.writeLock(lock);
 
         try {
             assert exchId.topologyVersion().compareTo(topVer) > 0 : "Invalid topology version [topVer=" + topVer +
@@ -208,67 +216,75 @@ public class GridClientPartitionTopology implements GridDhtPartitionTopology {
     }
 
     /** {@inheritDoc} */
-    @Override public void beforeExchange(GridDhtPartitionsExchangeFuture exchFut) {
+    @Override public void beforeExchange(GridDhtPartitionsExchangeFuture exchFut) throws IgniteCheckedException {
         ClusterNode loc = cctx.localNode();
 
-        lock.writeLock().lock();
+        U.writeLock(lock);
 
         try {
             if (stopping)
                 return;
 
-            GridDhtPartitionExchangeId exchId = exchFut.exchangeId();
+            beforeExchange0(loc, exchFut);
+        }
+        finally {
+            lock.writeLock().unlock();
+        }
+    }
 
-            assert topVer.equals(exchId.topologyVersion()) : "Invalid topology version [topVer=" +
-                topVer + ", exchId=" + exchId + ']';
+    /**
+     * @param loc Local node.
+     * @param exchFut Exchange future.
+     */
+    private void beforeExchange0(ClusterNode loc, GridDhtPartitionsExchangeFuture exchFut) {
+        GridDhtPartitionExchangeId exchId = exchFut.exchangeId();
 
-            if (!exchId.isJoined())
-                removeNode(exchId.nodeId());
+        assert topVer.equals(exchId.topologyVersion()) : "Invalid topology version [topVer=" +
+            topVer + ", exchId=" + exchId + ']';
 
-            // In case if node joins, get topology at the time of joining node.
-            ClusterNode oldest = CU.oldestAliveCacheServerNode(cctx, topVer);
+        if (!exchId.isJoined())
+            removeNode(exchId.nodeId());
 
-            assert oldest != null;
+        // In case if node joins, get topology at the time of joining node.
+        ClusterNode oldest = CU.oldestAliveCacheServerNode(cctx, topVer);
 
-            if (log.isDebugEnabled())
-                log.debug("Partition map beforeExchange [exchId=" + exchId + ", fullMap=" + fullMapString() + ']');
+        assert oldest != null;
 
-            long updateSeq = this.updateSeq.incrementAndGet();
-
-            // If this is the oldest node.
-            if (oldest.id().equals(loc.id()) || exchFut.isCacheAdded(cacheId, exchId.topologyVersion())) {
-                if (node2part == null) {
-                    node2part = new GridDhtPartitionFullMap(oldest.id(), oldest.order(), updateSeq);
+        if (log.isDebugEnabled())
+            log.debug("Partition map beforeExchange [exchId=" + exchId + ", fullMap=" + fullMapString() + ']');
 
-                    if (log.isDebugEnabled())
-                        log.debug("Created brand new full topology map on oldest node [exchId=" +
-                            exchId + ", fullMap=" + fullMapString() + ']');
-                }
-                else if (!node2part.valid()) {
-                    node2part = new GridDhtPartitionFullMap(oldest.id(), oldest.order(), updateSeq, node2part, false);
+        long updateSeq = this.updateSeq.incrementAndGet();
 
-                    if (log.isDebugEnabled())
-                        log.debug("Created new full topology map on oldest node [exchId=" + exchId + ", fullMap=" +
-                            node2part + ']');
-                }
-                else if (!node2part.nodeId().equals(loc.id())) {
-                    node2part = new GridDhtPartitionFullMap(oldest.id(), oldest.order(), updateSeq, node2part, false);
+        // If this is the oldest node.
+        if (oldest.id().equals(loc.id()) || exchFut.isCacheAdded(cacheId, exchId.topologyVersion())) {
+            if (node2part == null) {
+                node2part = new GridDhtPartitionFullMap(oldest.id(), oldest.order(), updateSeq);
 
-                    if (log.isDebugEnabled())
-                        log.debug("Copied old map into new map on oldest node (previous oldest node left) [exchId=" +
-                            exchId + ", fullMap=" + fullMapString() + ']');
-                }
+                if (log.isDebugEnabled())
+                    log.debug("Created brand new full topology map on oldest node [exchId=" +
+                        exchId + ", fullMap=" + fullMapString() + ']');
             }
+            else if (!node2part.valid()) {
+                node2part = new GridDhtPartitionFullMap(oldest.id(), oldest.order(), updateSeq, node2part, false);
 
-            consistencyCheck();
+                if (log.isDebugEnabled())
+                    log.debug("Created new full topology map on oldest node [exchId=" + exchId + ", fullMap=" +
+                        node2part + ']');
+            }
+            else if (!node2part.nodeId().equals(loc.id())) {
+                node2part = new GridDhtPartitionFullMap(oldest.id(), oldest.order(), updateSeq, node2part, false);
 
-            if (log.isDebugEnabled())
-                log.debug("Partition map after beforeExchange [exchId=" + exchId + ", fullMap=" +
-                    fullMapString() + ']');
-        }
-        finally {
-            lock.writeLock().unlock();
+                if (log.isDebugEnabled())
+                    log.debug("Copied old map into new map on oldest node (previous oldest node left) [exchId=" +
+                        exchId + ", fullMap=" + fullMapString() + ']');
+            }
         }
+
+        consistencyCheck();
+
+        if (log.isDebugEnabled())
+            log.debug("Partition map after beforeExchange [exchId=" + exchId + ", fullMap=" +
+                fullMapString() + ']');
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/a81cce72/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtPartitionTopology.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtPartitionTopology.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtPartitionTopology.java
index d4ea3d6..d642314 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtPartitionTopology.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtPartitionTopology.java
@@ -22,6 +22,7 @@ import java.util.List;
 import java.util.UUID;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.cluster.ClusterNode;
+import org.apache.ignite.internal.IgniteInterruptedCheckedException;
 import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
 import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionExchangeId;
 import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionFullMap;
@@ -50,13 +51,14 @@ public interface GridDhtPartitionTopology {
      *
      * @param exchId Exchange ID.
      * @param exchFut Exchange future.
+     * @throws IgniteInterruptedCheckedException If interrupted.
      */
     public void updateTopologyVersion(
         GridDhtPartitionExchangeId exchId,
         GridDhtPartitionsExchangeFuture exchFut,
         long updateSeq,
         boolean stopping
-    );
+    ) throws IgniteInterruptedCheckedException;
 
     /**
      * Topology version.

http://git-wip-us.apache.org/repos/asf/ignite/blob/a81cce72/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtPartitionTopologyImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtPartitionTopologyImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtPartitionTopologyImpl.java
index fcb012f..a0c9c88 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtPartitionTopologyImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtPartitionTopologyImpl.java
@@ -34,6 +34,7 @@ import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.IgniteLogger;
 import org.apache.ignite.cluster.ClusterNode;
 import org.apache.ignite.events.DiscoveryEvent;
+import org.apache.ignite.internal.IgniteInterruptedCheckedException;
 import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
 import org.apache.ignite.internal.processors.cache.GridCacheContext;
 import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionExchangeId;
@@ -205,8 +206,8 @@ class GridDhtPartitionTopologyImpl implements GridDhtPartitionTopology {
         GridDhtPartitionsExchangeFuture exchFut,
         long updSeq,
         boolean stopping
-    ) {
-        lock.writeLock().lock();
+    ) throws IgniteInterruptedCheckedException {
+        U.writeLock(lock);
 
         try {
             assert exchId.topologyVersion().compareTo(topVer) > 0 : "Invalid topology version [topVer=" + topVer +
@@ -267,7 +268,7 @@ class GridDhtPartitionTopologyImpl implements GridDhtPartitionTopology {
 
         int num = cctx.affinity().partitions();
 
-        lock.writeLock().lock();
+        U.writeLock(lock);
 
         try {
             GridDhtPartitionExchangeId exchId = exchFut.exchangeId();

http://git-wip-us.apache.org/repos/asf/ignite/blob/a81cce72/modules/core/src/main/java/org/apache/ignite/internal/util/IgniteUtils.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/IgniteUtils.java b/modules/core/src/main/java/org/apache/ignite/internal/util/IgniteUtils.java
index ba918f6..e5090cb 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/util/IgniteUtils.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/util/IgniteUtils.java
@@ -119,6 +119,7 @@ import java.util.concurrent.Semaphore;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicReference;
 import java.util.concurrent.locks.Condition;
+import java.util.concurrent.locks.ReadWriteLock;
 import java.util.jar.JarFile;
 import java.util.logging.ConsoleHandler;
 import java.util.logging.Handler;
@@ -9286,4 +9287,19 @@ public abstract class IgniteUtils {
 
         return hasShmem;
     }
+
+    /**
+     * @param lock Lock.
+     * @throws IgniteInterruptedCheckedException If interrupted.
+     */
+    public static void writeLock(ReadWriteLock lock) throws IgniteInterruptedCheckedException {
+        try {
+            lock.writeLock().lockInterruptibly();
+        }
+        catch (InterruptedException e) {
+            Thread.currentThread().interrupt();
+
+            throw new IgniteInterruptedCheckedException(e);
+        }
+    }
 }
\ No newline at end of file


[10/55] [abbrv] ignite git commit: ignite-1462: hid portable API in 1.4 release

Posted by ag...@apache.org.
http://git-wip-us.apache.org/repos/asf/ignite/blob/71379a80/modules/core/src/main/java/org/apache/ignite/IgnitePortables.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/IgnitePortables.java b/modules/core/src/main/java/org/apache/ignite/IgnitePortables.java
deleted file mode 100644
index ee0a4ec..0000000
--- a/modules/core/src/main/java/org/apache/ignite/IgnitePortables.java
+++ /dev/null
@@ -1,370 +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;
-
-import java.sql.Timestamp;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.Date;
-import java.util.HashMap;
-import java.util.LinkedList;
-import java.util.TreeMap;
-import java.util.UUID;
-import org.apache.ignite.marshaller.portable.PortableMarshaller;
-import org.apache.ignite.portable.PortableBuilder;
-import org.apache.ignite.portable.PortableException;
-import org.apache.ignite.portable.PortableIdMapper;
-import org.apache.ignite.portable.PortableMarshalAware;
-import org.apache.ignite.portable.PortableMetadata;
-import org.apache.ignite.portable.PortableObject;
-import org.apache.ignite.portable.PortableSerializer;
-import org.apache.ignite.portable.PortableTypeConfiguration;
-import org.jetbrains.annotations.Nullable;
-
-/**
- * Defines portable objects functionality. With portable objects you are able to:
- * <ul>
- * <li>Seamlessly interoperate between Java, .NET, and C++.</li>
- * <li>Make any object portable with zero code change to your existing code.</li>
- * <li>Nest portable objects within each other.</li>
- * <li>Automatically handle {@code circular} or {@code null} references.</li>
- * <li>Automatically convert collections and maps between Java, .NET, and C++.</li>
- * <li>
- *      Optionally avoid deserialization of objects on the server side
- *      (objects are stored in {@link PortableObject} format).
- * </li>
- * <li>Avoid need to have concrete class definitions on the server side.</li>
- * <li>Dynamically change structure of the classes without having to restart the cluster.</li>
- * <li>Index into portable objects for querying purposes.</li>
- * </ul>
- * <h1 class="header">Working With Portables Directly</h1>
- * Once an object is defined as portable,
- * Ignite will always store it in memory in the portable (i.e. binary) format.
- * User can choose to work either with the portable format or with the deserialized form
- * (assuming that class definitions are present in the classpath).
- * <p>
- * To work with the portable format directly, user should create a special cache projection
- * using {@link IgniteCache#withKeepPortable()} method and then retrieve individual fields as needed:
- * <pre name=code class=java>
- * IgniteCache&lt;PortableObject, PortableObject&gt; prj = cache.withKeepPortable();
- *
- * // Convert instance of MyKey to portable format.
- * // We could also use PortableBuilder to create the key in portable format directly.
- * PortableObject key = grid.portables().toPortable(new MyKey());
- *
- * PortableObject val = prj.get(key);
- *
- * String field = val.field("myFieldName");
- * </pre>
- * Alternatively, if we have class definitions in the classpath, we may choose to work with deserialized
- * typed objects at all times. In this case we do incur the deserialization cost. However, if
- * {@link PortableMarshaller#isKeepDeserialized()} is {@code true} then Ignite will only deserialize on the first access
- * and will cache the deserialized object, so it does not have to be deserialized again:
- * <pre name=code class=java>
- * IgniteCache&lt;MyKey.class, MyValue.class&gt; cache = grid.cache(null);
- *
- * MyValue val = cache.get(new MyKey());
- *
- * // Normal java getter.
- * String fieldVal = val.getMyFieldName();
- * </pre>
- * If we used, for example, one of the automatically handled portable types for a key, like integer,
- * and still wanted to work with binary portable format for values, then we would declare cache projection
- * as follows:
- * <pre name=code class=java>
- * IgniteCache&lt;Integer.class, PortableObject&gt; prj = cache.withKeepPortable();
- * </pre>
- * <h1 class="header">Automatic Portable Types</h1>
- * Note that only portable classes are converted to {@link PortableObject} format. Following
- * classes are never converted (e.g., {@link #toPortable(Object)} method will return original
- * object, and instances of these classes will be stored in cache without changes):
- * <ul>
- *     <li>All primitives (byte, int, ...) and there boxed versions (Byte, Integer, ...)</li>
- *     <li>Arrays of primitives (byte[], int[], ...)</li>
- *     <li>{@link String} and array of {@link String}s</li>
- *     <li>{@link UUID} and array of {@link UUID}s</li>
- *     <li>{@link Date} and array of {@link Date}s</li>
- *     <li>{@link Timestamp} and array of {@link Timestamp}s</li>
- *     <li>Enums and array of enums</li>
- *     <li>
- *         Maps, collections and array of objects (but objects inside
- *         them will still be converted if they are portable)
- *     </li>
- * </ul>
- * <h1 class="header">Working With Maps and Collections</h1>
- * All maps and collections in the portable objects are serialized automatically. When working
- * with different platforms, e.g. C++ or .NET, Ignite will automatically pick the most
- * adequate collection or map in either language. For example, {@link ArrayList} in Java will become
- * {@code List} in C#, {@link LinkedList} in Java is {@link LinkedList} in C#, {@link HashMap}
- * in Java is {@code Dictionary} in C#, and {@link TreeMap} in Java becomes {@code SortedDictionary}
- * in C#, etc.
- * <h1 class="header">Building Portable Objects</h1>
- * Ignite comes with {@link PortableBuilder} which allows to build portable objects dynamically:
- * <pre name=code class=java>
- * PortableBuilder builder = Ignition.ignite().portables().builder();
- *
- * builder.typeId("MyObject");
- *
- * builder.stringField("fieldA", "A");
- * build.intField("fieldB", "B");
- *
- * PortableObject portableObj = builder.build();
- * </pre>
- * For the cases when class definition is present
- * in the class path, it is also possible to populate a standard POJO and then
- * convert it to portable format, like so:
- * <pre name=code class=java>
- * MyObject obj = new MyObject();
- *
- * obj.setFieldA("A");
- * obj.setFieldB(123);
- *
- * PortableObject portableObj = Ignition.ignite().portables().toPortable(obj);
- * </pre>
- * NOTE: you don't need to convert typed objects to portable format before storing
- * them in cache, Ignite will do that automatically.
- * <h1 class="header">Portable Metadata</h1>
- * Even though Ignite portable protocol only works with hash codes for type and field names
- * to achieve better performance, Ignite provides metadata for all portable types which
- * can be queried ar runtime via any of the {@link IgnitePortables#metadata(Class)}
- * methods. Having metadata also allows for proper formatting of {@code PortableObject#toString()} method,
- * even when portable objects are kept in binary format only, which may be necessary for audit reasons.
- * <h1 class="header">Dynamic Structure Changes</h1>
- * Since objects are always cached in the portable binary format, server does not need to
- * be aware of the class definitions. Moreover, if class definitions are not present or not
- * used on the server, then clients can continuously change the structure of the portable
- * objects without having to restart the cluster. For example, if one client stores a
- * certain class with fields A and B, and another client stores the same class with
- * fields B and C, then the server-side portable object will have the fields A, B, and C.
- * As the structure of a portable object changes, the new fields become available for SQL queries
- * automatically.
- * <h1 class="header">Configuration</h1>
- * By default all your objects are considered as portables and no specific configuration is needed.
- * However, in some cases, like when an object is used by both Java and .Net, you may need to specify portable objects
- * explicitly by calling {@link PortableMarshaller#setClassNames(Collection)}.
- * The only requirement Ignite imposes is that your object has an empty
- * constructor. Note, that since server side does not have to know the class definition,
- * you only need to list portable objects in configuration on the client side. However, if you
- * list them on the server side as well, then you get the ability to deserialize portable objects
- * into concrete types on the server as well as on the client.
- * <p>
- * Here is an example of portable configuration (note that star (*) notation is supported):
- * <pre name=code class=xml>
- * ...
- * &lt;!-- Explicit portable objects configuration. --&gt;
- * &lt;property name="marshaller"&gt;
- *     &lt;bean class="org.apache.ignite.marshaller.portable.PortableMarshaller"&gt;
- *         &lt;property name="classNames"&gt;
- *             &lt;list&gt;
- *                 &lt;value&gt;my.package.for.portable.objects.*&lt;/value&gt;
- *                 &lt;value&gt;org.apache.ignite.examples.client.portable.Employee&lt;/value&gt;
- *             &lt;/list&gt;
- *         &lt;/property&gt;
- *     &lt;/bean&gt;
- * &lt;/property&gt;
- * ...
- * </pre>
- * or from code:
- * <pre name=code class=java>
- * IgniteConfiguration cfg = new IgniteConfiguration();
- *
- * PortableMarshaller marsh = new PortableMarshaller();
- *
- * marsh.setClassNames(Arrays.asList(
- *     Employee.class.getName(),
- *     Address.class.getName())
- * );
- *
- * cfg.setMarshaller(marsh);
- * </pre>
- * You can also specify class name for a portable object via {@link PortableTypeConfiguration}.
- * Do it in case if you need to override other configuration properties on per-type level, like
- * ID-mapper, or serializer.
- * <h1 class="header">Custom Affinity Keys</h1>
- * Often you need to specify an alternate key (not the cache key) for affinity routing whenever
- * storing objects in cache. For example, if you are caching {@code Employee} object with
- * {@code Organization}, and want to colocate employees with organization they work for,
- * so you can process them together, you need to specify an alternate affinity key.
- * With portable objects you would have to do it as following:
- * <pre name=code class=xml>
- * &lt;property name="marshaller"&gt;
- *     &lt;bean class="org.gridgain.grid.marshaller.portable.PortableMarshaller"&gt;
- *         ...
- *         &lt;property name="typeConfigurations"&gt;
- *             &lt;list&gt;
- *                 &lt;bean class="org.apache.ignite.portable.PortableTypeConfiguration"&gt;
- *                     &lt;property name="className" value="org.apache.ignite.examples.client.portable.EmployeeKey"/&gt;
- *                     &lt;property name="affinityKeyFieldName" value="organizationId"/&gt;
- *                 &lt;/bean&gt;
- *             &lt;/list&gt;
- *         &lt;/property&gt;
- *         ...
- *     &lt;/bean&gt;
- * &lt;/property&gt;
- * </pre>
- * <h1 class="header">Serialization</h1>
- * Serialization and deserialization works out-of-the-box in Ignite. However, you can provide your own custom
- * serialization logic by optionally implementing {@link PortableMarshalAware} interface, like so:
- * <pre name=code class=java>
- * public class Address implements PortableMarshalAware {
- *     private String street;
- *     private int zip;
- *
- *     // Empty constructor required for portable deserialization.
- *     public Address() {}
- *
- *     &#64;Override public void writePortable(PortableWriter writer) throws PortableException {
- *         writer.writeString("street", street);
- *         writer.writeInt("zip", zip);
- *     }
- *
- *     &#64;Override public void readPortable(PortableReader reader) throws PortableException {
- *         street = reader.readString("street");
- *         zip = reader.readInt("zip");
- *     }
- * }
- * </pre>
- * Alternatively, if you cannot change class definitions, you can provide custom serialization
- * logic in {@link PortableSerializer} either globally in {@link PortableMarshaller} or
- * for a specific type via {@link PortableTypeConfiguration} instance.
- * <p>
- * Similar to java serialization you can use {@code writeReplace()} and {@code readResolve()} methods.
- * <ul>
- *     <li>
- *         {@code readResolve} is defined as follows: {@code ANY-ACCESS-MODIFIER Object readResolve()}.
- *         It may be used to replace the de-serialized object by another one of your choice.
- *     </li>
- *     <li>
- *          {@code writeReplace} is defined as follows: {@code ANY-ACCESS-MODIFIER Object writeReplace()}. This method
- *          allows the developer to provide a replacement object that will be serialized instead of the original one.
- *     </li>
- * </ul>
- *
- * <h1 class="header">Custom ID Mappers</h1>
- * Ignite implementation uses name hash codes to generate IDs for class names or field names
- * internally. However, in cases when you want to provide your own ID mapping schema,
- * you can provide your own {@link PortableIdMapper} implementation.
- * <p>
- * ID-mapper may be provided either globally in {@link PortableMarshaller},
- * or for a specific type via {@link PortableTypeConfiguration} instance.
- * <h1 class="header">Query Indexing</h1>
- * Portable objects can be indexed for querying by specifying index fields in
- * {@link org.apache.ignite.cache.CacheTypeMetadata} inside of specific
- * {@link org.apache.ignite.configuration.CacheConfiguration} instance,
- * like so:
- * <pre name=code class=xml>
- * ...
- * &lt;bean class="org.apache.ignite.cache.CacheConfiguration"&gt;
- *     ...
- *     &lt;property name="typeMetadata"&gt;
- *         &lt;list&gt;
- *             &lt;bean class="CacheTypeMetadata"&gt;
- *                 &lt;property name="type" value="Employee"/&gt;
- *
- *                 &lt;!-- Fields to index in ascending order. --&gt;
- *                 &lt;property name="ascendingFields"&gt;
- *                     &lt;map&gt;
- *                     &lt;entry key="name" value="java.lang.String"/&gt;
- *
- *                         &lt;!-- Nested portable objects can also be indexed. --&gt;
- *                         &lt;entry key="address.zip" value="java.lang.Integer"/&gt;
- *                     &lt;/map&gt;
- *                 &lt;/property&gt;
- *             &lt;/bean&gt;
- *         &lt;/list&gt;
- *     &lt;/property&gt;
- * &lt;/bean&gt;
- * </pre>
- */
-public interface IgnitePortables {
-    /**
-     * Gets type ID for given type name.
-     *
-     * @param typeName Type name.
-     * @return Type ID.
-     */
-    public int typeId(String typeName);
-
-    /**
-     * Converts provided object to instance of {@link PortableObject}.
-     *
-     * @param obj Object to convert.
-     * @return Converted object.
-     * @throws PortableException In case of error.
-     */
-    public <T> T toPortable(@Nullable Object obj) throws PortableException;
-
-    /**
-     * Creates new portable builder.
-     *
-     * @param typeId ID of the type.
-     * @return Newly portable builder.
-     */
-    public PortableBuilder builder(int typeId);
-
-    /**
-     * Creates new portable builder.
-     *
-     * @param typeName Type name.
-     * @return Newly portable builder.
-     */
-    public PortableBuilder builder(String typeName);
-
-    /**
-     * Creates portable builder initialized by existing portable object.
-     *
-     * @param portableObj Portable object to initialize builder.
-     * @return Portable builder.
-     */
-    public PortableBuilder builder(PortableObject portableObj);
-
-    /**
-     * Gets metadata for provided class.
-     *
-     * @param cls Class.
-     * @return Metadata.
-     * @throws PortableException In case of error.
-     */
-    @Nullable public PortableMetadata metadata(Class<?> cls) throws PortableException;
-
-    /**
-     * Gets metadata for provided class name.
-     *
-     * @param typeName Type name.
-     * @return Metadata.
-     * @throws PortableException In case of error.
-     */
-    @Nullable public PortableMetadata metadata(String typeName) throws PortableException;
-
-    /**
-     * Gets metadata for provided type ID.
-     *
-     * @param typeId Type ID.
-     * @return Metadata.
-     * @throws PortableException In case of error.
-     */
-    @Nullable public PortableMetadata metadata(int typeId) throws PortableException;
-
-    /**
-     * Gets metadata for all known types.
-     *
-     * @return Metadata.
-     * @throws PortableException In case of error.
-     */
-    public Collection<PortableMetadata> metadata() throws PortableException;
-}

http://git-wip-us.apache.org/repos/asf/ignite/blob/71379a80/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 9fb56bc..7d1e14d 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,10 +17,24 @@
 
 package org.apache.ignite.configuration;
 
+import java.io.Serializable;
+import java.util.Collection;
+import javax.cache.Cache;
+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.*;
+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.affinity.AffinityFunction;
 import org.apache.ignite.cache.affinity.AffinityKeyMapper;
 import org.apache.ignite.cache.eviction.EvictionFilter;
@@ -36,15 +50,6 @@ 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
@@ -163,10 +168,6 @@ public class CacheConfiguration<K, V> extends MutableConfiguration<K, V> {
     /** Default size for onheap SQL row cache size. */
     public static final int DFLT_SQL_ONHEAP_ROW_CACHE_SIZE = 10 * 1024;
 
-    /** Default value for keep portable in store behavior .*/
-    @SuppressWarnings({"UnnecessaryBoxing", "BooleanConstructorCall"})
-    public static final Boolean DFLT_KEEP_PORTABLE_IN_STORE  = new Boolean(true);
-
     /** Cache name. */
     private String name;
 
@@ -219,9 +220,6 @@ public class CacheConfiguration<K, V> extends MutableConfiguration<K, V> {
     private Factory storeFactory;
 
     /** */
-    private Boolean keepPortableInStore = DFLT_KEEP_PORTABLE_IN_STORE;
-
-    /** */
     private boolean loadPrevVal = DFLT_LOAD_PREV_VAL;
 
     /** Node group resolver. */
@@ -383,8 +381,6 @@ public class CacheConfiguration<K, V> extends MutableConfiguration<K, V> {
         invalidate = cc.isInvalidate();
         isReadThrough = cc.isReadThrough();
         isWriteThrough = cc.isWriteThrough();
-        keepPortableInStore = cc.isKeepPortableInStore() != null ? cc.isKeepPortableInStore() :
-            DFLT_KEEP_PORTABLE_IN_STORE;
         listenerConfigurations = cc.listenerConfigurations;
         loadPrevVal = cc.isLoadPreviousValue();
         longQryWarnTimeout = cc.getLongQueryWarningTimeout();
@@ -825,38 +821,6 @@ public class CacheConfiguration<K, V> extends MutableConfiguration<K, V> {
     }
 
     /**
-     * Flag indicating that {@link CacheStore} implementation
-     * is working with portable objects instead of Java objects.
-     * Default value of this flag is {@link #DFLT_KEEP_PORTABLE_IN_STORE},
-     * because this is recommended behavior from performance standpoint.
-     * <p>
-     * If set to {@code false}, Ignite will deserialize keys and
-     * values stored in portable format before they are passed
-     * to cache store.
-     * <p>
-     * Note that setting this flag to {@code false} can simplify
-     * store implementation in some cases, but it can cause performance
-     * degradation due to additional serializations and deserializations
-     * of portable objects. You will also need to have key and value
-     * classes on all nodes since portables will be deserialized when
-     * store is called.
-     *
-     * @return Keep portables in store flag.
-     */
-    public Boolean isKeepPortableInStore() {
-        return keepPortableInStore;
-    }
-
-    /**
-     * Sets keep portables in store flag.
-     *
-     * @param keepPortableInStore Keep portables in store flag.
-     */
-    public void setKeepPortableInStore(boolean keepPortableInStore) {
-        this.keepPortableInStore = keepPortableInStore;
-    }
-
-    /**
      * Gets key topology resolver to provide mapping from keys to nodes.
      *
      * @return Key topology resolver to provide mapping from keys to nodes.
@@ -1860,4 +1824,4 @@ public class CacheConfiguration<K, V> extends MutableConfiguration<K, V> {
             return obj.getClass().equals(this.getClass());
         }
     }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/71379a80/modules/core/src/main/java/org/apache/ignite/internal/IgniteEx.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/IgniteEx.java b/modules/core/src/main/java/org/apache/ignite/internal/IgniteEx.java
index e3859c5..9443f21 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/IgniteEx.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/IgniteEx.java
@@ -20,6 +20,7 @@ package org.apache.ignite.internal;
 import java.util.Collection;
 import org.apache.ignite.Ignite;
 import org.apache.ignite.IgniteFileSystem;
+import org.apache.ignite.internal.portable.api.IgnitePortables;
 import org.apache.ignite.cluster.ClusterNode;
 import org.apache.ignite.internal.cluster.IgniteClusterEx;
 import org.apache.ignite.internal.processors.cache.GridCacheUtilityKey;
@@ -140,4 +141,12 @@ public interface IgniteEx extends Ignite {
      * @return Kernal context.
      */
     public GridKernalContext context();
+
+
+    /**
+     * Gets an instance of {@link IgnitePortables} interface.
+     *
+     * @return Instance of {@link IgnitePortables} interface.
+     */
+    public IgnitePortables portables();
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/71379a80/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 9b615b1..abab1f3 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
@@ -64,7 +64,7 @@ import org.apache.ignite.IgniteException;
 import org.apache.ignite.IgniteFileSystem;
 import org.apache.ignite.IgniteLogger;
 import org.apache.ignite.IgniteMessaging;
-import org.apache.ignite.IgnitePortables;
+import org.apache.ignite.internal.portable.api.IgnitePortables;
 import org.apache.ignite.IgniteQueue;
 import org.apache.ignite.IgniteScheduler;
 import org.apache.ignite.IgniteServices;
@@ -157,7 +157,7 @@ import org.apache.ignite.lifecycle.LifecycleBean;
 import org.apache.ignite.lifecycle.LifecycleEventType;
 import org.apache.ignite.marshaller.MarshallerExclusions;
 import org.apache.ignite.marshaller.optimized.OptimizedMarshaller;
-import org.apache.ignite.marshaller.portable.PortableMarshaller;
+import org.apache.ignite.internal.portable.api.PortableMarshaller;
 import org.apache.ignite.mxbean.ClusterLocalNodeMetricsMXBean;
 import org.apache.ignite.mxbean.IgniteMXBean;
 import org.apache.ignite.mxbean.ThreadPoolMXBean;
@@ -203,7 +203,6 @@ import static org.apache.ignite.internal.IgniteNodeAttributes.ATTR_MARSHALLER;
 import static org.apache.ignite.internal.IgniteNodeAttributes.ATTR_NODE_CONSISTENT_ID;
 import static org.apache.ignite.internal.IgniteNodeAttributes.ATTR_PEER_CLASSLOADING;
 import static org.apache.ignite.internal.IgniteNodeAttributes.ATTR_PHY_RAM;
-import static org.apache.ignite.internal.IgniteNodeAttributes.ATTR_PORTABLE_PROTO_VER;
 import static org.apache.ignite.internal.IgniteNodeAttributes.ATTR_PREFIX;
 import static org.apache.ignite.internal.IgniteNodeAttributes.ATTR_RESTART_ENABLED;
 import static org.apache.ignite.internal.IgniteNodeAttributes.ATTR_REST_PORT_RANGE;
@@ -1270,9 +1269,6 @@ public class IgniteKernal implements IgniteEx, IgniteMXBean, Externalizable {
         add(ATTR_MARSHALLER, cfg.getMarshaller().getClass().getName());
         add(ATTR_USER_NAME, System.getProperty("user.name"));
         add(ATTR_GRID_NAME, gridName);
-        add(ATTR_PORTABLE_PROTO_VER, cfg.getMarshaller() instanceof PortableMarshaller ?
-            ((PortableMarshaller)cfg.getMarshaller()).getProtocolVersion().toString() :
-            PortableMarshaller.DFLT_PORTABLE_PROTO_VER.toString());
 
         add(ATTR_PEER_CLASSLOADING, cfg.isPeerClassLoadingEnabled());
         add(ATTR_DEPLOYMENT_MODE, cfg.getDeploymentMode());

http://git-wip-us.apache.org/repos/asf/ignite/blob/71379a80/modules/core/src/main/java/org/apache/ignite/internal/IgniteNodeAttributes.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/IgniteNodeAttributes.java b/modules/core/src/main/java/org/apache/ignite/internal/IgniteNodeAttributes.java
index 7be2af3..10b8df0 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/IgniteNodeAttributes.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/IgniteNodeAttributes.java
@@ -135,13 +135,10 @@ public final class IgniteNodeAttributes {
     /** Node consistent id. */
     public static final String ATTR_NODE_CONSISTENT_ID = ATTR_PREFIX + ".consistent.id";
 
-    /** Portable protocol version. */
-    public static final String ATTR_PORTABLE_PROTO_VER = ATTR_PREFIX + ".portable.proto.ver";
-
     /**
      * Enforces singleton.
      */
     private IgniteNodeAttributes() {
         /* No-op. */
     }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/71379a80/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 bc4f756..3a09b2c 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
@@ -125,7 +125,6 @@ import static org.apache.ignite.events.EventType.EVT_NODE_SEGMENTED;
 import static org.apache.ignite.internal.IgniteNodeAttributes.ATTR_DEPLOYMENT_MODE;
 import static org.apache.ignite.internal.IgniteNodeAttributes.ATTR_MACS;
 import static org.apache.ignite.internal.IgniteNodeAttributes.ATTR_PEER_CLASSLOADING;
-import static org.apache.ignite.internal.IgniteNodeAttributes.ATTR_PORTABLE_PROTO_VER;
 import static org.apache.ignite.internal.IgniteNodeAttributes.ATTR_USER_NAME;
 import static org.apache.ignite.internal.IgniteVersionUtils.VER;
 import static org.apache.ignite.plugin.segmentation.SegmentationPolicy.NOOP;
@@ -982,8 +981,6 @@ public class GridDiscoveryManager extends GridManagerAdapter<DiscoverySpi> {
         // Fetch local node attributes once.
         String locPreferIpV4 = locNode.attribute("java.net.preferIPv4Stack");
 
-        String locPortableProtoVer = locNode.attribute(ATTR_PORTABLE_PROTO_VER);
-
         Object locMode = locNode.attribute(ATTR_DEPLOYMENT_MODE);
 
         int locJvmMajVer = nodeJavaMajorVersion(locNode);
@@ -1033,13 +1030,6 @@ public class GridDiscoveryManager extends GridManagerAdapter<DiscoverySpi> {
                         ", rmtId8=" + U.id8(n.id()) + ", rmtPeerClassLoading=" + rmtP2pEnabled +
                         ", rmtAddrs=" + U.addressesAsString(n) + ']');
             }
-
-            String rmtPortableProtoVer = n.attribute(ATTR_PORTABLE_PROTO_VER);
-
-            // In order to support backward compatibility skip the check for nodes that don't have this attribute.
-            if (rmtPortableProtoVer != null && !F.eq(locPortableProtoVer, rmtPortableProtoVer))
-                throw new IgniteCheckedException("Remote node has portable protocol version different from local " +
-                    "[locVersion=" + locPortableProtoVer + ", rmtVersion=" + rmtPortableProtoVer + ']');
         }
 
         if (log.isDebugEnabled())

http://git-wip-us.apache.org/repos/asf/ignite/blob/71379a80/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 c7a9e6f..4bc8545 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
@@ -19,7 +19,7 @@ package org.apache.ignite.internal.portable;
 
 import org.apache.ignite.internal.portable.streams.PortableInputStream;
 import org.apache.ignite.internal.portable.streams.PortableOutputStream;
-import org.apache.ignite.portable.PortableException;
+import org.apache.ignite.internal.portable.api.PortableException;
 import org.jetbrains.annotations.Nullable;
 
 /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/71379a80/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 a2b4b74..e1b7324 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
@@ -40,11 +40,11 @@ import org.apache.ignite.internal.processors.cache.CacheObjectImpl;
 import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.marshaller.MarshallerExclusions;
 import org.apache.ignite.marshaller.optimized.OptimizedMarshaller;
-import org.apache.ignite.marshaller.portable.PortableMarshaller;
-import org.apache.ignite.portable.PortableException;
-import org.apache.ignite.portable.PortableIdMapper;
-import org.apache.ignite.portable.PortableMarshalAware;
-import org.apache.ignite.portable.PortableSerializer;
+import org.apache.ignite.internal.portable.api.PortableMarshaller;
+import org.apache.ignite.internal.portable.api.PortableException;
+import org.apache.ignite.internal.portable.api.PortableIdMapper;
+import org.apache.ignite.internal.portable.api.PortableMarshalAware;
+import org.apache.ignite.internal.portable.api.PortableSerializer;
 import org.jetbrains.annotations.Nullable;
 
 import static java.lang.reflect.Modifier.isStatic;

http://git-wip-us.apache.org/repos/asf/ignite/blob/71379a80/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 165ad9a..2ee96b7 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
@@ -60,16 +60,16 @@ import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.lang.IgniteBiTuple;
 import org.apache.ignite.marshaller.MarshallerContext;
 import org.apache.ignite.marshaller.optimized.OptimizedMarshaller;
-import org.apache.ignite.marshaller.portable.PortableMarshaller;
+import org.apache.ignite.internal.portable.api.PortableMarshaller;
 import org.apache.ignite.internal.processors.platform.dotnet.PlatformDotNetConfiguration;
 import org.apache.ignite.internal.processors.platform.dotnet.PlatformDotNetPortableConfiguration;
 import org.apache.ignite.internal.processors.platform.dotnet.PlatformDotNetPortableTypeConfiguration;
-import org.apache.ignite.portable.PortableException;
-import org.apache.ignite.portable.PortableIdMapper;
-import org.apache.ignite.portable.PortableInvalidClassException;
-import org.apache.ignite.portable.PortableMetadata;
-import org.apache.ignite.portable.PortableSerializer;
-import org.apache.ignite.portable.PortableTypeConfiguration;
+import org.apache.ignite.internal.portable.api.PortableException;
+import org.apache.ignite.internal.portable.api.PortableIdMapper;
+import org.apache.ignite.internal.portable.api.PortableInvalidClassException;
+import org.apache.ignite.internal.portable.api.PortableMetadata;
+import org.apache.ignite.internal.portable.api.PortableSerializer;
+import org.apache.ignite.internal.portable.api.PortableTypeConfiguration;
 import org.jetbrains.annotations.Nullable;
 import org.jsr166.ConcurrentHashMap8;
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/71379a80/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableMetaDataCollector.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableMetaDataCollector.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableMetaDataCollector.java
index ae5fbf0..05e7f20 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableMetaDataCollector.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableMetaDataCollector.java
@@ -27,9 +27,9 @@ import java.util.Date;
 import java.util.HashMap;
 import java.util.Map;
 import java.util.UUID;
-import org.apache.ignite.portable.PortableException;
-import org.apache.ignite.portable.PortableRawWriter;
-import org.apache.ignite.portable.PortableWriter;
+import org.apache.ignite.internal.portable.api.PortableException;
+import org.apache.ignite.internal.portable.api.PortableRawWriter;
+import org.apache.ignite.internal.portable.api.PortableWriter;
 import org.jetbrains.annotations.Nullable;
 
 /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/71379a80/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableMetaDataHandler.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableMetaDataHandler.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableMetaDataHandler.java
index e03d67f..fafafad 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableMetaDataHandler.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableMetaDataHandler.java
@@ -17,8 +17,8 @@
 
 package org.apache.ignite.internal.portable;
 
-import org.apache.ignite.portable.PortableException;
-import org.apache.ignite.portable.PortableMetadata;
+import org.apache.ignite.internal.portable.api.PortableException;
+import org.apache.ignite.internal.portable.api.PortableMetadata;
 
 /**
  * Portable meta data handler.

http://git-wip-us.apache.org/repos/asf/ignite/blob/71379a80/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableMetaDataImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableMetaDataImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableMetaDataImpl.java
index 1d26007..c0423eb 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableMetaDataImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableMetaDataImpl.java
@@ -27,13 +27,13 @@ import java.util.Map;
 import org.apache.ignite.internal.util.tostring.GridToStringInclude;
 import org.apache.ignite.internal.util.typedef.internal.S;
 import org.apache.ignite.internal.util.typedef.internal.U;
-import org.apache.ignite.portable.PortableException;
-import org.apache.ignite.portable.PortableMarshalAware;
-import org.apache.ignite.portable.PortableMetadata;
-import org.apache.ignite.portable.PortableRawReader;
-import org.apache.ignite.portable.PortableRawWriter;
-import org.apache.ignite.portable.PortableReader;
-import org.apache.ignite.portable.PortableWriter;
+import org.apache.ignite.internal.portable.api.PortableException;
+import org.apache.ignite.internal.portable.api.PortableMarshalAware;
+import org.apache.ignite.internal.portable.api.PortableMetadata;
+import org.apache.ignite.internal.portable.api.PortableRawReader;
+import org.apache.ignite.internal.portable.api.PortableRawWriter;
+import org.apache.ignite.internal.portable.api.PortableReader;
+import org.apache.ignite.internal.portable.api.PortableWriter;
 import org.jetbrains.annotations.Nullable;
 
 /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/71379a80/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableObjectEx.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableObjectEx.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableObjectEx.java
index fe4b628..229c90f 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableObjectEx.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableObjectEx.java
@@ -23,9 +23,9 @@ import java.util.IdentityHashMap;
 import org.apache.ignite.IgniteException;
 import org.apache.ignite.internal.util.offheap.unsafe.GridUnsafeMemory;
 import org.apache.ignite.internal.util.typedef.internal.SB;
-import org.apache.ignite.portable.PortableException;
-import org.apache.ignite.portable.PortableMetadata;
-import org.apache.ignite.portable.PortableObject;
+import org.apache.ignite.internal.portable.api.PortableException;
+import org.apache.ignite.internal.portable.api.PortableMetadata;
+import org.apache.ignite.internal.portable.api.PortableObject;
 import org.jetbrains.annotations.Nullable;
 
 /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/71379a80/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableObjectImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableObjectImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableObjectImpl.java
index 47ff1ab..cb81efe 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableObjectImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableObjectImpl.java
@@ -34,9 +34,9 @@ import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.plugin.extensions.communication.Message;
 import org.apache.ignite.plugin.extensions.communication.MessageReader;
 import org.apache.ignite.plugin.extensions.communication.MessageWriter;
-import org.apache.ignite.portable.PortableException;
-import org.apache.ignite.portable.PortableMetadata;
-import org.apache.ignite.portable.PortableObject;
+import org.apache.ignite.internal.portable.api.PortableException;
+import org.apache.ignite.internal.portable.api.PortableMetadata;
+import org.apache.ignite.internal.portable.api.PortableObject;
 import org.jetbrains.annotations.Nullable;
 
 /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/71379a80/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableObjectOffheapImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableObjectOffheapImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableObjectOffheapImpl.java
index ba8ee83..e788422 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableObjectOffheapImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableObjectOffheapImpl.java
@@ -30,9 +30,9 @@ import org.apache.ignite.internal.util.GridUnsafe;
 import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.plugin.extensions.communication.MessageReader;
 import org.apache.ignite.plugin.extensions.communication.MessageWriter;
-import org.apache.ignite.portable.PortableException;
-import org.apache.ignite.portable.PortableMetadata;
-import org.apache.ignite.portable.PortableObject;
+import org.apache.ignite.internal.portable.api.PortableException;
+import org.apache.ignite.internal.portable.api.PortableMetadata;
+import org.apache.ignite.internal.portable.api.PortableObject;
 import org.jetbrains.annotations.Nullable;
 import sun.misc.Unsafe;
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/71379a80/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableRawReaderEx.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableRawReaderEx.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableRawReaderEx.java
index e703f2f..e401142 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableRawReaderEx.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableRawReaderEx.java
@@ -17,8 +17,8 @@
 
 package org.apache.ignite.internal.portable;
 
-import org.apache.ignite.portable.PortableException;
-import org.apache.ignite.portable.PortableRawReader;
+import org.apache.ignite.internal.portable.api.PortableException;
+import org.apache.ignite.internal.portable.api.PortableRawReader;
 import org.jetbrains.annotations.Nullable;
 
 /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/71379a80/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableRawWriterEx.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableRawWriterEx.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableRawWriterEx.java
index a59f157..43b7650 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableRawWriterEx.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableRawWriterEx.java
@@ -18,8 +18,8 @@
 package org.apache.ignite.internal.portable;
 
 import org.apache.ignite.internal.portable.streams.PortableOutputStream;
-import org.apache.ignite.portable.PortableException;
-import org.apache.ignite.portable.PortableRawWriter;
+import org.apache.ignite.internal.portable.api.PortableException;
+import org.apache.ignite.internal.portable.api.PortableRawWriter;
 import org.jetbrains.annotations.Nullable;
 
 /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/71379a80/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableReaderContext.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableReaderContext.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableReaderContext.java
index 2d4a1c3..2537926 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableReaderContext.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableReaderContext.java
@@ -20,7 +20,7 @@ package org.apache.ignite.internal.portable;
 import java.util.HashMap;
 import java.util.Map;
 import org.apache.ignite.internal.util.typedef.internal.S;
-import org.apache.ignite.portable.PortableObject;
+import org.apache.ignite.internal.portable.api.PortableObject;
 import org.jetbrains.annotations.Nullable;
 
 /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/71379a80/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 4ad125a..a101db5 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
@@ -45,11 +45,11 @@ import org.apache.ignite.internal.util.GridEnumCache;
 import org.apache.ignite.internal.util.lang.GridMapEntry;
 import org.apache.ignite.internal.util.typedef.internal.SB;
 import org.apache.ignite.internal.util.typedef.internal.U;
-import org.apache.ignite.portable.PortableException;
-import org.apache.ignite.portable.PortableInvalidClassException;
-import org.apache.ignite.portable.PortableObject;
-import org.apache.ignite.portable.PortableRawReader;
-import org.apache.ignite.portable.PortableReader;
+import org.apache.ignite.internal.portable.api.PortableException;
+import org.apache.ignite.internal.portable.api.PortableInvalidClassException;
+import org.apache.ignite.internal.portable.api.PortableObject;
+import org.apache.ignite.internal.portable.api.PortableRawReader;
+import org.apache.ignite.internal.portable.api.PortableReader;
 import org.jetbrains.annotations.NotNull;
 import org.jetbrains.annotations.Nullable;
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/71379a80/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 7259cc9..ccc1a5b 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
@@ -35,7 +35,7 @@ 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.apache.ignite.internal.portable.api.PortableObject;
 import org.jetbrains.annotations.Nullable;
 import org.jsr166.ConcurrentHashMap8;
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/71379a80/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 3152c4b..1d5ca60 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
@@ -18,12 +18,8 @@
 package org.apache.ignite.internal.portable;
 
 import java.io.IOException;
-import java.io.ObjectInputStream;
 import java.io.ObjectOutput;
-import java.io.ObjectOutputStream;
 import java.lang.reflect.InvocationTargetException;
-import java.lang.reflect.Method;
-import java.lang.reflect.Modifier;
 import java.math.BigDecimal;
 import java.math.BigInteger;
 import java.sql.Timestamp;
@@ -32,14 +28,13 @@ import java.util.Date;
 import java.util.IdentityHashMap;
 import java.util.Map;
 import java.util.UUID;
-import java.util.concurrent.ConcurrentHashMap;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.internal.portable.streams.PortableHeapOutputStream;
 import org.apache.ignite.internal.portable.streams.PortableOutputStream;
 import org.apache.ignite.internal.util.typedef.internal.A;
-import org.apache.ignite.portable.PortableException;
-import org.apache.ignite.portable.PortableRawWriter;
-import org.apache.ignite.portable.PortableWriter;
+import org.apache.ignite.internal.portable.api.PortableException;
+import org.apache.ignite.internal.portable.api.PortableRawWriter;
+import org.apache.ignite.internal.portable.api.PortableWriter;
 import org.jetbrains.annotations.Nullable;
 
 import static java.nio.charset.StandardCharsets.UTF_8;

http://git-wip-us.apache.org/repos/asf/ignite/blob/71379a80/modules/core/src/main/java/org/apache/ignite/internal/portable/api/IgnitePortables.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/api/IgnitePortables.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/api/IgnitePortables.java
new file mode 100644
index 0000000..56f3768
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/portable/api/IgnitePortables.java
@@ -0,0 +1,362 @@
+/*
+ * 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.api;
+
+import java.sql.Timestamp;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Date;
+import java.util.HashMap;
+import java.util.LinkedList;
+import java.util.TreeMap;
+import java.util.UUID;
+import org.apache.ignite.IgniteCache;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Defines portable objects functionality. With portable objects you are able to:
+ * <ul>
+ * <li>Seamlessly interoperate between Java, .NET, and C++.</li>
+ * <li>Make any object portable with zero code change to your existing code.</li>
+ * <li>Nest portable objects within each other.</li>
+ * <li>Automatically handle {@code circular} or {@code null} references.</li>
+ * <li>Automatically convert collections and maps between Java, .NET, and C++.</li>
+ * <li>
+ *      Optionally avoid deserialization of objects on the server side
+ *      (objects are stored in {@link PortableObject} format).
+ * </li>
+ * <li>Avoid need to have concrete class definitions on the server side.</li>
+ * <li>Dynamically change structure of the classes without having to restart the cluster.</li>
+ * <li>Index into portable objects for querying purposes.</li>
+ * </ul>
+ * <h1 class="header">Working With Portables Directly</h1>
+ * Once an object is defined as portable,
+ * Ignite will always store it in memory in the portable (i.e. binary) format.
+ * User can choose to work either with the portable format or with the deserialized form
+ * (assuming that class definitions are present in the classpath).
+ * <p>
+ * To work with the portable format directly, user should create a special cache projection
+ * using {@link IgniteCache#withKeepPortable()} method and then retrieve individual fields as needed:
+ * <pre name=code class=java>
+ * IgniteCache&lt;PortableObject, PortableObject&gt; prj = cache.withKeepPortable();
+ *
+ * // Convert instance of MyKey to portable format.
+ * // We could also use PortableBuilder to create the key in portable format directly.
+ * PortableObject key = grid.portables().toPortable(new MyKey());
+ *
+ * PortableObject val = prj.get(key);
+ *
+ * String field = val.field("myFieldName");
+ * </pre>
+ * Alternatively, if we have class definitions in the classpath, we may choose to work with deserialized
+ * typed objects at all times. In this case we do incur the deserialization cost. However, if
+ * {@link PortableMarshaller#isKeepDeserialized()} is {@code true} then Ignite will only deserialize on the first access
+ * and will cache the deserialized object, so it does not have to be deserialized again:
+ * <pre name=code class=java>
+ * IgniteCache&lt;MyKey.class, MyValue.class&gt; cache = grid.cache(null);
+ *
+ * MyValue val = cache.get(new MyKey());
+ *
+ * // Normal java getter.
+ * String fieldVal = val.getMyFieldName();
+ * </pre>
+ * If we used, for example, one of the automatically handled portable types for a key, like integer,
+ * and still wanted to work with binary portable format for values, then we would declare cache projection
+ * as follows:
+ * <pre name=code class=java>
+ * IgniteCache&lt;Integer.class, PortableObject&gt; prj = cache.withKeepPortable();
+ * </pre>
+ * <h1 class="header">Automatic Portable Types</h1>
+ * Note that only portable classes are converted to {@link PortableObject} format. Following
+ * classes are never converted (e.g., {@link #toPortable(Object)} method will return original
+ * object, and instances of these classes will be stored in cache without changes):
+ * <ul>
+ *     <li>All primitives (byte, int, ...) and there boxed versions (Byte, Integer, ...)</li>
+ *     <li>Arrays of primitives (byte[], int[], ...)</li>
+ *     <li>{@link String} and array of {@link String}s</li>
+ *     <li>{@link UUID} and array of {@link UUID}s</li>
+ *     <li>{@link Date} and array of {@link Date}s</li>
+ *     <li>{@link Timestamp} and array of {@link Timestamp}s</li>
+ *     <li>Enums and array of enums</li>
+ *     <li>
+ *         Maps, collections and array of objects (but objects inside
+ *         them will still be converted if they are portable)
+ *     </li>
+ * </ul>
+ * <h1 class="header">Working With Maps and Collections</h1>
+ * All maps and collections in the portable objects are serialized automatically. When working
+ * with different platforms, e.g. C++ or .NET, Ignite will automatically pick the most
+ * adequate collection or map in either language. For example, {@link ArrayList} in Java will become
+ * {@code List} in C#, {@link LinkedList} in Java is {@link LinkedList} in C#, {@link HashMap}
+ * in Java is {@code Dictionary} in C#, and {@link TreeMap} in Java becomes {@code SortedDictionary}
+ * in C#, etc.
+ * <h1 class="header">Building Portable Objects</h1>
+ * Ignite comes with {@link PortableBuilder} which allows to build portable objects dynamically:
+ * <pre name=code class=java>
+ * PortableBuilder builder = Ignition.ignite().portables().builder();
+ *
+ * builder.typeId("MyObject");
+ *
+ * builder.stringField("fieldA", "A");
+ * build.intField("fieldB", "B");
+ *
+ * PortableObject portableObj = builder.build();
+ * </pre>
+ * For the cases when class definition is present
+ * in the class path, it is also possible to populate a standard POJO and then
+ * convert it to portable format, like so:
+ * <pre name=code class=java>
+ * MyObject obj = new MyObject();
+ *
+ * obj.setFieldA("A");
+ * obj.setFieldB(123);
+ *
+ * PortableObject portableObj = Ignition.ignite().portables().toPortable(obj);
+ * </pre>
+ * NOTE: you don't need to convert typed objects to portable format before storing
+ * them in cache, Ignite will do that automatically.
+ * <h1 class="header">Portable Metadata</h1>
+ * Even though Ignite portable protocol only works with hash codes for type and field names
+ * to achieve better performance, Ignite provides metadata for all portable types which
+ * can be queried ar runtime via any of the {@link IgnitePortables#metadata(Class)}
+ * methods. Having metadata also allows for proper formatting of {@code PortableObject#toString()} method,
+ * even when portable objects are kept in binary format only, which may be necessary for audit reasons.
+ * <h1 class="header">Dynamic Structure Changes</h1>
+ * Since objects are always cached in the portable binary format, server does not need to
+ * be aware of the class definitions. Moreover, if class definitions are not present or not
+ * used on the server, then clients can continuously change the structure of the portable
+ * objects without having to restart the cluster. For example, if one client stores a
+ * certain class with fields A and B, and another client stores the same class with
+ * fields B and C, then the server-side portable object will have the fields A, B, and C.
+ * As the structure of a portable object changes, the new fields become available for SQL queries
+ * automatically.
+ * <h1 class="header">Configuration</h1>
+ * By default all your objects are considered as portables and no specific configuration is needed.
+ * However, in some cases, like when an object is used by both Java and .Net, you may need to specify portable objects
+ * explicitly by calling {@link PortableMarshaller#setClassNames(Collection)}.
+ * The only requirement Ignite imposes is that your object has an empty
+ * constructor. Note, that since server side does not have to know the class definition,
+ * you only need to list portable objects in configuration on the client side. However, if you
+ * list them on the server side as well, then you get the ability to deserialize portable objects
+ * into concrete types on the server as well as on the client.
+ * <p>
+ * Here is an example of portable configuration (note that star (*) notation is supported):
+ * <pre name=code class=xml>
+ * ...
+ * &lt;!-- Explicit portable objects configuration. --&gt;
+ * &lt;property name="marshaller"&gt;
+ *     &lt;bean class="org.apache.ignite.internal.portable.api.PortableMarshaller"&gt;
+ *         &lt;property name="classNames"&gt;
+ *             &lt;list&gt;
+ *                 &lt;value&gt;my.package.for.portable.objects.*&lt;/value&gt;
+ *                 &lt;value&gt;org.apache.ignite.examples.client.portable.Employee&lt;/value&gt;
+ *             &lt;/list&gt;
+ *         &lt;/property&gt;
+ *     &lt;/bean&gt;
+ * &lt;/property&gt;
+ * ...
+ * </pre>
+ * or from code:
+ * <pre name=code class=java>
+ * IgniteConfiguration cfg = new IgniteConfiguration();
+ *
+ * PortableMarshaller marsh = new PortableMarshaller();
+ *
+ * marsh.setClassNames(Arrays.asList(
+ *     Employee.class.getName(),
+ *     Address.class.getName())
+ * );
+ *
+ * cfg.setMarshaller(marsh);
+ * </pre>
+ * You can also specify class name for a portable object via {@link PortableTypeConfiguration}.
+ * Do it in case if you need to override other configuration properties on per-type level, like
+ * ID-mapper, or serializer.
+ * <h1 class="header">Custom Affinity Keys</h1>
+ * Often you need to specify an alternate key (not the cache key) for affinity routing whenever
+ * storing objects in cache. For example, if you are caching {@code Employee} object with
+ * {@code Organization}, and want to colocate employees with organization they work for,
+ * so you can process them together, you need to specify an alternate affinity key.
+ * With portable objects you would have to do it as following:
+ * <pre name=code class=xml>
+ * &lt;property name="marshaller"&gt;
+ *     &lt;bean class="org.gridgain.grid.marshaller.portable.PortableMarshaller"&gt;
+ *         ...
+ *         &lt;property name="typeConfigurations"&gt;
+ *             &lt;list&gt;
+ *                 &lt;bean class="org.apache.ignite.internal.portable.api.PortableTypeConfiguration"&gt;
+ *                     &lt;property name="className" value="org.apache.ignite.examples.client.portable.EmployeeKey"/&gt;
+ *                     &lt;property name="affinityKeyFieldName" value="organizationId"/&gt;
+ *                 &lt;/bean&gt;
+ *             &lt;/list&gt;
+ *         &lt;/property&gt;
+ *         ...
+ *     &lt;/bean&gt;
+ * &lt;/property&gt;
+ * </pre>
+ * <h1 class="header">Serialization</h1>
+ * Serialization and deserialization works out-of-the-box in Ignite. However, you can provide your own custom
+ * serialization logic by optionally implementing {@link PortableMarshalAware} interface, like so:
+ * <pre name=code class=java>
+ * public class Address implements PortableMarshalAware {
+ *     private String street;
+ *     private int zip;
+ *
+ *     // Empty constructor required for portable deserialization.
+ *     public Address() {}
+ *
+ *     &#64;Override public void writePortable(PortableWriter writer) throws PortableException {
+ *         writer.writeString("street", street);
+ *         writer.writeInt("zip", zip);
+ *     }
+ *
+ *     &#64;Override public void readPortable(PortableReader reader) throws PortableException {
+ *         street = reader.readString("street");
+ *         zip = reader.readInt("zip");
+ *     }
+ * }
+ * </pre>
+ * Alternatively, if you cannot change class definitions, you can provide custom serialization
+ * logic in {@link PortableSerializer} either globally in {@link PortableMarshaller} or
+ * for a specific type via {@link PortableTypeConfiguration} instance.
+ * <p>
+ * Similar to java serialization you can use {@code writeReplace()} and {@code readResolve()} methods.
+ * <ul>
+ *     <li>
+ *         {@code readResolve} is defined as follows: {@code ANY-ACCESS-MODIFIER Object readResolve()}.
+ *         It may be used to replace the de-serialized object by another one of your choice.
+ *     </li>
+ *     <li>
+ *          {@code writeReplace} is defined as follows: {@code ANY-ACCESS-MODIFIER Object writeReplace()}. This method
+ *          allows the developer to provide a replacement object that will be serialized instead of the original one.
+ *     </li>
+ * </ul>
+ *
+ * <h1 class="header">Custom ID Mappers</h1>
+ * Ignite implementation uses name hash codes to generate IDs for class names or field names
+ * internally. However, in cases when you want to provide your own ID mapping schema,
+ * you can provide your own {@link PortableIdMapper} implementation.
+ * <p>
+ * ID-mapper may be provided either globally in {@link PortableMarshaller},
+ * or for a specific type via {@link PortableTypeConfiguration} instance.
+ * <h1 class="header">Query Indexing</h1>
+ * Portable objects can be indexed for querying by specifying index fields in
+ * {@link org.apache.ignite.cache.CacheTypeMetadata} inside of specific
+ * {@link org.apache.ignite.configuration.CacheConfiguration} instance,
+ * like so:
+ * <pre name=code class=xml>
+ * ...
+ * &lt;bean class="org.apache.ignite.cache.CacheConfiguration"&gt;
+ *     ...
+ *     &lt;property name="typeMetadata"&gt;
+ *         &lt;list&gt;
+ *             &lt;bean class="CacheTypeMetadata"&gt;
+ *                 &lt;property name="type" value="Employee"/&gt;
+ *
+ *                 &lt;!-- Fields to index in ascending order. --&gt;
+ *                 &lt;property name="ascendingFields"&gt;
+ *                     &lt;map&gt;
+ *                     &lt;entry key="name" value="java.lang.String"/&gt;
+ *
+ *                         &lt;!-- Nested portable objects can also be indexed. --&gt;
+ *                         &lt;entry key="address.zip" value="java.lang.Integer"/&gt;
+ *                     &lt;/map&gt;
+ *                 &lt;/property&gt;
+ *             &lt;/bean&gt;
+ *         &lt;/list&gt;
+ *     &lt;/property&gt;
+ * &lt;/bean&gt;
+ * </pre>
+ */
+public interface IgnitePortables {
+    /**
+     * Gets type ID for given type name.
+     *
+     * @param typeName Type name.
+     * @return Type ID.
+     */
+    public int typeId(String typeName);
+
+    /**
+     * Converts provided object to instance of {@link PortableObject}.
+     *
+     * @param obj Object to convert.
+     * @return Converted object.
+     * @throws PortableException In case of error.
+     */
+    public <T> T toPortable(@Nullable Object obj) throws PortableException;
+
+    /**
+     * Creates new portable builder.
+     *
+     * @param typeId ID of the type.
+     * @return Newly portable builder.
+     */
+    public PortableBuilder builder(int typeId);
+
+    /**
+     * Creates new portable builder.
+     *
+     * @param typeName Type name.
+     * @return Newly portable builder.
+     */
+    public PortableBuilder builder(String typeName);
+
+    /**
+     * Creates portable builder initialized by existing portable object.
+     *
+     * @param portableObj Portable object to initialize builder.
+     * @return Portable builder.
+     */
+    public PortableBuilder builder(PortableObject portableObj);
+
+    /**
+     * Gets metadata for provided class.
+     *
+     * @param cls Class.
+     * @return Metadata.
+     * @throws PortableException In case of error.
+     */
+    @Nullable public PortableMetadata metadata(Class<?> cls) throws PortableException;
+
+    /**
+     * Gets metadata for provided class name.
+     *
+     * @param typeName Type name.
+     * @return Metadata.
+     * @throws PortableException In case of error.
+     */
+    @Nullable public PortableMetadata metadata(String typeName) throws PortableException;
+
+    /**
+     * Gets metadata for provided type ID.
+     *
+     * @param typeId Type ID.
+     * @return Metadata.
+     * @throws PortableException In case of error.
+     */
+    @Nullable public PortableMetadata metadata(int typeId) throws PortableException;
+
+    /**
+     * Gets metadata for all known types.
+     *
+     * @return Metadata.
+     * @throws PortableException In case of error.
+     */
+    public Collection<PortableMetadata> metadata() throws PortableException;
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/71379a80/modules/core/src/main/java/org/apache/ignite/internal/portable/api/PortableBuilder.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/api/PortableBuilder.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/api/PortableBuilder.java
new file mode 100644
index 0000000..c819f46
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/portable/api/PortableBuilder.java
@@ -0,0 +1,136 @@
+/*
+ * 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.api;
+
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Portable object builder. Provides ability to build portable objects dynamically without having class definitions.
+ * <p>
+ * Here is an example of how a portable object can be built dynamically:
+ * <pre name=code class=java>
+ * PortableBuilder builder = Ignition.ignite().portables().builder("org.project.MyObject");
+ *
+ * builder.setField("fieldA", "A");
+ * builder.setField("fieldB", "B");
+ *
+ * PortableObject portableObj = builder.build();
+ * </pre>
+ *
+ * <p>
+ * Also builder can be initialized by existing portable object. This allows changing some fields without affecting
+ * other fields.
+ * <pre name=code class=java>
+ * PortableBuilder builder = Ignition.ignite().portables().builder(person);
+ *
+ * builder.setField("name", "John");
+ *
+ * person = builder.build();
+ * </pre>
+ * </p>
+ *
+ * If you need to modify nested portable object you can get builder for nested object using
+ * {@link #getField(String)}, changes made on nested builder will affect parent object,
+ * for example:
+ *
+ * <pre name=code class=java>
+ * PortableBuilder personBuilder = grid.portables().createBuilder(personPortableObj);
+ * PortableBuilder addressBuilder = personBuilder.setField("address");
+ *
+ * addressBuilder.setField("city", "New York");
+ *
+ * personPortableObj = personBuilder.build();
+ *
+ * // Should be "New York".
+ * String city = personPortableObj.getField("address").getField("city");
+ * </pre>
+ *
+ * @see IgnitePortables#builder(int)
+ * @see IgnitePortables#builder(String)
+ * @see IgnitePortables#builder(PortableObject)
+ */
+public interface PortableBuilder {
+    /**
+     * Returns value assigned to the specified field.
+     * If the value is a portable object instance of {@code GridPortableBuilder} will be returned,
+     * which can be modified.
+     * <p>
+     * Collections and maps returned from this method are modifiable.
+     *
+     * @param name Field name.
+     * @return Filed value.
+     */
+    public <T> T getField(String name);
+
+    /**
+     * Sets field value.
+     *
+     * @param name Field name.
+     * @param val Field value (cannot be {@code null}).
+     * @see PortableObject#metaData()
+     */
+    public PortableBuilder setField(String name, Object val);
+
+    /**
+     * Sets field value with value type specification.
+     * <p>
+     * Field type is needed for proper metadata update.
+     *
+     * @param name Field name.
+     * @param val Field value.
+     * @param type Field type.
+     * @see PortableObject#metaData()
+     */
+    public <T> PortableBuilder setField(String name, @Nullable T val, Class<? super T> type);
+
+    /**
+     * Sets field value.
+     * <p>
+     * This method should be used if field is portable object.
+     *
+     * @param name Field name.
+     * @param builder Builder for object field.
+     */
+    public PortableBuilder setField(String name, @Nullable PortableBuilder builder);
+
+    /**
+     * Removes field from this builder.
+     *
+     * @param fieldName Field name.
+     * @return {@code this} instance for chaining.
+     */
+    public PortableBuilder removeField(String fieldName);
+
+    /**
+     * Sets hash code for resulting portable object returned by {@link #build()} method.
+     * <p>
+     * If not set {@code 0} is used.
+     *
+     * @param hashCode Hash code.
+     * @return {@code this} instance for chaining.
+     */
+    public PortableBuilder hashCode(int hashCode);
+
+    /**
+     * Builds portable object.
+     *
+     * @return Portable object.
+     * @throws PortableException In case of error.
+     */
+    public PortableObject build() throws PortableException;
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/71379a80/modules/core/src/main/java/org/apache/ignite/internal/portable/api/PortableException.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/api/PortableException.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/api/PortableException.java
new file mode 100644
index 0000000..f230182
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/portable/api/PortableException.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.api;
+
+import org.apache.ignite.IgniteException;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Exception indicating portable object serialization error.
+ */
+public class PortableException extends IgniteException {
+    /** */
+    private static final long serialVersionUID = 0L;
+
+    /**
+     * Creates portable exception with error message.
+     *
+     * @param msg Error message.
+     */
+    public PortableException(String msg) {
+        super(msg);
+    }
+
+    /**
+     * Creates portable exception with {@link Throwable} as a cause.
+     *
+     * @param cause Cause.
+     */
+    public PortableException(Throwable cause) {
+        super(cause);
+    }
+
+    /**
+     * Creates portable exception with error message and {@link Throwable} as a cause.
+     *
+     * @param msg Error message.
+     * @param cause Cause.
+     */
+    public PortableException(String msg, @Nullable Throwable cause) {
+        super(msg, cause);
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/71379a80/modules/core/src/main/java/org/apache/ignite/internal/portable/api/PortableIdMapper.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/api/PortableIdMapper.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/api/PortableIdMapper.java
new file mode 100644
index 0000000..1e20f8e
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/portable/api/PortableIdMapper.java
@@ -0,0 +1,54 @@
+/*
+ * 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.api;
+
+/**
+ * Type and field ID mapper for portable objects. Ignite never writes full
+ * strings for field or type names. Instead, for performance reasons, Ignite
+ * writes integer hash codes for type and field names. It has been tested that
+ * hash code conflicts for the type names or the field names
+ * within the same type are virtually non-existent and, to gain performance, it is safe
+ * to work with hash codes. For the cases when hash codes for different types or fields
+ * actually do collide {@code PortableIdMapper} allows to override the automatically
+ * generated hash code IDs for the type and field names.
+ * <p>
+ * Portable ID mapper can be configured for all portable objects via {@link PortableMarshaller#getIdMapper()} method,
+ * or for a specific portable type via {@link PortableTypeConfiguration#getIdMapper()} method.
+ */
+public interface PortableIdMapper {
+    /**
+     * Gets type ID for provided class name.
+     * <p>
+     * If {@code 0} is returned, hash code of class simple name will be used.
+     *
+     * @param clsName Class name.
+     * @return Type ID.
+     */
+    public int typeId(String clsName);
+
+    /**
+     * Gets ID for provided field.
+     * <p>
+     * If {@code 0} is returned, hash code of field name will be used.
+     *
+     * @param typeId Type ID.
+     * @param fieldName Field name.
+     * @return Field ID.
+     */
+    public int fieldId(int typeId, String fieldName);
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/71379a80/modules/core/src/main/java/org/apache/ignite/internal/portable/api/PortableInvalidClassException.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/api/PortableInvalidClassException.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/api/PortableInvalidClassException.java
new file mode 100644
index 0000000..82e6697
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/portable/api/PortableInvalidClassException.java
@@ -0,0 +1,58 @@
+/*
+ * 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.api;
+
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Exception indicating that class needed for deserialization of portable object does not exist.
+ * <p>
+ * Thrown from {@link PortableObject#deserialize()} method.
+ */
+public class PortableInvalidClassException extends PortableException {
+    /** */
+    private static final long serialVersionUID = 0L;
+
+    /**
+     * Creates invalid class exception with error message.
+     *
+     * @param msg Error message.
+     */
+    public PortableInvalidClassException(String msg) {
+        super(msg);
+    }
+
+    /**
+     * Creates invalid class exception with {@link Throwable} as a cause.
+     *
+     * @param cause Cause.
+     */
+    public PortableInvalidClassException(Throwable cause) {
+        super(cause);
+    }
+
+    /**
+     * Creates invalid class exception with error message and {@link Throwable} as a cause.
+     *
+     * @param msg Error message.
+     * @param cause Cause.
+     */
+    public PortableInvalidClassException(String msg, @Nullable Throwable cause) {
+        super(msg, cause);
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/71379a80/modules/core/src/main/java/org/apache/ignite/internal/portable/api/PortableMarshalAware.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/api/PortableMarshalAware.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/api/PortableMarshalAware.java
new file mode 100644
index 0000000..f304afb
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/portable/api/PortableMarshalAware.java
@@ -0,0 +1,48 @@
+/*
+ * 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.api;
+
+/**
+ * Interface that allows to implement custom serialization
+ * logic for portable objects. Portable objects are not required
+ * to implement this interface, in which case Ignite will automatically
+ * serialize portable objects using reflection.
+ * <p>
+ * This interface, in a way, is analogous to {@link java.io.Externalizable}
+ * interface, which allows users to override default serialization logic,
+ * usually for performance reasons. The only difference here is that portable
+ * serialization is already very fast and implementing custom serialization
+ * logic for portables does not provide significant performance gains.
+ */
+public interface PortableMarshalAware {
+    /**
+     * Writes fields to provided writer.
+     *
+     * @param writer Portable object writer.
+     * @throws PortableException In case of error.
+     */
+    public void writePortable(PortableWriter writer) throws PortableException;
+
+    /**
+     * Reads fields from provided reader.
+     *
+     * @param reader Portable object reader.
+     * @throws PortableException In case of error.
+     */
+    public void readPortable(PortableReader reader) throws PortableException;
+}
\ No newline at end of file


[14/55] [abbrv] ignite git commit: IGNITE-1090 - Fixed backup check for one-phase commit transaction.

Posted by ag...@apache.org.
IGNITE-1090 - Fixed backup check for one-phase commit transaction.


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

Branch: refs/heads/ignite-1171
Commit: 06fdd7d44dda36900b4c7a76dec2a7848ca9e8fb
Parents: e5f1681
Author: Alexey Goncharuk <al...@gmail.com>
Authored: Mon Sep 14 13:35:41 2015 -0700
Committer: Alexey Goncharuk <al...@gmail.com>
Committed: Mon Sep 14 13:35:41 2015 -0700

----------------------------------------------------------------------
 .../near/GridNearTxFinishFuture.java            | 20 ++++++++++++--------
 1 file changed, 12 insertions(+), 8 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/06fdd7d4/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxFinishFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxFinishFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxFinishFuture.java
index ddc8be5..21aaef2 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxFinishFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxFinishFuture.java
@@ -396,20 +396,24 @@ public final class GridNearTxFinishFuture<K, V> extends GridCompoundIdentityFutu
 
                 ClusterNode backup = cctx.discovery().node(backupId);
 
-                // Nothing to do if backup has left the grid.
-                if (backup == null)
-                    return;
-
                 MiniFuture mini = new MiniFuture(backup, mapping);
 
                 add(mini);
 
-                if (backup.isLocal()) {
-                    if (cctx.tm().txHandler().checkDhtRemoteTxCommitted(tx.xidVersion())) {
-                        readyNearMappingFromBackup(mapping);
+                // Nothing to do if backup has left the grid.
+                if (backup == null) {
+                    readyNearMappingFromBackup(mapping);
 
+                    mini.onDone(new IgniteTxRollbackCheckedException("Failed to commit transaction " +
+                        "(backup has left grid): " + tx.xidVersion()));
+                }
+                else if (backup.isLocal()) {
+                    boolean committed = cctx.tm().txHandler().checkDhtRemoteTxCommitted(tx.xidVersion());
+
+                    readyNearMappingFromBackup(mapping);
+
+                    if (committed)
                         mini.onDone(tx);
-                    }
                     else
                         mini.onDone(new IgniteTxRollbackCheckedException("Failed to commit transaction " +
                             "(transaction has been rolled back on backup node): " + tx.xidVersion()));


[30/55] [abbrv] ignite git commit: ignite-971 Fixed offheap to swap eviction, added failover tests with swap/offheap, added retries for tx 'check backup' rollback.

Posted by ag...@apache.org.
ignite-971 Fixed offheap to swap eviction, added failover tests with swap/offheap, added retries for tx 'check backup' rollback.


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

Branch: refs/heads/ignite-1171
Commit: a7490a6e48d4c9f1e85a3ae08f97d6c5ced7a71b
Parents: eb7d2b0
Author: sboikov <sb...@gridgain.com>
Authored: Tue Sep 15 12:46:48 2015 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Tue Sep 15 12:46:48 2015 +0300

----------------------------------------------------------------------
 .../processors/cache/CacheMetricsImpl.java      |   2 +-
 .../processors/cache/GridCacheAdapter.java      |  21 +-
 .../processors/cache/GridCacheEntryEx.java      |  13 ++
 .../cache/GridCacheEvictionManager.java         |  18 +-
 .../processors/cache/GridCacheMapEntry.java     |  52 ++++-
 .../cache/GridCacheSwapEntryImpl.java           |  24 +--
 .../processors/cache/GridCacheSwapManager.java  | 190 +++++++++++++++---
 .../processors/cache/GridCacheUtils.java        |  21 +-
 .../distributed/dht/GridDhtLocalPartition.java  |   7 +-
 .../dht/GridDhtPartitionTopologyImpl.java       |   9 +-
 .../near/GridNearOptimisticTxPrepareFuture.java |   3 +
 .../near/GridNearTxFinishFuture.java            |  32 ++-
 .../cache/transactions/IgniteTxHandler.java     |   9 +-
 .../offheap/GridOffHeapProcessor.java           |  25 ++-
 .../util/offheap/GridOffHeapEvictListener.java  |   5 +
 .../internal/util/offheap/GridOffHeapMap.java   |  13 +-
 .../util/offheap/GridOffHeapMapFactory.java     |  28 +--
 .../util/offheap/GridOffHeapPartitionedMap.java |  12 ++
 .../util/offheap/unsafe/GridUnsafeMap.java      | 128 ++++++++----
 .../unsafe/GridUnsafePartitionedMap.java        |   9 +
 .../cache/CacheSwapUnswapGetTest.java           |  85 +++++++-
 .../cache/GridCacheAbstractFullApiSelfTest.java |   7 +-
 .../GridCacheAbstractRemoveFailureTest.java     | 199 +++++++++++++------
 .../cache/GridCacheMemoryModeSelfTest.java      |   9 +-
 .../cache/GridCachePutAllFailoverSelfTest.java  |  60 ++++++
 .../processors/cache/GridCacheTestEntryEx.java  |   6 +
 .../GridCacheAbstractNodeRestartSelfTest.java   | 149 ++++++++++++--
 .../GridCacheDhtAtomicRemoveFailureTest.java    |  16 +-
 .../dht/GridCacheDhtRemoveFailureTest.java      |  16 +-
 .../dht/GridCacheTxNodeFailureSelfTest.java     |   2 +-
 .../IgniteCacheCrossCacheTxFailoverTest.java    |  53 +++--
 .../IgniteCachePutRetryAbstractSelfTest.java    | 166 ++++++++++++++--
 .../dht/IgniteCachePutRetryAtomicSelfTest.java  |   2 +
 ...gniteCachePutRetryTransactionalSelfTest.java |  50 +++--
 ...eAtomicInvalidPartitionHandlingSelfTest.java |  98 +++++++--
 ...tomicPrimaryWriteOrderRemoveFailureTest.java |  15 +-
 .../GridCacheAtomicRemoveFailureTest.java       |  15 +-
 .../GridCacheAtomicNearRemoveFailureTest.java   |  15 +-
 ...cPrimaryWriteOrderNearRemoveFailureTest.java |  15 +-
 .../near/GridCacheNearRemoveFailureTest.java    |  15 +-
 .../GridCachePartitionedNodeRestartTest.java    |   9 +-
 ...ePartitionedOptimisticTxNodeRestartTest.java |   9 +-
 .../GridCacheReplicatedNodeRestartSelfTest.java |   8 +-
 .../offheap/GridOffHeapMapAbstractSelfTest.java |  16 +-
 .../GridOffHeapMapPerformanceAbstractTest.java  |   4 +-
 ...idOffHeapPartitionedMapAbstractSelfTest.java |  20 ++
 .../unsafe/GridUnsafeMapPerformanceTest.java    |   2 +-
 .../offheap/unsafe/GridUnsafeMapSelfTest.java   |   2 +-
 .../GridOffHeapMapPerformanceAbstractTest.java  |   4 +-
 .../unsafe/GridUnsafeMapPerformanceTest.java    |   2 +-
 .../ignite/testframework/GridTestUtils.java     | 117 +++++++++++
 .../IgniteCacheFailoverTestSuite.java           |   6 -
 .../IgniteCacheFailoverTestSuite3.java          |  62 ++++++
 .../testsuites/IgniteCacheRestartTestSuite.java |  14 +-
 .../IgniteCacheRestartTestSuite2.java           |  47 +++++
 .../query/h2/opt/GridH2AbstractKeyValueRow.java |  31 ++-
 .../cache/GridCacheOffHeapAndSwapSelfTest.java  |   4 +
 .../IgniteCacheQueryMultiThreadedSelfTest.java  |   9 +-
 ...QueryOffheapEvictsMultiThreadedSelfTest.java |   5 -
 ...lientQueryReplicatedNodeRestartSelfTest.java |   8 +-
 .../IgniteCacheQueryNodeRestartSelfTest.java    |   4 +-
 .../IgniteCacheQueryNodeRestartSelfTest2.java   |   8 +-
 62 files changed, 1584 insertions(+), 421 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/a7490a6e/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheMetricsImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheMetricsImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheMetricsImpl.java
index 1554e07..dfa0217 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheMetricsImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheMetricsImpl.java
@@ -843,7 +843,7 @@ public class CacheMetricsImpl implements CacheMetrics {
         offHeapEvicts.incrementAndGet();
 
         if (delegate != null)
-            delegate.onOffHeapRemove();
+            delegate.onOffHeapEvict();
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/a7490a6e/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java
index 9329e94..1fc94ec 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java
@@ -4096,21 +4096,22 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
 
                     return t;
                 }
-                catch (IgniteInterruptedCheckedException | IgniteTxHeuristicCheckedException |
-                    IgniteTxRollbackCheckedException e) {
+                catch (IgniteInterruptedCheckedException | IgniteTxHeuristicCheckedException e) {
                     throw e;
                 }
                 catch (IgniteCheckedException e) {
-                    try {
-                        tx.rollback();
+                    if (!(e instanceof IgniteTxRollbackCheckedException)) {
+                        try {
+                            tx.rollback();
 
-                        e = new IgniteTxRollbackCheckedException("Transaction has been rolled back: " +
-                            tx.xid(), e);
-                    }
-                    catch (IgniteCheckedException | AssertionError | RuntimeException e1) {
-                        U.error(log, "Failed to rollback transaction (cache may contain stale locks): " + tx, e1);
+                            e = new IgniteTxRollbackCheckedException("Transaction has been rolled back: " +
+                                tx.xid(), e);
+                        }
+                        catch (IgniteCheckedException | AssertionError | RuntimeException e1) {
+                            U.error(log, "Failed to rollback transaction (cache may contain stale locks): " + tx, e1);
 
-                        U.addLastCause(e, e1, log);
+                            U.addLastCause(e, e1, log);
+                        }
                     }
 
                     if (X.hasCause(e, ClusterTopologyCheckedException.class) && i != retries - 1) {

http://git-wip-us.apache.org/repos/asf/ignite/blob/a7490a6e/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheEntryEx.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheEntryEx.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheEntryEx.java
index 98e86ed..430590a 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheEntryEx.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheEntryEx.java
@@ -867,6 +867,19 @@ public interface GridCacheEntryEx {
     public void updateTtl(@Nullable GridCacheVersion ver, long ttl);
 
     /**
+     * Tries to do offheap -> swap eviction.
+     *
+     * @param entry Serialized swap entry.
+     * @param evictVer Version when entry was selected for eviction.
+     * @param obsoleteVer Obsolete version.
+     * @throws IgniteCheckedException If failed.
+     * @throws GridCacheEntryRemovedException If entry was removed.
+     * @return {@code True} if entry was obsoleted and written to swap.
+     */
+    public boolean offheapSwapEvict(byte[] entry, GridCacheVersion evictVer, GridCacheVersion obsoleteVer)
+        throws IgniteCheckedException, GridCacheEntryRemovedException;
+
+    /**
      * @return Value.
      * @throws IgniteCheckedException If failed to read from swap storage.
      * @throws GridCacheEntryRemovedException If entry was removed.

http://git-wip-us.apache.org/repos/asf/ignite/blob/a7490a6e/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheEvictionManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheEvictionManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheEvictionManager.java
index f60c0eb..3e0e2f9 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheEvictionManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheEvictionManager.java
@@ -958,7 +958,7 @@ public class GridCacheEvictionManager extends GridCacheManagerAdapter {
 
         List<GridCacheEntryEx> locked = new ArrayList<>(keys.size());
 
-        Set<GridCacheEntryEx> notRemove = null;
+        Set<GridCacheEntryEx> notRmv = null;
 
         Collection<GridCacheBatchSwapEntry> swapped = new ArrayList<>(keys.size());
 
@@ -990,10 +990,10 @@ public class GridCacheEvictionManager extends GridCacheManagerAdapter {
                 locked.add(entry);
 
                 if (entry.obsolete()) {
-                    if (notRemove == null)
-                        notRemove = new HashSet<>();
+                    if (notRmv == null)
+                        notRmv = new HashSet<>();
 
-                    notRemove.add(entry);
+                    notRmv.add(entry);
 
                     continue;
                 }
@@ -1004,11 +1004,19 @@ public class GridCacheEvictionManager extends GridCacheManagerAdapter {
                 GridCacheBatchSwapEntry swapEntry = entry.evictInBatchInternal(obsoleteVer);
 
                 if (swapEntry != null) {
+                    assert entry.obsolete() : entry;
+
                     swapped.add(swapEntry);
 
                     if (log.isDebugEnabled())
                         log.debug("Entry was evicted [entry=" + entry + ", localNode=" + cctx.nodeId() + ']');
                 }
+                else if (!entry.obsolete()) {
+                    if (notRmv == null)
+                        notRmv = new HashSet<>();
+
+                    notRmv.add(entry);
+                }
             }
 
             // Batch write to swap.
@@ -1025,7 +1033,7 @@ public class GridCacheEvictionManager extends GridCacheManagerAdapter {
 
             // Remove entries and fire events outside the locks.
             for (GridCacheEntryEx entry : locked) {
-                if (entry.obsolete() && (notRemove == null || !notRemove.contains(entry))) {
+                if (entry.obsolete() && (notRmv == null || !notRmv.contains(entry))) {
                     entry.onMarkedObsolete();
 
                     cache.removeEntry(entry);

http://git-wip-us.apache.org/repos/asf/ignite/blob/a7490a6e/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMapEntry.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMapEntry.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMapEntry.java
index eb4d864..f2bb646 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMapEntry.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMapEntry.java
@@ -433,6 +433,41 @@ public abstract class GridCacheMapEntry extends GridMetadataAwareAdapter impleme
     }
 
     /** {@inheritDoc} */
+    @Override public boolean offheapSwapEvict(byte[] entry, GridCacheVersion evictVer, GridCacheVersion obsoleteVer)
+        throws IgniteCheckedException, GridCacheEntryRemovedException {
+        assert cctx.swap().swapEnabled() && cctx.swap().offHeapEnabled() : this;
+
+        boolean obsolete;
+
+        synchronized (this) {
+            checkObsolete();
+
+            if (hasReaders() || !isStartVersion())
+                return false;
+
+            GridCacheMvcc mvcc = mvccExtras();
+
+            if (mvcc != null && !mvcc.isEmpty(obsoleteVer))
+                return false;
+
+            if (cctx.swap().offheapSwapEvict(key, entry, partition(), evictVer)) {
+                assert !hasValueUnlocked() : this;
+
+                obsolete = markObsolete0(obsoleteVer, false);
+
+                assert obsolete : this;
+            }
+            else
+                obsolete = false;
+        }
+
+        if (obsolete)
+            onMarkedObsolete();
+
+        return obsolete;
+    }
+
+    /** {@inheritDoc} */
     @Override public CacheObject unswap() throws IgniteCheckedException, GridCacheEntryRemovedException {
         return unswap(true);
     }
@@ -536,7 +571,7 @@ public abstract class GridCacheMapEntry extends GridMetadataAwareAdapter impleme
                     log.debug("Value did not change, skip write swap entry: " + this);
 
                 if (cctx.swap().offheapEvictionEnabled())
-                    cctx.swap().enableOffheapEviction(key());
+                    cctx.swap().enableOffheapEviction(key(), partition());
 
                 return;
             }
@@ -2988,7 +3023,7 @@ public abstract class GridCacheMapEntry extends GridMetadataAwareAdapter impleme
         synchronized (this) {
             checkObsolete();
 
-            if (isNew() || (!preload && deletedUnlocked())) {
+            if ((isNew() && !cctx.swap().containsKey(key, partition())) || (!preload && deletedUnlocked())) {
                 long expTime = expireTime < 0 ? CU.toExpireTime(ttl) : expireTime;
 
                 val = cctx.kernalContext().cacheObjects().prepareForCache(val, cctx);
@@ -3643,6 +3678,9 @@ public abstract class GridCacheMapEntry extends GridMetadataAwareAdapter impleme
         try {
             if (F.isEmptyOrNulls(filter)) {
                 synchronized (this) {
+                    if (obsoleteVersionExtras() != null)
+                        return true;
+
                     CacheObject prev = saveValueForIndexUnlocked();
 
                     if (!hasReaders() && markObsolete0(obsoleteVer, false)) {
@@ -3684,6 +3722,9 @@ public abstract class GridCacheMapEntry extends GridMetadataAwareAdapter impleme
                         return false;
 
                     synchronized (this) {
+                        if (obsoleteVersionExtras() != null)
+                            return true;
+
                         if (!v.equals(ver))
                             // Version has changed since entry passed the filter. Do it again.
                             continue;
@@ -3768,6 +3809,13 @@ public abstract class GridCacheMapEntry extends GridMetadataAwareAdapter impleme
         try {
             if (!hasReaders() && markObsolete0(obsoleteVer, false)) {
                 if (!isStartVersion() && hasValueUnlocked()) {
+                    if (cctx.offheapTiered() && hasOffHeapPointer()) {
+                        if (cctx.swap().offheapEvictionEnabled())
+                            cctx.swap().enableOffheapEviction(key(), partition());
+
+                        return null;
+                    }
+
                     IgniteUuid valClsLdrId = null;
                     IgniteUuid keyClsLdrId = null;
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/a7490a6e/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheSwapEntryImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheSwapEntryImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheSwapEntryImpl.java
index 81490a7..b7c66d3 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheSwapEntryImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheSwapEntryImpl.java
@@ -126,9 +126,9 @@ public class GridCacheSwapEntryImpl implements GridCacheSwapEntry {
      * @return Version.
      */
     public static GridCacheVersion version(byte[] bytes) {
-        int off = VERSION_OFFSET; // Skip ttl, expire time.
+        long off = BYTE_ARR_OFF + VERSION_OFFSET; // Skip ttl, expire time.
 
-        boolean verEx = bytes[off++] != 0;
+        boolean verEx = UNSAFE.getByte(bytes, off++) != 0;
 
         return U.readVersion(bytes, off, verEx);
     }
@@ -157,26 +157,6 @@ public class GridCacheSwapEntryImpl implements GridCacheSwapEntry {
         return new IgniteBiTuple<>(valBytes, type);
     }
 
-    /**
-     * @param bytes Entry bytes.
-     * @return Value bytes offset.
-     */
-    public static int valueOffset(byte[] bytes) {
-        assert bytes.length > 40 : bytes.length;
-
-        int off = VERSION_OFFSET; // Skip ttl, expire time.
-
-        boolean verEx = bytes[off++] != 0;
-
-        off += verEx ? VERSION_EX_SIZE : VERSION_SIZE;
-
-        off += 5; // Byte array flag + array size.
-
-        assert bytes.length >= off;
-
-        return off;
-    }
-
     /** {@inheritDoc} */
     @Override public byte[] valueBytes() {
         if (valBytes != null) {

http://git-wip-us.apache.org/repos/asf/ignite/blob/a7490a6e/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheSwapManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheSwapManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheSwapManager.java
index 7fd6013..9b6381e 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheSwapManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheSwapManager.java
@@ -34,6 +34,7 @@ import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.IgniteException;
 import org.apache.ignite.internal.managers.swapspace.GridSwapSpaceManager;
 import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
+import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtInvalidPartitionException;
 import org.apache.ignite.internal.processors.cache.query.GridCacheQueryManager;
 import org.apache.ignite.internal.processors.cache.version.GridCacheVersion;
 import org.apache.ignite.internal.processors.cache.version.GridCacheVersionAware;
@@ -54,6 +55,7 @@ import org.apache.ignite.internal.util.typedef.T2;
 import org.apache.ignite.internal.util.typedef.internal.CU;
 import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.lang.IgniteBiInClosure;
+import org.apache.ignite.lang.IgniteBiPredicate;
 import org.apache.ignite.lang.IgniteBiTuple;
 import org.apache.ignite.lang.IgniteUuid;
 import org.apache.ignite.spi.swapspace.SwapKey;
@@ -101,8 +103,16 @@ public class GridCacheSwapManager extends GridCacheManagerAdapter {
     private final ReferenceQueue<Iterator<Map.Entry>> itQ = new ReferenceQueue<>();
 
     /** Soft iterator set. */
-    private final Collection<GridWeakIterator<Map.Entry>> itSet =
-        new GridConcurrentHashSet<>();
+    private final Collection<GridWeakIterator<Map.Entry>> itSet = new GridConcurrentHashSet<>();
+
+    /** {@code True} if offheap to swap eviction is possible. */
+    private boolean offheapToSwapEvicts;
+
+    /** Values to be evicted from offheap to swap. */
+    private ThreadLocal<Collection<IgniteBiTuple<byte[], byte[]>>> offheapEvicts = new ThreadLocal<>();
+
+    /** First offheap eviction warning flag. */
+    private volatile boolean firstEvictWarn;
 
     /**
      * @param enabled Flag to indicate if swap is enabled.
@@ -127,9 +137,58 @@ public class GridCacheSwapManager extends GridCacheManagerAdapter {
     }
 
     /**
+     *
+     */
+    public void unwindOffheapEvicts() {
+        if (!offheapToSwapEvicts)
+            return;
+
+        Collection<IgniteBiTuple<byte[], byte[]>> evicts = offheapEvicts.get();
+
+        if (evicts != null) {
+            GridCacheVersion obsoleteVer = cctx.versions().next();
+
+            for (IgniteBiTuple<byte[], byte[]> t : evicts) {
+                try {
+                    byte[] kb = t.get1();
+                    byte[] vb = t.get2();
+
+                    GridCacheVersion evictVer = GridCacheSwapEntryImpl.version(vb);
+
+                    KeyCacheObject key = cctx.toCacheKeyObject(kb);
+
+                    while (true) {
+                        GridCacheEntryEx entry = cctx.cache().entryEx(key);
+
+                        try {
+                            if (entry.offheapSwapEvict(vb, evictVer, obsoleteVer))
+                                cctx.cache().removeEntry(entry);
+
+                            break;
+                        }
+                        catch (GridCacheEntryRemovedException ignore) {
+                            // Retry.
+                        }
+                    }
+                }
+                catch (GridDhtInvalidPartitionException e) {
+                    // Skip entry.
+                }
+                catch (IgniteCheckedException e) {
+                    U.error(log, "Failed to unmarshal off-heap entry", e);
+                }
+            }
+
+            offheapEvicts.set(null);
+        }
+    }
+
+    /**
      * Initializes off-heap space.
      */
     private void initOffHeap() {
+        assert offheapEnabled;
+
         // Register big data usage.
         long max = cctx.config().getOffHeapMaxMemory();
 
@@ -137,43 +196,69 @@ public class GridCacheSwapManager extends GridCacheManagerAdapter {
 
         int parts = cctx.config().getAffinity().partitions();
 
-        GridOffHeapEvictListener lsnr = !swapEnabled && !offheapEnabled ? null : new GridOffHeapEvictListener() {
-            private volatile boolean firstEvictWarn;
+        GridOffHeapEvictListener lsnr;
 
-            @Override public void onEvict(int part, int hash, byte[] kb, byte[] vb) {
-                try {
-                    if (!firstEvictWarn)
-                        warnFirstEvict();
+        if (swapEnabled) {
+            offheapToSwapEvicts = true;
 
-                    writeToSwap(part, cctx.toCacheKeyObject(kb), vb);
+            lsnr = new GridOffHeapEvictListener() {
+                @Override public void onEvict(int part, int hash, byte[] kb, byte[] vb) {
+                    assert offheapToSwapEvicts;
 
-                    if (cctx.config().isStatisticsEnabled())
-                        cctx.cache().metrics0().onOffHeapEvict();
-                }
-                catch (IgniteCheckedException e) {
-                    U.error(log, "Failed to unmarshal off-heap entry [part=" + part + ", hash=" + hash + ']', e);
-                }
-            }
+                    onOffheapEvict();
 
-            private void warnFirstEvict() {
-                synchronized (this) {
-                    if (firstEvictWarn)
-                        return;
+                    Collection<IgniteBiTuple<byte[], byte[]>> evicts = offheapEvicts.get();
 
-                    firstEvictWarn = true;
+                    if (evicts == null)
+                        offheapEvicts.set(evicts = new ArrayList<>());
+
+                    evicts.add(new IgniteBiTuple<>(kb, vb));
                 }
 
-                U.warn(log, "Off-heap evictions started. You may wish to increase 'offHeapMaxMemory' in " +
-                    "cache configuration [cache=" + cctx.name() +
-                    ", offHeapMaxMemory=" + cctx.config().getOffHeapMaxMemory() + ']',
-                    "Off-heap evictions started: " + cctx.name());
-            }
-        };
+                @Override public boolean removeEvicted() {
+                    return false;
+                }
+            };
+        }
+        else {
+            lsnr = new GridOffHeapEvictListener() {
+                @Override public void onEvict(int part, int hash, byte[] kb, byte[] vb) {
+                    onOffheapEvict();
+                }
+
+                @Override public boolean removeEvicted() {
+                    return true;
+                }
+            };
+        }
 
         offheap.create(spaceName, parts, init, max, lsnr);
     }
 
     /**
+     * Warns on first evict from off-heap.
+     */
+    private void onOffheapEvict() {
+        if (cctx.config().isStatisticsEnabled())
+            cctx.cache().metrics0().onOffHeapEvict();
+
+        if (firstEvictWarn)
+            return;
+
+        synchronized (this) {
+            if (firstEvictWarn)
+                return;
+
+            firstEvictWarn = true;
+        }
+
+        U.warn(log, "Off-heap evictions started. You may wish to increase 'offHeapMaxMemory' in " +
+            "cache configuration [cache=" + cctx.name() +
+            ", offHeapMaxMemory=" + cctx.config().getOffHeapMaxMemory() + ']',
+            "Off-heap evictions started: " + cctx.name());
+    }
+
+    /**
      * @return {@code True} if swap store is enabled.
      */
     public boolean swapEnabled() {
@@ -440,17 +525,16 @@ public class GridCacheSwapManager extends GridCacheManagerAdapter {
 
     /**
      * @param key Key to check.
+     * @param part Partition.
      * @return {@code True} if key is contained.
      * @throws IgniteCheckedException If failed.
      */
-    public boolean containsKey(KeyCacheObject key) throws IgniteCheckedException {
+    public boolean containsKey(KeyCacheObject key, int part) throws IgniteCheckedException {
         if (!offheapEnabled && !swapEnabled)
             return false;
 
         checkIteratorQueue();
 
-        int part = cctx.affinity().partition(key);
-
         // First check off-heap store.
         if (offheapEnabled) {
             boolean contains = offheap.contains(spaceName, part, key, key.valueBytes(cctx.cacheObjectContext()));
@@ -480,6 +564,7 @@ public class GridCacheSwapManager extends GridCacheManagerAdapter {
 
     /**
      * @param key Key to read.
+     * @param keyBytes Key bytes.
      * @param part Key partition.
      * @param entryLocked {@code True} if cache entry is locked.
      * @param readOffheap Read offheap flag.
@@ -966,6 +1051,46 @@ public class GridCacheSwapManager extends GridCacheManagerAdapter {
     }
 
     /**
+     * @param key Key to move from offheap to swap.
+     * @param entry Serialized swap entry.
+     * @param part Partition.
+     * @param ver Expected entry version.
+     * @return {@code True} if removed.
+     * @throws IgniteCheckedException If failed.
+     */
+    boolean offheapSwapEvict(final KeyCacheObject key, byte[] entry, int part, final GridCacheVersion ver)
+        throws IgniteCheckedException {
+        assert offheapEnabled;
+
+        checkIteratorQueue();
+
+        boolean rmv = offheap.removex(spaceName, part, key, key.valueBytes(cctx.cacheObjectContext()),
+            new IgniteBiPredicate<Long, Integer>() {
+                @Override public boolean apply(Long ptr, Integer len) {
+                    GridCacheVersion ver0 = GridCacheOffheapSwapEntry.version(ptr);
+
+                    return ver.equals(ver0);
+                }
+            }
+        );
+
+        if (rmv) {
+            Collection<GridCacheSwapListener> lsnrs = offheapLsnrs.get(part);
+
+            if (lsnrs != null) {
+                GridCacheSwapEntry e = swapEntry(GridCacheSwapEntryImpl.unmarshal(entry));
+
+                for (GridCacheSwapListener lsnr : lsnrs)
+                    lsnr.onEntryUnswapped(part, key, e);
+            }
+
+            cctx.swap().writeToSwap(part, key, entry);
+        }
+
+        return rmv;
+    }
+
+    /**
      * @return {@code True} if offheap eviction is enabled.
      */
     boolean offheapEvictionEnabled() {
@@ -976,16 +1101,15 @@ public class GridCacheSwapManager extends GridCacheManagerAdapter {
      * Enables eviction for offheap entry after {@link #readOffheapPointer} was called.
      *
      * @param key Key.
+     * @param part Partition.
      * @throws IgniteCheckedException If failed.
      */
-    void enableOffheapEviction(final KeyCacheObject key) throws IgniteCheckedException {
+    void enableOffheapEviction(final KeyCacheObject key, int part) throws IgniteCheckedException {
         if (!offheapEnabled)
             return;
 
         checkIteratorQueue();
 
-        int part = cctx.affinity().partition(key);
-
         offheap.enableEviction(spaceName, part, key, key.valueBytes(cctx.cacheObjectContext()));
     }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/a7490a6e/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheUtils.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheUtils.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheUtils.java
index 980971c..2d5698a 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheUtils.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheUtils.java
@@ -1029,8 +1029,15 @@ public class GridCacheUtils {
 
         ctx.evicts().unwind();
 
-        if (ctx.isNear())
-            ctx.near().dht().context().evicts().unwind();
+        ctx.swap().unwindOffheapEvicts();
+
+        if (ctx.isNear()) {
+            GridCacheContext dhtCtx = ctx.near().dht().context();
+
+            dhtCtx.evicts().unwind();
+
+            dhtCtx.swap().unwindOffheapEvicts();
+        }
 
         ctx.ttl().expire();
     }
@@ -1041,14 +1048,8 @@ public class GridCacheUtils {
     public static <K, V> void unwindEvicts(GridCacheSharedContext<K, V> ctx) {
         assert ctx != null;
 
-        for (GridCacheContext<K, V> cacheCtx : ctx.cacheContexts()) {
-            cacheCtx.evicts().unwind();
-
-            if (cacheCtx.isNear())
-                cacheCtx.near().dht().context().evicts().unwind();
-
-            cacheCtx.ttl().expire();
-        }
+        for (GridCacheContext<K, V> cacheCtx : ctx.cacheContexts())
+            unwindEvicts(cacheCtx);
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/a7490a6e/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtLocalPartition.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtLocalPartition.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtLocalPartition.java
index c5f15cd..956f2bf 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtLocalPartition.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtLocalPartition.java
@@ -262,8 +262,11 @@ public class GridDhtLocalPartition implements Comparable<GridDhtLocalPartition>,
 
         map.put(entry.key(), entry);
 
-        if (!entry.isInternal())
+        if (!entry.isInternal()) {
+            assert !entry.deleted() : entry;
+
             mapPubSize.increment();
+        }
     }
 
     /**
@@ -271,7 +274,7 @@ public class GridDhtLocalPartition implements Comparable<GridDhtLocalPartition>,
      */
     @SuppressWarnings("SynchronizationOnLocalVariableOrMethodParameter")
     void onRemoved(GridDhtCacheEntry entry) {
-        assert entry.obsolete();
+        assert entry.obsolete() : entry;
 
         // Make sure to remove exactly this entry.
         synchronized (entry) {

http://git-wip-us.apache.org/repos/asf/ignite/blob/a7490a6e/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtPartitionTopologyImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtPartitionTopologyImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtPartitionTopologyImpl.java
index 5e183e9..fcb012f 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtPartitionTopologyImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtPartitionTopologyImpl.java
@@ -546,10 +546,13 @@ class GridDhtPartitionTopologyImpl implements GridDhtPartitionTopology {
      * @param updateSeq Update sequence.
      * @return Local partition.
      */
-    private GridDhtLocalPartition localPartition(int p, AffinityTopologyVersion topVer, boolean create, boolean updateSeq) {
-        while (true) {
-            boolean belongs = cctx.affinity().localNode(p, topVer);
+    private GridDhtLocalPartition localPartition(int p,
+        AffinityTopologyVersion topVer,
+        boolean create,
+        boolean updateSeq) {
+        boolean belongs = create && cctx.affinity().localNode(p, topVer);
 
+        while (true) {
             GridDhtLocalPartition loc = locParts.get(p);
 
             if (loc != null && loc.state() == EVICTED) {

http://git-wip-us.apache.org/repos/asf/ignite/blob/a7490a6e/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearOptimisticTxPrepareFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearOptimisticTxPrepareFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearOptimisticTxPrepareFuture.java
index 3f9decf..2048fdf 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearOptimisticTxPrepareFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearOptimisticTxPrepareFuture.java
@@ -234,6 +234,8 @@ public class GridNearOptimisticTxPrepareFuture extends GridNearTxPrepareFutureAd
 
     /**
      * Completeness callback.
+     *
+     * @return {@code True} if future was finished by this call.
      */
     private boolean onComplete() {
         Throwable err0 = err.get();
@@ -457,6 +459,7 @@ public class GridNearOptimisticTxPrepareFuture extends GridNearTxPrepareFutureAd
     /**
      * @param reads Read entries.
      * @param writes Write entries.
+     * @throws IgniteCheckedException If failed.
      */
     private void prepare(
         Iterable<IgniteTxEntry> reads,

http://git-wip-us.apache.org/repos/asf/ignite/blob/a7490a6e/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxFinishFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxFinishFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxFinishFuture.java
index 21aaef2..ab6dc3c 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxFinishFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxFinishFuture.java
@@ -404,8 +404,13 @@ public final class GridNearTxFinishFuture<K, V> extends GridCompoundIdentityFutu
                 if (backup == null) {
                     readyNearMappingFromBackup(mapping);
 
+                    ClusterTopologyCheckedException cause =
+                        new ClusterTopologyCheckedException("Backup node left grid: " + backupId);
+
+                    cause.retryReadyFuture(cctx.nextAffinityReadyFuture(tx.topologyVersion()));
+
                     mini.onDone(new IgniteTxRollbackCheckedException("Failed to commit transaction " +
-                        "(backup has left grid): " + tx.xidVersion()));
+                        "(backup has left grid): " + tx.xidVersion(), cause));
                 }
                 else if (backup.isLocal()) {
                     boolean committed = cctx.tm().txHandler().checkDhtRemoteTxCommitted(tx.xidVersion());
@@ -414,9 +419,15 @@ public final class GridNearTxFinishFuture<K, V> extends GridCompoundIdentityFutu
 
                     if (committed)
                         mini.onDone(tx);
-                    else
+                    else {
+                        ClusterTopologyCheckedException cause =
+                            new ClusterTopologyCheckedException("Primary node left grid: " + nodeId);
+
+                        cause.retryReadyFuture(cctx.nextAffinityReadyFuture(tx.topologyVersion()));
+
                         mini.onDone(new IgniteTxRollbackCheckedException("Failed to commit transaction " +
-                            "(transaction has been rolled back on backup node): " + tx.xidVersion()));
+                            "(transaction has been rolled back on backup node): " + tx.xidVersion(), cause));
+                    }
                 }
                 else {
                     GridDhtTxFinishRequest finishReq = new GridDhtTxFinishRequest(
@@ -731,8 +742,19 @@ public final class GridNearTxFinishFuture<K, V> extends GridCompoundIdentityFutu
 
             readyNearMappingFromBackup(m);
 
-            if (res.checkCommittedError() != null)
-                onDone(res.checkCommittedError());
+            Throwable err = res.checkCommittedError();
+
+            if (err != null) {
+                if (err instanceof IgniteCheckedException) {
+                    ClusterTopologyCheckedException cause =
+                        ((IgniteCheckedException)err).getCause(ClusterTopologyCheckedException.class);
+
+                    if (cause != null)
+                        cause.retryReadyFuture(cctx.nextAffinityReadyFuture(tx.topologyVersion()));
+                }
+
+                onDone(err);
+            }
             else
                 onDone(tx);
         }

http://git-wip-us.apache.org/repos/asf/ignite/blob/a7490a6e/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxHandler.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxHandler.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxHandler.java
index 9efa43a..756672a 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxHandler.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxHandler.java
@@ -1049,6 +1049,7 @@ public class IgniteTxHandler {
      *
      * @param nodeId Node id that originated finish request.
      * @param req Request.
+     * @param {@code True} if transaction committed on this node.
      */
     protected void sendReply(UUID nodeId, GridDhtTxFinishRequest req, boolean committed) {
         if (req.replyRequired()) {
@@ -1057,9 +1058,13 @@ public class IgniteTxHandler {
             if (req.checkCommitted()) {
                 res.checkCommitted(true);
 
-                if (!committed)
+                if (!committed) {
+                    ClusterTopologyCheckedException cause =
+                        new ClusterTopologyCheckedException("Primary node left grid.");
+
                     res.checkCommittedError(new IgniteTxRollbackCheckedException("Failed to commit transaction " +
-                        "(transaction has been rolled back on backup node): " + req.version()));
+                        "(transaction has been rolled back on backup node): " + req.version(), cause));
+                }
             }
 
             try {

http://git-wip-us.apache.org/repos/asf/ignite/blob/a7490a6e/modules/core/src/main/java/org/apache/ignite/internal/processors/offheap/GridOffHeapProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/offheap/GridOffHeapProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/offheap/GridOffHeapProcessor.java
index 024ea7c..492fa07 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/offheap/GridOffHeapProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/offheap/GridOffHeapProcessor.java
@@ -30,6 +30,7 @@ import org.apache.ignite.internal.util.offheap.GridOffHeapPartitionedMap;
 import org.apache.ignite.internal.util.typedef.CX2;
 import org.apache.ignite.internal.util.typedef.T2;
 import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.lang.IgniteBiPredicate;
 import org.apache.ignite.lang.IgniteBiTuple;
 import org.apache.ignite.marshaller.Marshaller;
 import org.jetbrains.annotations.Nullable;
@@ -261,13 +262,35 @@ public class GridOffHeapProcessor extends GridProcessorAdapter {
      * @return {@code true} If succeeded.
      * @throws IgniteCheckedException If failed.
      */
-    public boolean removex(@Nullable String spaceName, int part, KeyCacheObject key, byte[] keyBytes) throws IgniteCheckedException {
+    public boolean removex(@Nullable String spaceName, int part, KeyCacheObject key, byte[] keyBytes)
+        throws IgniteCheckedException {
         GridOffHeapPartitionedMap m = offheap(spaceName);
 
         return m != null && m.removex(part, U.hash(key), keyBytes(key, keyBytes));
     }
 
     /**
+     * Removes value from offheap space for the given key.
+     *
+     * @param spaceName Space name.
+     * @param part Partition.
+     * @param key Key.
+     * @param keyBytes Key bytes.
+     * @param p Value predicate (arguments are value address and value length).
+     * @return {@code true} If succeeded.
+     * @throws IgniteCheckedException If failed.
+     */
+    public boolean removex(@Nullable String spaceName,
+        int part,
+        KeyCacheObject key,
+        byte[] keyBytes,
+        IgniteBiPredicate<Long, Integer> p) throws IgniteCheckedException {
+        GridOffHeapPartitionedMap m = offheap(spaceName);
+
+        return m != null && m.removex(part, U.hash(key), keyBytes(key, keyBytes), p);
+    }
+
+    /**
      * Gets iterator over contents of the given space.
      *
      * @param spaceName Space name.

http://git-wip-us.apache.org/repos/asf/ignite/blob/a7490a6e/modules/core/src/main/java/org/apache/ignite/internal/util/offheap/GridOffHeapEvictListener.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/offheap/GridOffHeapEvictListener.java b/modules/core/src/main/java/org/apache/ignite/internal/util/offheap/GridOffHeapEvictListener.java
index 4597be8..beafea4 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/util/offheap/GridOffHeapEvictListener.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/util/offheap/GridOffHeapEvictListener.java
@@ -30,4 +30,9 @@ public interface GridOffHeapEvictListener {
      * @param valBytes Value bytes.
      */
     public void onEvict(int part, int hash, byte[] keyBytes, byte[] valBytes);
+
+    /**
+     * @return {@code True} if entry selected for eviction should be immediately removed.
+     */
+    public boolean removeEvicted();
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/a7490a6e/modules/core/src/main/java/org/apache/ignite/internal/util/offheap/GridOffHeapMap.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/offheap/GridOffHeapMap.java b/modules/core/src/main/java/org/apache/ignite/internal/util/offheap/GridOffHeapMap.java
index 1fcddd7..d14a582 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/util/offheap/GridOffHeapMap.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/util/offheap/GridOffHeapMap.java
@@ -20,13 +20,14 @@ package org.apache.ignite.internal.util.offheap;
 import org.apache.ignite.internal.util.lang.GridCloseableIterator;
 import org.apache.ignite.internal.util.typedef.CX2;
 import org.apache.ignite.internal.util.typedef.T2;
+import org.apache.ignite.lang.IgniteBiPredicate;
 import org.apache.ignite.lang.IgniteBiTuple;
 import org.jetbrains.annotations.Nullable;
 
 /**
  * Off-heap map.
  */
-public interface GridOffHeapMap<K> {
+public interface GridOffHeapMap {
     /**
      * Gets partition this map belongs to.
      *
@@ -102,6 +103,16 @@ public interface GridOffHeapMap<K> {
     public boolean removex(int hash, byte[] keyBytes);
 
     /**
+     * Removes value from off-heap map without returning it.
+     *
+     * @param hash Hash.
+     * @param keyBytes Key bytes.
+     * @param p Value predicate (arguments are value address and value length).
+     * @return {@code True} if value was removed.
+     */
+    public boolean removex(int hash, byte[] keyBytes, IgniteBiPredicate<Long, Integer> p);
+
+    /**
      * Puts key and value bytes into the map potentially replacing
      * existing entry.
      *

http://git-wip-us.apache.org/repos/asf/ignite/blob/a7490a6e/modules/core/src/main/java/org/apache/ignite/internal/util/offheap/GridOffHeapMapFactory.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/offheap/GridOffHeapMapFactory.java b/modules/core/src/main/java/org/apache/ignite/internal/util/offheap/GridOffHeapMapFactory.java
index 1a3d219..4dd911f 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/util/offheap/GridOffHeapMapFactory.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/util/offheap/GridOffHeapMapFactory.java
@@ -32,8 +32,8 @@ public class GridOffHeapMapFactory {
      * @param initCap Initial capacity.
      * @return Off-heap map.
      */
-    public static <K> GridOffHeapMap<K> unsafeMap(long initCap) {
-        return new GridUnsafeMap<>(128, 0.75f, initCap, 0, (short)0, null);
+    public static GridOffHeapMap unsafeMap(long initCap) {
+        return new GridUnsafeMap(128, 0.75f, initCap, 0, (short)0, null);
     }
 
     /**
@@ -43,8 +43,8 @@ public class GridOffHeapMapFactory {
      * @param initCap Initial capacity.
      * @return Off-heap map.
      */
-    public static <K> GridOffHeapMap<K> unsafeMap(int concurrency, long initCap) {
-        return new GridUnsafeMap<>(concurrency, 0.75f, initCap, 0, (short)0, null);
+    public static GridOffHeapMap unsafeMap(int concurrency, long initCap) {
+        return new GridUnsafeMap(concurrency, 0.75f, initCap, 0, (short)0, null);
     }
 
     /**
@@ -55,8 +55,8 @@ public class GridOffHeapMapFactory {
      * @param initCap Initial capacity.
      * @return Off-heap map.
      */
-    public static <K> GridOffHeapMap<K> unsafeMap(int concurrency, float load, long initCap) {
-        return new GridUnsafeMap<>(concurrency, load, initCap, 0, (short)0, null);
+    public static GridOffHeapMap unsafeMap(int concurrency, float load, long initCap) {
+        return new GridUnsafeMap(concurrency, load, initCap, 0, (short)0, null);
     }
 
     /**
@@ -68,8 +68,8 @@ public class GridOffHeapMapFactory {
      * @param lruStripes Number of LRU stripes.
      * @return Off-heap map.
      */
-    public static <K> GridOffHeapMap<K> unsafeMap(long initCap, long totalMem, short lruStripes) {
-        return new GridUnsafeMap<>(128, 0.75f, initCap, totalMem, lruStripes, null);
+    public static GridOffHeapMap unsafeMap(long initCap, long totalMem, short lruStripes) {
+        return new GridUnsafeMap(128, 0.75f, initCap, totalMem, lruStripes, null);
     }
 
     /**
@@ -82,9 +82,9 @@ public class GridOffHeapMapFactory {
      * @param lsnr Optional eviction listener which gets notified every time an entry is evicted.
      * @return Off-heap map.
      */
-    public static <K> GridOffHeapMap<K> unsafeMap(long initCap, long totalMem, short lruStripes,
+    public static GridOffHeapMap unsafeMap(long initCap, long totalMem, short lruStripes,
         @Nullable GridOffHeapEvictListener lsnr) {
-        return new GridUnsafeMap<>(128, 0.75f, initCap, totalMem, lruStripes, lsnr);
+        return new GridUnsafeMap(128, 0.75f, initCap, totalMem, lruStripes, lsnr);
     }
 
     /**
@@ -98,9 +98,9 @@ public class GridOffHeapMapFactory {
      * @param lsnr Optional eviction listener which gets notified every time an entry is evicted.
      * @return Off-heap map.
      */
-    public static <K> GridOffHeapMap<K> unsafeMap(int concurrency, long initCap, long totalMem, short lruStripes,
+    public static GridOffHeapMap unsafeMap(int concurrency, long initCap, long totalMem, short lruStripes,
         @Nullable GridOffHeapEvictListener lsnr) {
-        return new GridUnsafeMap<>(concurrency, 0.75f, initCap, totalMem, lruStripes, lsnr);
+        return new GridUnsafeMap(concurrency, 0.75f, initCap, totalMem, lruStripes, lsnr);
     }
 
     /**
@@ -115,9 +115,9 @@ public class GridOffHeapMapFactory {
      * @param lsnr Optional eviction listener which gets notified every time an entry is evicted.
      * @return Off-heap map.
      */
-    public static <K> GridOffHeapMap<K> unsafeMap(int concurrency, float load, long initCap, long totalMem,
+    public static <K> GridOffHeapMap unsafeMap(int concurrency, float load, long initCap, long totalMem,
         short lruStripes, @Nullable GridOffHeapEvictListener lsnr) {
-        return new GridUnsafeMap<>(concurrency, load, initCap, totalMem, lruStripes, lsnr);
+        return new GridUnsafeMap(concurrency, load, initCap, totalMem, lruStripes, lsnr);
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/a7490a6e/modules/core/src/main/java/org/apache/ignite/internal/util/offheap/GridOffHeapPartitionedMap.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/offheap/GridOffHeapPartitionedMap.java b/modules/core/src/main/java/org/apache/ignite/internal/util/offheap/GridOffHeapPartitionedMap.java
index 3afdfa9..c1e1bfa 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/util/offheap/GridOffHeapPartitionedMap.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/util/offheap/GridOffHeapPartitionedMap.java
@@ -21,6 +21,7 @@ import java.util.Set;
 import org.apache.ignite.internal.util.lang.GridCloseableIterator;
 import org.apache.ignite.internal.util.typedef.CX2;
 import org.apache.ignite.internal.util.typedef.T2;
+import org.apache.ignite.lang.IgniteBiPredicate;
 import org.apache.ignite.lang.IgniteBiTuple;
 import org.jetbrains.annotations.Nullable;
 
@@ -111,6 +112,17 @@ public interface GridOffHeapPartitionedMap {
     public boolean removex(int p, int hash, byte[] keyBytes);
 
     /**
+     * Removes value from off-heap map without returning it.
+     *
+     * @param part Partition.
+     * @param hash Hash.
+     * @param keyBytes Key bytes.
+     * @param p Value predicate (arguments are value address and value length).
+     * @return {@code True} if value was removed.
+     */
+    public boolean removex(int part, int hash, byte[] keyBytes, IgniteBiPredicate<Long, Integer> p);
+
+    /**
      * Puts key and value bytes into the map potentially replacing
      * existing entry.
      *

http://git-wip-us.apache.org/repos/asf/ignite/blob/a7490a6e/modules/core/src/main/java/org/apache/ignite/internal/util/offheap/unsafe/GridUnsafeMap.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/offheap/unsafe/GridUnsafeMap.java b/modules/core/src/main/java/org/apache/ignite/internal/util/offheap/unsafe/GridUnsafeMap.java
index 40fb3e8..359d36c 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/util/offheap/unsafe/GridUnsafeMap.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/util/offheap/unsafe/GridUnsafeMap.java
@@ -33,6 +33,7 @@ import org.apache.ignite.internal.util.typedef.CX2;
 import org.apache.ignite.internal.util.typedef.F;
 import org.apache.ignite.internal.util.typedef.T2;
 import org.apache.ignite.internal.util.typedef.X;
+import org.apache.ignite.lang.IgniteBiPredicate;
 import org.apache.ignite.lang.IgniteBiTuple;
 import org.jetbrains.annotations.Nullable;
 import org.jsr166.LongAdder8;
@@ -42,7 +43,7 @@ import static org.apache.ignite.internal.util.offheap.GridOffHeapEvent.REHASH;
 /**
  * Off-heap map based on {@code Unsafe} implementation.
  */
-public class GridUnsafeMap<K> implements GridOffHeapMap<K> {
+public class GridUnsafeMap implements GridOffHeapMap {
     /** Header size. */
     private static final int HEADER = 4 /*hash*/ + 4 /*key-size*/  + 4 /*value-size*/ + 8 /*queue-address*/ +
         8 /*next-address*/;
@@ -77,7 +78,7 @@ public class GridUnsafeMap<K> implements GridOffHeapMap<K> {
     private final float load;
 
     /** Segments. */
-    private final Segment<K>[] segs;
+    private final Segment[] segs;
 
     /** Total memory. */
     private final GridUnsafeMemory mem;
@@ -111,6 +112,9 @@ public class GridUnsafeMap<K> implements GridOffHeapMap<K> {
     /** LRU poller. */
     private final GridUnsafeLruPoller lruPoller;
 
+    /** */
+    private final boolean rmvEvicted;
+
     /**
      * @param concurrency Concurrency.
      * @param load Load factor.
@@ -180,6 +184,8 @@ public class GridUnsafeMap<K> implements GridOffHeapMap<K> {
                 }
             }
         };
+
+        rmvEvicted = evictLsnr == null || evictLsnr.removeEvicted();
     }
 
     /**
@@ -225,6 +231,8 @@ public class GridUnsafeMap<K> implements GridOffHeapMap<K> {
         segs = new Segment[size];
 
         init(initCap, size);
+
+        rmvEvicted = evictLsnr == null || evictLsnr.removeEvicted();
     }
 
     /**
@@ -247,7 +255,7 @@ public class GridUnsafeMap<K> implements GridOffHeapMap<K> {
 
         for (int i = 0; i < size; i++) {
             try {
-                segs[i] = new Segment<>(i, cap);
+                segs[i] = new Segment(i, cap);
             }
             catch (GridOffHeapOutOfMemoryException e) {
                 destruct();
@@ -327,6 +335,11 @@ public class GridUnsafeMap<K> implements GridOffHeapMap<K> {
     }
 
     /** {@inheritDoc} */
+    @Override public boolean removex(int hash, byte[] keyBytes, IgniteBiPredicate<Long, Integer> p) {
+        return segmentFor(hash).removex(hash, keyBytes, p);
+    }
+
+    /** {@inheritDoc} */
     @Override public boolean put(int hash, byte[] keyBytes, byte[] valBytes) {
         return segmentFor(hash).put(hash, keyBytes, valBytes);
     }
@@ -559,7 +572,7 @@ public class GridUnsafeMap<K> implements GridOffHeapMap<K> {
     /**
      * Segment.
      */
-    private class Segment<K> {
+    private class Segment {
         /** Lock. */
         private final ReadWriteLock lock = new ReentrantReadWriteLock();
 
@@ -1009,41 +1022,51 @@ public class GridUnsafeMap<K> implements GridOffHeapMap<K> {
                         }
 
                         if (cur != 0) {
-                            long next = Entry.nextAddress(cur, mem);
+                            long qAddr0 = Entry.queueAddress(cur, mem);
 
-                            if (prev != 0)
-                                Entry.nextAddress(prev, next, mem); // Relink.
-                            else {
-                                if (next == 0)
-                                    Bin.clear(binAddr, mem);
-                                else
-                                    Bin.first(binAddr, next, mem);
-                            }
+                            assert qAddr == qAddr0 : "Queue node address mismatch " +
+                                "[qAddr=" + qAddr + ", entryQueueAddr=" + qAddr + ']';
 
                             if (evictLsnr != null) {
                                 keyBytes = Entry.readKeyBytes(cur, mem);
 
-                                // TODO: GG-8123: Inlined as a workaround. Revert when 7u60 is released.
-//                                valBytes = Entry.readValueBytes(cur, mem);
-                                {
-                                    int keyLen = Entry.readKeyLength(cur, mem);
-                                    int valLen = Entry.readValueLength(cur, mem);
+                                int keyLen = Entry.readKeyLength(cur, mem);
+                                int valLen = Entry.readValueLength(cur, mem);
 
-                                    valBytes = mem.readBytes(cur + HEADER + keyLen, valLen);
-                                }
+                                valBytes = mem.readBytes(cur + HEADER + keyLen, valLen);
                             }
 
-                            long a;
+                            if (rmvEvicted) {
+                                long a;
 
-                            assert qAddr == (a = Entry.queueAddress(cur, mem)) : "Queue node address mismatch " +
-                                "[qAddr=" + qAddr + ", entryQueueAddr=" + a + ']';
+                                assert qAddr == (a = Entry.queueAddress(cur, mem)) : "Queue node address mismatch " +
+                                    "[qAddr=" + qAddr + ", entryQueueAddr=" + a + ']';
 
-                            relSize = Entry.size(cur, mem);
-                            relAddr = cur;
+                                long next = Entry.nextAddress(cur, mem);
 
-                            cnt--;
+                                if (prev != 0)
+                                    Entry.nextAddress(prev, next, mem); // Relink.
+                                else {
+                                    if (next == 0)
+                                        Bin.clear(binAddr, mem);
+                                    else
+                                        Bin.first(binAddr, next, mem);
+                                }
 
-                            totalCnt.decrement();
+                                relSize = Entry.size(cur, mem);
+                                relAddr = cur;
+
+                                cnt--;
+
+                                totalCnt.decrement();
+                            }
+                            else {
+                                boolean clear = Entry.clearQueueAddress(cur, qAddr, mem);
+
+                                assert clear;
+
+                                relSize = Entry.size(cur, mem);
+                            }
                         }
                     }
                 }
@@ -1251,7 +1274,7 @@ public class GridUnsafeMap<K> implements GridOffHeapMap<K> {
          */
         @SuppressWarnings("TooBroadScope")
         byte[] remove(int hash, byte[] keyBytes) {
-            return remove(hash, keyBytes, true);
+            return remove(hash, keyBytes, true, null);
         }
 
         /**
@@ -1260,17 +1283,28 @@ public class GridUnsafeMap<K> implements GridOffHeapMap<K> {
          * @return {@code True} if value was removed.
          */
         boolean removex(int hash, byte[] keyBytes) {
-            return remove(hash, keyBytes, false) == EMPTY_BYTES;
+            return remove(hash, keyBytes, false, null) == EMPTY_BYTES;
+        }
+
+        /**
+         * @param hash Hash.
+         * @param keyBytes Key bytes.
+         * @param p Value predicate.
+         * @return {@code True} if value was removed.
+         */
+        boolean removex(int hash, byte[] keyBytes, IgniteBiPredicate<Long, Integer> p) {
+            return remove(hash, keyBytes, false, p) == EMPTY_BYTES;
         }
 
         /**
          * @param hash Hash.
          * @param keyBytes Key bytes.
          * @param retval {@code True} if need removed value.
+         * @param p Value predicate.
          * @return Removed value bytes.
          */
         @SuppressWarnings("TooBroadScope")
-        byte[] remove(int hash, byte[] keyBytes, boolean retval) {
+        byte[] remove(int hash, byte[] keyBytes, boolean retval, @Nullable IgniteBiPredicate<Long, Integer> p) {
             int relSize = 0;
             long relAddr = 0;
             long qAddr = 0;
@@ -1291,6 +1325,19 @@ public class GridUnsafeMap<K> implements GridOffHeapMap<K> {
 
                         // If found match.
                         if (Entry.keyEquals(cur, keyBytes, mem)) {
+                            int keyLen = 0;
+                            int valLen = 0;
+
+                            if (p != null) {
+                                keyLen = Entry.readKeyLength(cur, mem);
+                                valLen = Entry.readValueLength(cur, mem);
+
+                                long valPtr = cur + HEADER + keyLen;
+
+                                if (!p.apply(valPtr, valLen))
+                                    return null;
+                            }
+
                             if (prev != 0)
                                 Entry.nextAddress(prev, next, mem); // Relink.
                             else {
@@ -1300,18 +1347,16 @@ public class GridUnsafeMap<K> implements GridOffHeapMap<K> {
                                     Bin.first(binAddr, next, mem);
                             }
 
-                            // TODO: GG-8123: Inlined as a workaround. Revert when 7u60 is released.
-//                            valBytes = retval ? Entry.readValueBytes(cur, mem) : EMPTY_BYTES;
-                            {
-                                if (retval) {
-                                    int keyLen = Entry.readKeyLength(cur, mem);
-                                    int valLen = Entry.readValueLength(cur, mem);
-
-                                    valBytes = mem.readBytes(cur + HEADER + keyLen, valLen);
+                            if (retval) {
+                                if (keyLen == 0) {
+                                    keyLen = Entry.readKeyLength(cur, mem);
+                                    valLen = Entry.readValueLength(cur, mem);
                                 }
-                                else
-                                    valBytes = EMPTY_BYTES;
+
+                                valBytes = mem.readBytes(cur + HEADER + keyLen, valLen);
                             }
+                            else
+                                valBytes = EMPTY_BYTES;
 
                             // Prepare release of memory.
                             qAddr = Entry.queueAddress(cur, mem);
@@ -1382,8 +1427,7 @@ public class GridUnsafeMap<K> implements GridOffHeapMap<K> {
          * @param keyBytes Key bytes.
          * @return Value pointer.
          */
-        @Nullable
-        IgniteBiTuple<Long, Integer> valuePointer(int hash, byte[] keyBytes) {
+        @Nullable IgniteBiTuple<Long, Integer> valuePointer(int hash, byte[] keyBytes) {
             long binAddr = readLock(hash);
 
             try {

http://git-wip-us.apache.org/repos/asf/ignite/blob/a7490a6e/modules/core/src/main/java/org/apache/ignite/internal/util/offheap/unsafe/GridUnsafePartitionedMap.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/offheap/unsafe/GridUnsafePartitionedMap.java b/modules/core/src/main/java/org/apache/ignite/internal/util/offheap/unsafe/GridUnsafePartitionedMap.java
index 070da51..fb8ac14 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/util/offheap/unsafe/GridUnsafePartitionedMap.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/util/offheap/unsafe/GridUnsafePartitionedMap.java
@@ -28,6 +28,7 @@ import org.apache.ignite.internal.util.offheap.GridOffHeapMap;
 import org.apache.ignite.internal.util.offheap.GridOffHeapPartitionedMap;
 import org.apache.ignite.internal.util.typedef.CX2;
 import org.apache.ignite.internal.util.typedef.T2;
+import org.apache.ignite.lang.IgniteBiPredicate;
 import org.apache.ignite.lang.IgniteBiTuple;
 import org.jetbrains.annotations.Nullable;
 import org.jsr166.LongAdder8;
@@ -198,6 +199,14 @@ public class GridUnsafePartitionedMap implements GridOffHeapPartitionedMap {
     }
 
     /** {@inheritDoc} */
+    @Override public boolean removex(int part,
+        int hash,
+        byte[] keyBytes,
+        IgniteBiPredicate<Long, Integer> p) {
+        return mapFor(part).removex(hash, keyBytes, p);
+    }
+
+    /** {@inheritDoc} */
     @Override public boolean put(int p, int hash, byte[] keyBytes, byte[] valBytes) {
         return mapFor(p).put(hash, keyBytes, valBytes);
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/a7490a6e/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheSwapUnswapGetTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheSwapUnswapGetTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheSwapUnswapGetTest.java
index 271d8b1..214beb6 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheSwapUnswapGetTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheSwapUnswapGetTest.java
@@ -26,6 +26,7 @@ import org.apache.ignite.IgniteCache;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.cache.CacheAtomicityMode;
 import org.apache.ignite.cache.CacheMemoryMode;
+import org.apache.ignite.cache.CachePeekMode;
 import org.apache.ignite.cache.eviction.lru.LruEvictionPolicy;
 import org.apache.ignite.configuration.CacheConfiguration;
 import org.apache.ignite.configuration.IgniteConfiguration;
@@ -53,6 +54,12 @@ public class CacheSwapUnswapGetTest extends GridCommonAbstractTest {
     /** */
     private static final long DURATION = 30_000;
 
+    /** */
+    private static final long OFFHEAP_MEM = 1000;
+
+    /** */
+    private static final int MAX_HEAP_SIZE = 100;
+
     /** {@inheritDoc} */
     @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
         IgniteConfiguration cfg = super.getConfiguration(gridName);
@@ -81,7 +88,7 @@ public class CacheSwapUnswapGetTest extends GridCommonAbstractTest {
 
         if (memMode == CacheMemoryMode.ONHEAP_TIERED) {
             LruEvictionPolicy plc = new LruEvictionPolicy();
-            plc.setMaxSize(100);
+            plc.setMaxSize(MAX_HEAP_SIZE);
 
             ccfg.setEvictionPolicy(plc);
         }
@@ -89,7 +96,7 @@ public class CacheSwapUnswapGetTest extends GridCommonAbstractTest {
         if (swap) {
             ccfg.setSwapEnabled(true);
 
-            ccfg.setOffHeapMaxMemory(1000);
+            ccfg.setOffHeapMaxMemory(OFFHEAP_MEM);
         }
         else
             ccfg.setOffHeapMaxMemory(0);
@@ -133,6 +140,20 @@ public class CacheSwapUnswapGetTest extends GridCommonAbstractTest {
     /**
      * @throws Exception If failed.
      */
+    public void testTxCacheOffheapSwapEvict() throws Exception {
+        swapUnswap(TRANSACTIONAL, CacheMemoryMode.ONHEAP_TIERED, true);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testTxCacheOffheapTieredSwapEvict() throws Exception {
+        swapUnswap(TRANSACTIONAL, CacheMemoryMode.OFFHEAP_TIERED, true);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
     public void testAtomicCacheOffheapEvict() throws Exception {
         swapUnswap(ATOMIC, CacheMemoryMode.ONHEAP_TIERED, false);
     }
@@ -145,6 +166,20 @@ public class CacheSwapUnswapGetTest extends GridCommonAbstractTest {
     }
 
     /**
+     * @throws Exception If failed.
+     */
+    public void testAtomicCacheOffheapSwapEvict() throws Exception {
+        swapUnswap(ATOMIC, CacheMemoryMode.ONHEAP_TIERED, true);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testAtomicCacheOffheapTieredSwapEvict() throws Exception {
+        swapUnswap(ATOMIC, CacheMemoryMode.OFFHEAP_TIERED, true);
+    }
+
+    /**
      * @param atomicityMode Cache atomicity mode.
      * @param memMode Cache memory mode.
      * @param swap {@code True} if swap enabled.
@@ -220,12 +255,56 @@ public class CacheSwapUnswapGetTest extends GridCommonAbstractTest {
                 }
             });
 
-            Thread.sleep(DURATION);
+            long endTime = System.currentTimeMillis() + DURATION;
+
+            while (System.currentTimeMillis() < endTime) {
+                Thread.sleep(5000);
+
+                log.info("Cache size [heap=" + cache.localSize(CachePeekMode.ONHEAP) +
+                    ", offheap=" + cache.localSize(CachePeekMode.OFFHEAP) +
+                    ", swap=" + cache.localSize(CachePeekMode.SWAP) +
+                    ", total=" + cache.localSize() +
+                    ", offheapMem=" + cache.metrics().getOffHeapAllocatedSize() + ']');
+            }
 
             done.set(true);
 
             fut.get();
             getFut.get();
+
+            for (Integer key : keys) {
+                String val = cache.get(key);
+
+                assertNotNull(val);
+            }
+
+            int onheapSize = cache.localSize(CachePeekMode.ONHEAP);
+            int offheapSize = cache.localSize(CachePeekMode.OFFHEAP);
+            int swapSize = cache.localSize(CachePeekMode.SWAP);
+            int total = cache.localSize();
+            long offheapMem = cache.metrics().getOffHeapAllocatedSize();
+
+            log.info("Cache size [heap=" + onheapSize +
+                ", offheap=" + offheapSize +
+                ", swap=" + swapSize +
+                ", total=" + total +
+                ", offheapMem=" + offheapMem +  ']');
+
+            assertTrue(total > 0);
+
+            assertEquals(onheapSize + offheapSize + swapSize, total);
+
+            if (memMode == CacheMemoryMode.OFFHEAP_TIERED)
+                assertEquals(0, onheapSize);
+            else
+                assertEquals(MAX_HEAP_SIZE, onheapSize);
+
+            if (swap) {
+                assertTrue(swapSize > 0);
+                assertTrue(offheapMem <= OFFHEAP_MEM);
+            }
+            else
+                assertEquals(0, swapSize);
         }
         finally {
             done.set(true);

http://git-wip-us.apache.org/repos/asf/ignite/blob/a7490a6e/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractFullApiSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractFullApiSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractFullApiSelfTest.java
index 3e646d3..2a64963 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractFullApiSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractFullApiSelfTest.java
@@ -2825,7 +2825,12 @@ public abstract class GridCacheAbstractFullApiSelfTest extends GridCacheAbstract
             try {
                 cache.clear();
 
-                assertEquals(vals.get(first), cache.localPeek(first, ONHEAP));
+                GridCacheContext<String, Integer> cctx = context(0);
+
+                GridCacheEntryEx entry = cctx.isNear() ? cctx.near().dht().peekEx(first) :
+                    cctx.cache().peekEx(first);
+
+                assertNotNull(entry);
             }
             finally {
                 lock.unlock();

http://git-wip-us.apache.org/repos/asf/ignite/blob/a7490a6e/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractRemoveFailureTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractRemoveFailureTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractRemoveFailureTest.java
index 31488e0..647746e 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractRemoveFailureTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractRemoveFailureTest.java
@@ -21,38 +21,50 @@ import java.util.Collection;
 import java.util.HashSet;
 import java.util.Map;
 import java.util.concurrent.Callable;
-import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.CyclicBarrier;
 import java.util.concurrent.ThreadLocalRandom;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.concurrent.atomic.AtomicLong;
-import java.util.concurrent.locks.Lock;
-import java.util.concurrent.locks.ReentrantLock;
+import java.util.concurrent.atomic.AtomicReference;
 import javax.cache.CacheException;
 import org.apache.ignite.Ignite;
 import org.apache.ignite.IgniteCache;
-import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.cache.CacheAtomicWriteOrderMode;
+import org.apache.ignite.cache.CacheAtomicityMode;
+import org.apache.ignite.cache.CacheMode;
+import org.apache.ignite.configuration.CacheConfiguration;
 import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.configuration.NearCacheConfiguration;
 import org.apache.ignite.internal.IgniteInternalFuture;
 import org.apache.ignite.internal.util.lang.GridTuple;
 import org.apache.ignite.internal.util.typedef.F;
 import org.apache.ignite.internal.util.typedef.T2;
 import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi;
+import org.apache.ignite.spi.discovery.tcp.ipfinder.TcpDiscoveryIpFinder;
+import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder;
+import org.apache.ignite.spi.swapspace.file.FileSwapSpaceSpi;
 import org.apache.ignite.testframework.GridTestUtils;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
 import org.jsr166.ConcurrentHashMap8;
 
 import static org.apache.ignite.IgniteSystemProperties.IGNITE_ATOMIC_CACHE_DELETE_HISTORY_SIZE;
+import static org.apache.ignite.cache.CacheMode.PARTITIONED;
+import static org.apache.ignite.cache.CacheWriteSynchronizationMode.FULL_SYNC;
 
 /**
  * Tests that removes are not lost when topology changes.
  */
-public abstract class GridCacheAbstractRemoveFailureTest extends GridCacheAbstractSelfTest {
+public abstract class GridCacheAbstractRemoveFailureTest extends GridCommonAbstractTest {
+    /** IP finder. */
+    private static final TcpDiscoveryIpFinder IP_FINDER = new TcpDiscoveryVmIpFinder(true);
+
     /** */
     private static final int GRID_CNT = 3;
 
     /** Keys count. */
-    private static final int KEYS_CNT = 10000;
+    private static final int KEYS_CNT = 10_000;
 
     /** Test duration. */
     private static final long DUR = 90 * 1000L;
@@ -66,36 +78,21 @@ public abstract class GridCacheAbstractRemoveFailureTest extends GridCacheAbstra
     /** Start delay. */
     private static final T2<Integer, Integer> START_DELAY = new T2<>(2000, 5000);
 
-    /** Node kill lock (used to prevent killing while cache data is compared). */
-    private final Lock killLock = new ReentrantLock();
-
     /** */
-    private CountDownLatch assertLatch;
-
-    /** */
-    private CountDownLatch updateLatch;
-
-    /** Caches comparison request flag. */
-    private volatile boolean cmp;
-
-    /** */
-    private String sizePropVal;
+    private static String sizePropVal;
 
     /** {@inheritDoc} */
     @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
         IgniteConfiguration cfg = super.getConfiguration(gridName);
 
-        ((TcpDiscoverySpi)cfg.getDiscoverySpi()).setForceServerMode(true);
+        ((TcpDiscoverySpi)cfg.getDiscoverySpi()).setIpFinder(IP_FINDER).setForceServerMode(true);
 
         if (testClientNode() && getTestGridName(0).equals(gridName))
             cfg.setClientMode(true);
 
-        return cfg;
-    }
+        cfg.setSwapSpaceSpi(new FileSwapSpaceSpi());
 
-    /** {@inheritDoc} */
-    @Override protected int gridCount() {
-        return GRID_CNT;
+        return cfg;
     }
 
     /** {@inheritDoc} */
@@ -104,6 +101,8 @@ public abstract class GridCacheAbstractRemoveFailureTest extends GridCacheAbstra
         sizePropVal = System.getProperty(IGNITE_ATOMIC_CACHE_DELETE_HISTORY_SIZE);
 
         System.setProperty(IGNITE_ATOMIC_CACHE_DELETE_HISTORY_SIZE, "100000");
+
+        startGrids(GRID_CNT);
     }
 
     /** {@inheritDoc} */
@@ -111,15 +110,7 @@ public abstract class GridCacheAbstractRemoveFailureTest extends GridCacheAbstra
         super.afterTestsStopped();
 
         System.setProperty(IGNITE_ATOMIC_CACHE_DELETE_HISTORY_SIZE, sizePropVal != null ? sizePropVal : "");
-    }
 
-    /** {@inheritDoc} */
-    @Override protected void beforeTest() throws Exception {
-        startGrids(gridCount());
-    }
-
-    /** {@inheritDoc} */
-    @Override protected void afterTest() throws Exception {
         stopAllGrids();
     }
 
@@ -129,6 +120,28 @@ public abstract class GridCacheAbstractRemoveFailureTest extends GridCacheAbstra
     }
 
     /**
+     * @return Cache mode.
+     */
+    protected abstract CacheMode cacheMode();
+
+    /**
+     * @return Cache atomicity mode.
+     */
+    protected abstract CacheAtomicityMode atomicityMode();
+
+    /**
+     * @return Near cache configuration.
+     */
+    protected abstract NearCacheConfiguration nearCache();
+
+    /**
+     * @return Atomic cache write order mode.
+     */
+    protected CacheAtomicWriteOrderMode atomicWriteOrderMode() {
+        return null;
+    }
+
+    /**
      * @return {@code True} if test updates from client node.
      */
     protected boolean testClientNode() {
@@ -139,9 +152,49 @@ public abstract class GridCacheAbstractRemoveFailureTest extends GridCacheAbstra
      * @throws Exception If failed.
      */
     public void testPutAndRemove() throws Exception {
-        assertEquals(testClientNode(), (boolean)grid(0).configuration().isClientMode());
+        putAndRemove(DUR, GridTestUtils.TestMemoryMode.HEAP);
+    }
 
-        final IgniteCache<Integer, Integer> sndCache0 = grid(0).cache(null);
+    /**
+     * @throws Exception If failed.
+     */
+    public void testPutAndRemoveOffheapEvict() throws Exception {
+        putAndRemove(30_000, GridTestUtils.TestMemoryMode.OFFHEAP_EVICT);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testPutAndRemoveOffheapEvictSwap() throws Exception {
+        putAndRemove(30_000, GridTestUtils.TestMemoryMode.OFFHEAP_EVICT_SWAP);
+    }
+
+    /**
+     * @param duration Test duration.
+     * @param memMode Memory mode.
+     * @throws Exception If failed.
+     */
+    private void putAndRemove(long duration, GridTestUtils.TestMemoryMode memMode) throws Exception {
+        assertEquals(testClientNode(), (boolean) grid(0).configuration().isClientMode());
+
+        grid(0).destroyCache(null);
+
+        CacheConfiguration<Integer, Integer> ccfg = new CacheConfiguration<>();
+
+        ccfg.setWriteSynchronizationMode(FULL_SYNC);
+
+        ccfg.setCacheMode(cacheMode());
+
+        if (cacheMode() == PARTITIONED)
+            ccfg.setBackups(1);
+
+        ccfg.setAtomicityMode(atomicityMode());
+        ccfg.setAtomicWriteOrderMode(atomicWriteOrderMode());
+        ccfg.setNearConfiguration(nearCache());
+
+        GridTestUtils.setMemoryMode(null, ccfg, memMode, 100, 1024);
+
+        final IgniteCache<Integer, Integer> sndCache0 = grid(0).createCache(ccfg);
 
         final AtomicBoolean stop = new AtomicBoolean();
 
@@ -152,8 +205,12 @@ public abstract class GridCacheAbstractRemoveFailureTest extends GridCacheAbstra
         // Expected values in cache.
         final Map<Integer, GridTuple<Integer>> expVals = new ConcurrentHashMap8<>();
 
+        final AtomicReference<CyclicBarrier> cmp = new AtomicReference<>();
+
         IgniteInternalFuture<?> updateFut = GridTestUtils.runAsync(new Callable<Void>() {
             @Override public Void call() throws Exception {
+                Thread.currentThread().setName("update-thread");
+
                 ThreadLocalRandom rnd = ThreadLocalRandom.current();
 
                 while (!stop.get()) {
@@ -190,10 +247,14 @@ public abstract class GridCacheAbstractRemoveFailureTest extends GridCacheAbstra
 
                     cntr.addAndGet(100);
 
-                    if (cmp) {
-                        assertLatch.countDown();
+                    CyclicBarrier barrier = cmp.get();
+
+                    if (barrier != null) {
+                        log.info("Wait data check.");
 
-                        updateLatch.await();
+                        barrier.await(60_000, TimeUnit.MILLISECONDS);
+
+                        log.info("Finished wait data check.");
                     }
                 }
 
@@ -203,16 +264,21 @@ public abstract class GridCacheAbstractRemoveFailureTest extends GridCacheAbstra
 
         IgniteInternalFuture<?> killFut = GridTestUtils.runAsync(new Callable<Void>() {
             @Override public Void call() throws Exception {
+                Thread.currentThread().setName("restart-thread");
+
                 while (!stop.get()) {
                     U.sleep(random(KILL_DELAY.get1(), KILL_DELAY.get2()));
 
-                    killLock.lock();
+                    killAndRestart(stop);
 
-                    try {
-                        killAndRestart(stop);
-                    }
-                    finally {
-                        killLock.unlock();
+                    CyclicBarrier barrier = cmp.get();
+
+                    if (barrier != null) {
+                        log.info("Wait data check.");
+
+                        barrier.await(60_000, TimeUnit.MILLISECONDS);
+
+                        log.info("Finished wait data check.");
                     }
                 }
 
@@ -221,7 +287,7 @@ public abstract class GridCacheAbstractRemoveFailureTest extends GridCacheAbstra
         });
 
         try {
-            long stopTime = DUR + U.currentTimeMillis() ;
+            long stopTime = duration + U.currentTimeMillis() ;
 
             long nextAssert = U.currentTimeMillis() + ASSERT_FREQ;
 
@@ -241,31 +307,34 @@ public abstract class GridCacheAbstractRemoveFailureTest extends GridCacheAbstra
                 log.info("Operations/second: " + opsPerSecond);
 
                 if (U.currentTimeMillis() >= nextAssert) {
-                    updateLatch = new CountDownLatch(1);
+                    CyclicBarrier barrier = new CyclicBarrier(3, new Runnable() {
+                        @Override public void run() {
+                            try {
+                                cmp.set(null);
 
-                    assertLatch = new CountDownLatch(1);
+                                log.info("Checking cache content.");
 
-                    cmp = true;
+                                assertCacheContent(expVals);
 
-                    killLock.lock();
+                                log.info("Finished check cache content.");
+                            }
+                            catch (Throwable e) {
+                                log.error("Unexpected error: " + e, e);
 
-                    try {
-                        if (!assertLatch.await(60_000, TimeUnit.MILLISECONDS))
-                            throw new IgniteCheckedException("Failed to suspend thread executing updates.");
+                                throw e;
+                            }
+                        }
+                    });
 
-                        log.info("Checking cache content.");
+                    log.info("Start cache content check.");
 
-                        assertCacheContent(expVals);
+                    cmp.set(barrier);
 
-                        nextAssert = System.currentTimeMillis() + ASSERT_FREQ;
-                    }
-                    finally {
-                        killLock.unlock();
+                    barrier.await(60_000, TimeUnit.MILLISECONDS);
 
-                        updateLatch.countDown();
+                    log.info("Cache content check done.");
 
-                        U.sleep(500);
-                    }
+                    nextAssert = System.currentTimeMillis() + ASSERT_FREQ;
                 }
             }
         }
@@ -278,18 +347,17 @@ public abstract class GridCacheAbstractRemoveFailureTest extends GridCacheAbstra
         updateFut.get();
 
         log.info("Test finished. Update errors: " + errCntr.get());
-
     }
 
     /**
      * @param stop Stop flag.
      * @throws Exception If failed.
      */
-    void killAndRestart(AtomicBoolean stop) throws Exception {
+    private void killAndRestart(AtomicBoolean stop) throws Exception {
         if (stop.get())
             return;
 
-        int idx = random(1, gridCount() + 1);
+        int idx = random(1, GRID_CNT + 1);
 
         log.info("Killing node " + idx);
 
@@ -309,10 +377,9 @@ public abstract class GridCacheAbstractRemoveFailureTest extends GridCacheAbstra
 
     /**
      * @param expVals Expected values in cache.
-     * @throws Exception If failed.
      */
     @SuppressWarnings({"TooBroadScope", "ConstantIfStatement"})
-    private void assertCacheContent(Map<Integer, GridTuple<Integer>> expVals) throws Exception {
+    private void assertCacheContent(Map<Integer, GridTuple<Integer>> expVals) {
         assert !expVals.isEmpty();
 
         Collection<Integer> failedKeys = new HashSet<>();

http://git-wip-us.apache.org/repos/asf/ignite/blob/a7490a6e/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheMemoryModeSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheMemoryModeSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheMemoryModeSelfTest.java
index c5c1c39..2f6ee8f 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheMemoryModeSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheMemoryModeSelfTest.java
@@ -232,9 +232,15 @@ public class GridCacheMemoryModeSelfTest extends GridCommonAbstractTest {
         //putAll
         doTest(cache, offheapSwap, offheapEmpty, swapEmpty, new CIX1<IgniteCache<String, Integer>>() {
             @Override public void applyx(IgniteCache<String, Integer> c) throws IgniteCheckedException {
+                putAll(c, 0, all / 2);
+
+                putAll(c, all / 2 + 1, all - 1);
+            }
+
+            private void putAll(IgniteCache<String, Integer> c, int k1, int k2) {
                 Map<String, Integer> m = new HashMap<>();
 
-                for (int i = 0; i < all; i++)
+                for (int i = k1; i <= k2; i++)
                     m.put(valueOf(i), i);
 
                 c.putAll(m);
@@ -264,6 +270,7 @@ public class GridCacheMemoryModeSelfTest extends GridCommonAbstractTest {
         assertEquals(offheapSwap, c.localSize(CachePeekMode.OFFHEAP) + c.localSize(CachePeekMode.SWAP));
 
         info("size: " + c.size());
+        info("heap: " + c.localSize(CachePeekMode.ONHEAP));
         info("offheap: " + c.localSize(CachePeekMode.OFFHEAP));
         info("swap: " + c.localSize(CachePeekMode.SWAP));
 


[26/55] [abbrv] ignite git commit: IGNITE-1477: Fixed.

Posted by ag...@apache.org.
IGNITE-1477: Fixed.


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

Branch: refs/heads/ignite-1171
Commit: dc44a2a78ce90f5fcab82c0b39d6888724778181
Parents: c0e1ac1
Author: vozerov-gridgain <vo...@gridgain.com>
Authored: Tue Sep 15 10:42:15 2015 +0300
Committer: vozerov-gridgain <vo...@gridgain.com>
Committed: Tue Sep 15 10:42:15 2015 +0300

----------------------------------------------------------------------
 .../internal/processors/hadoop/igfs/HadoopIgfsWrapper.java   | 8 +++++---
 1 file changed, 5 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/dc44a2a7/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/igfs/HadoopIgfsWrapper.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/igfs/HadoopIgfsWrapper.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/igfs/HadoopIgfsWrapper.java
index 01189f7..857db71 100644
--- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/igfs/HadoopIgfsWrapper.java
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/igfs/HadoopIgfsWrapper.java
@@ -79,7 +79,7 @@ public class HadoopIgfsWrapper implements HadoopIgfs {
      * @param log Current logger.
      */
     public HadoopIgfsWrapper(String authority, String logDir, Configuration conf, Log log, String user)
-            throws IOException {
+        throws IOException {
         try {
             this.authority = authority;
             this.endpoint = new HadoopIgfsEndpoint(authority);
@@ -351,7 +351,9 @@ public class HadoopIgfsWrapper implements HadoopIgfs {
             return curDelegate;
 
         // 2. Guess that we are in the same VM.
-        if (!parameter(conf, PARAM_IGFS_ENDPOINT_NO_EMBED, authority, false)) {
+        boolean skipInProc = parameter(conf, PARAM_IGFS_ENDPOINT_NO_EMBED, authority, false);
+
+        if (!skipInProc) {
             IgfsEx igfs = null;
 
             if (endpoint.grid() == null) {
@@ -471,7 +473,7 @@ public class HadoopIgfsWrapper implements HadoopIgfs {
             return curDelegate;
         }
         else {
-            SB errMsg = new SB("Failed to connect to IGFS [endpoint=" + authority + ", attempts=[");
+            SB errMsg = new SB("Failed to connect to IGFS [endpoint=igfs://" + authority + ", attempts=[");
 
             if (errShmem != null)
                 errMsg.a("[type=SHMEM, port=" + endpoint.port() + ", err=" + errShmem + "], ");


[20/55] [abbrv] ignite git commit: Removed portable API information from 1.4 release notes

Posted by ag...@apache.org.
Removed portable API information from 1.4 release notes


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

Branch: refs/heads/ignite-1171
Commit: 961a46719b7de465ad7f263d106a4e9a7b926065
Parents: d39345b
Author: Denis Magda <dm...@gridgain.com>
Authored: Tue Sep 15 08:37:27 2015 +0300
Committer: Denis Magda <dm...@gridgain.com>
Committed: Tue Sep 15 08:37:27 2015 +0300

----------------------------------------------------------------------
 RELEASE_NOTES.txt | 1 -
 1 file changed, 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/961a4671/RELEASE_NOTES.txt
----------------------------------------------------------------------
diff --git a/RELEASE_NOTES.txt b/RELEASE_NOTES.txt
index 661ebe4..3f3ac7b 100644
--- a/RELEASE_NOTES.txt
+++ b/RELEASE_NOTES.txt
@@ -4,7 +4,6 @@ Apache Ignite Release Notes
 Apache Ignite In-Memory Data Fabric 1.4
 ---------------------------------------
 * Added SSL support to communication and discovery.
-* Added portable objects API.
 * Added support for log4j2.
 * Added versioned entry to cache API.
 * Fixed IGNITE_HOME resolution with JBoss.


[11/55] [abbrv] ignite git commit: ignite-1462: hid portable API in 1.4 release

Posted by ag...@apache.org.
ignite-1462: hid portable API in 1.4 release


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

Branch: refs/heads/ignite-1171
Commit: 71379a8061f50f336adc31fa20cd593b659b050f
Parents: b4c515e
Author: Denis Magda <dm...@gridgain.com>
Authored: Mon Sep 14 17:24:27 2015 +0300
Committer: Denis Magda <dm...@gridgain.com>
Committed: Mon Sep 14 17:24:29 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       |  230 --
 .../CacheClientPortableQueryExample.java        |  325 --
 .../portable/datagrid/package-info.java         |   21 -
 .../ignite/examples/portable/package-info.java  |   21 -
 .../CacheClientPortableExampleTest.java         |   46 -
 .../ComputeClientPortableExampleTest.java       |   37 -
 .../testsuites/IgniteExamplesSelfTestSuite.java |    6 -
 modules/core/pom.xml                            |   21 -
 .../src/main/java/org/apache/ignite/Ignite.java |    7 -
 .../java/org/apache/ignite/IgniteCache.java     |   44 +-
 .../java/org/apache/ignite/IgnitePortables.java |  370 --
 .../configuration/CacheConfiguration.java       |   70 +-
 .../org/apache/ignite/internal/IgniteEx.java    |    9 +
 .../apache/ignite/internal/IgniteKernal.java    |    8 +-
 .../ignite/internal/IgniteNodeAttributes.java   |    5 +-
 .../discovery/GridDiscoveryManager.java         |   10 -
 .../portable/GridPortableMarshaller.java        |    2 +-
 .../portable/PortableClassDescriptor.java       |   10 +-
 .../internal/portable/PortableContext.java      |   14 +-
 .../portable/PortableMetaDataCollector.java     |    6 +-
 .../portable/PortableMetaDataHandler.java       |    4 +-
 .../internal/portable/PortableMetaDataImpl.java |   14 +-
 .../internal/portable/PortableObjectEx.java     |    6 +-
 .../internal/portable/PortableObjectImpl.java   |    6 +-
 .../portable/PortableObjectOffheapImpl.java     |    6 +-
 .../internal/portable/PortableRawReaderEx.java  |    4 +-
 .../internal/portable/PortableRawWriterEx.java  |    4 +-
 .../portable/PortableReaderContext.java         |    2 +-
 .../internal/portable/PortableReaderExImpl.java |   10 +-
 .../ignite/internal/portable/PortableUtils.java |    2 +-
 .../internal/portable/PortableWriterExImpl.java |   11 +-
 .../internal/portable/api/IgnitePortables.java  |  362 ++
 .../internal/portable/api/PortableBuilder.java  |  136 +
 .../portable/api/PortableException.java         |   57 +
 .../internal/portable/api/PortableIdMapper.java |   54 +
 .../api/PortableInvalidClassException.java      |   58 +
 .../portable/api/PortableMarshalAware.java      |   48 +
 .../portable/api/PortableMarshaller.java        |  358 ++
 .../internal/portable/api/PortableMetadata.java |   60 +
 .../internal/portable/api/PortableObject.java   |  152 +
 .../portable/api/PortableProtocolVersion.java   |   41 +
 .../portable/api/PortableRawReader.java         |  234 ++
 .../portable/api/PortableRawWriter.java         |  219 +
 .../internal/portable/api/PortableReader.java   |  284 ++
 .../portable/api/PortableSerializer.java        |   47 +
 .../portable/api/PortableTypeConfiguration.java |  195 +
 .../internal/portable/api/PortableWriter.java   |  266 ++
 .../portable/builder/PortableBuilderEnum.java   |    2 +-
 .../portable/builder/PortableBuilderImpl.java   |   14 +-
 .../portable/builder/PortableBuilderReader.java |    2 +-
 .../builder/PortableBuilderSerializer.java      |    2 +-
 .../builder/PortableEnumArrayLazyValue.java     |    4 +-
 .../builder/PortableObjectArrayLazyValue.java   |    2 +-
 .../builder/PortablePlainPortableObject.java    |    2 +-
 .../streams/PortableAbstractInputStream.java    |    2 +-
 .../processors/cache/GridCacheProcessor.java    |    8 +-
 .../processors/cache/IgniteCacheProxy.java      |    5 -
 .../CacheDefaultPortableAffinityKeyMapper.java  |    2 +-
 .../portable/CacheObjectPortableContext.java    |    2 +-
 .../portable/CacheObjectPortableProcessor.java  |    8 +-
 .../CacheObjectPortableProcessorImpl.java       |   12 +-
 .../cache/portable/IgnitePortablesImpl.java     |   10 +-
 .../cache/store/CacheOsStoreManager.java        |    4 +-
 .../dotnet/PlatformDotNetConfiguration.java     |   12 +-
 .../PlatformDotNetPortableConfiguration.java    |   12 +-
 ...PlatformDotNetPortableTypeConfiguration.java |   12 +-
 .../marshaller/portable/PortableMarshaller.java |  358 --
 .../marshaller/portable/package-info.java       |   22 -
 .../apache/ignite/portable/PortableBuilder.java |  137 -
 .../ignite/portable/PortableException.java      |   57 -
 .../ignite/portable/PortableIdMapper.java       |   56 -
 .../portable/PortableInvalidClassException.java |   58 -
 .../ignite/portable/PortableMarshalAware.java   |   48 -
 .../ignite/portable/PortableMetadata.java       |   61 -
 .../apache/ignite/portable/PortableObject.java  |  154 -
 .../portable/PortableProtocolVersion.java       |   41 -
 .../ignite/portable/PortableRawReader.java      |  234 --
 .../ignite/portable/PortableRawWriter.java      |  219 -
 .../apache/ignite/portable/PortableReader.java  |  284 --
 .../ignite/portable/PortableSerializer.java     |   49 -
 .../portable/PortableTypeConfiguration.java     |  196 -
 .../apache/ignite/portable/PortableWriter.java  |  266 --
 .../apache/ignite/portable/package-info.java    |   22 -
 .../resources/META-INF/classnames.properties    |    8 +-
 .../GridDiscoveryManagerAttributesSelfTest.java |   45 -
 .../GridPortableAffinityKeySelfTest.java        |  218 -
 .../GridPortableBuilderAdditionalSelfTest.java  | 1226 ------
 .../portable/GridPortableBuilderSelfTest.java   | 1021 -----
 ...eBuilderStringAsCharsAdditionalSelfTest.java |   28 -
 ...ridPortableBuilderStringAsCharsSelfTest.java |   28 -
 ...idPortableMarshallerCtxDisabledSelfTest.java |  256 --
 .../GridPortableMarshallerSelfTest.java         | 3807 ------------------
 .../GridPortableMetaDataDisabledSelfTest.java   |  238 --
 .../portable/GridPortableMetaDataSelfTest.java  |  371 --
 .../portable/GridPortableWildcardsSelfTest.java |  482 ---
 .../GridPortableMarshalerAwareTestClass.java    |   67 -
 .../mutabletest/GridPortableTestClasses.java    |  434 --
 .../portable/mutabletest/package-info.java      |   22 -
 .../ignite/internal/portable/package-info.java  |   22 -
 .../portable/test/GridPortableTestClass1.java   |   28 -
 .../portable/test/GridPortableTestClass2.java   |   24 -
 .../internal/portable/test/package-info.java    |   22 -
 .../test/subpackage/GridPortableTestClass3.java |   24 -
 .../portable/test/subpackage/package-info.java  |   22 -
 ...ClientNodePortableMetadataMultinodeTest.java |  295 --
 ...GridCacheClientNodePortableMetadataTest.java |  286 --
 ...ableObjectsAbstractDataStreamerSelfTest.java |  190 -
 ...bleObjectsAbstractMultiThreadedSelfTest.java |  231 --
 ...ridCachePortableObjectsAbstractSelfTest.java |  978 -----
 .../GridCachePortableStoreAbstractSelfTest.java |  297 --
 .../GridCachePortableStoreObjectsSelfTest.java  |   55 -
 ...GridCachePortableStorePortablesSelfTest.java |   66 -
 ...ridPortableCacheEntryMemorySizeSelfTest.java |   55 -
 ...leDuplicateIndexObjectsAbstractSelfTest.java |  158 -
 .../DataStreamProcessorPortableSelfTest.java    |   66 -
 .../GridDataStreamerImplSelfTest.java           |  345 --
 ...ridCacheAffinityRoutingPortableSelfTest.java |   47 -
 ...lyPortableDataStreamerMultiNodeSelfTest.java |   29 -
 ...rtableDataStreamerMultithreadedSelfTest.java |   47 -
 ...artitionedOnlyPortableMultiNodeSelfTest.java |   28 -
 ...tionedOnlyPortableMultithreadedSelfTest.java |   47 -
 .../GridCacheMemoryModePortableSelfTest.java    |   36 -
 ...acheOffHeapTieredAtomicPortableSelfTest.java |   47 -
 ...eapTieredEvictionAtomicPortableSelfTest.java |   95 -
 ...heOffHeapTieredEvictionPortableSelfTest.java |   95 -
 .../GridCacheOffHeapTieredPortableSelfTest.java |   47 -
 ...ateIndexObjectPartitionedAtomicSelfTest.java |   38 -
 ...xObjectPartitionedTransactionalSelfTest.java |   41 -
 ...AtomicNearDisabledOffheapTieredSelfTest.java |   29 -
 ...rtableObjectsAtomicNearDisabledSelfTest.java |   51 -
 ...tableObjectsAtomicOffheapTieredSelfTest.java |   29 -
 .../GridCachePortableObjectsAtomicSelfTest.java |   51 -
 ...tionedNearDisabledOffheapTieredSelfTest.java |   30 -
 ...eObjectsPartitionedNearDisabledSelfTest.java |   51 -
 ...ObjectsPartitionedOffheapTieredSelfTest.java |   30 -
 ...CachePortableObjectsPartitionedSelfTest.java |   51 -
 ...sNearPartitionedByteArrayValuesSelfTest.java |   41 -
 ...sPartitionedOnlyByteArrayValuesSelfTest.java |   42 -
 ...dCachePortableObjectsReplicatedSelfTest.java |   51 -
 ...CachePortableObjectsAtomicLocalSelfTest.java |   32 -
 ...rtableObjectsLocalOffheapTieredSelfTest.java |   29 -
 .../GridCachePortableObjectsLocalSelfTest.java  |   51 -
 .../ignite/testframework/junits/IgniteMock.java |    8 +-
 .../multijvm/IgniteCacheProcessProxy.java       |    5 -
 .../junits/multijvm/IgniteProcessProxy.java     |    2 +-
 .../IgnitePortableCacheFullApiTestSuite.java    |   37 -
 .../IgnitePortableCacheTestSuite.java           |  103 -
 .../IgnitePortableObjectsTestSuite.java         |   92 -
 .../ignite/portable/test1/1.1/test1-1.1.jar     |  Bin 2548 -> 0 bytes
 .../ignite/portable/test1/1.1/test1-1.1.pom     |    9 -
 .../portable/test1/maven-metadata-local.xml     |   12 -
 .../ignite/portable/test2/1.1/test2-1.1.jar     |  Bin 1361 -> 0 bytes
 .../ignite/portable/test2/1.1/test2-1.1.pom     |    9 -
 .../portable/test2/maven-metadata-local.xml     |   12 -
 .../HadoopDefaultMapReducePlannerSelfTest.java  |    6 +
 .../IgnitePortableCacheQueryTestSuite.java      |  117 -
 .../platform/PlatformContextImpl.java           |    4 +-
 .../platform/compute/PlatformCompute.java       |    2 +-
 .../cpp/PlatformCppConfigurationClosure.java    |    2 +-
 .../PlatformDotNetConfigurationClosure.java     |    6 +-
 .../Config/Compute/compute-grid1.xml            |    8 +-
 .../PlatformComputePortableArgTask.java         |    8 +-
 .../org/apache/ignite/IgniteSpringBean.java     |    7 -
 parent/pom.xml                                  |   10 -
 176 files changed, 2778 insertions(+), 17425 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/71379a80/examples/config/example-default.xml
----------------------------------------------------------------------
diff --git a/examples/config/example-default.xml b/examples/config/example-default.xml
deleted file mode 100644
index e6c359d..0000000
--- a/examples/config/example-default.xml
+++ /dev/null
@@ -1,76 +0,0 @@
-<?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/71379a80/examples/config/example-ignite.xml
----------------------------------------------------------------------
diff --git a/examples/config/example-ignite.xml b/examples/config/example-ignite.xml
index d842a6d..e7adb54 100644
--- a/examples/config/example-ignite.xml
+++ b/examples/config/example-ignite.xml
@@ -22,18 +22,62 @@
 -->
 <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"/>
+       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"/>
 
-    <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/71379a80/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
deleted file mode 100644
index cde15ea..0000000
--- a/examples/config/portable/example-ignite-portable.xml
+++ /dev/null
@@ -1,44 +0,0 @@
-<?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/71379a80/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
deleted file mode 100644
index cb08b25..0000000
--- a/examples/src/main/java/org/apache/ignite/examples/portable/Address.java
+++ /dev/null
@@ -1,72 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.ignite.examples.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/71379a80/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
deleted file mode 100644
index 9614168..0000000
--- a/examples/src/main/java/org/apache/ignite/examples/portable/Employee.java
+++ /dev/null
@@ -1,93 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.ignite.examples.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/71379a80/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
deleted file mode 100644
index f322167..0000000
--- a/examples/src/main/java/org/apache/ignite/examples/portable/EmployeeKey.java
+++ /dev/null
@@ -1,90 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.ignite.examples.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/71379a80/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
deleted file mode 100644
index 87a41f7..0000000
--- a/examples/src/main/java/org/apache/ignite/examples/portable/ExamplePortableNodeStartup.java
+++ /dev/null
@@ -1,36 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.ignite.examples.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/71379a80/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
deleted file mode 100644
index f52cac1..0000000
--- a/examples/src/main/java/org/apache/ignite/examples/portable/Organization.java
+++ /dev/null
@@ -1,93 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.ignite.examples.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/71379a80/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
deleted file mode 100644
index c753e2d..0000000
--- a/examples/src/main/java/org/apache/ignite/examples/portable/OrganizationType.java
+++ /dev/null
@@ -1,32 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.ignite.examples.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/71379a80/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
deleted file mode 100644
index 34d9cde..0000000
--- a/examples/src/main/java/org/apache/ignite/examples/portable/computegrid/ComputeClientPortableTaskExecutionExample.java
+++ /dev/null
@@ -1,154 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.ignite.examples.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/71379a80/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
deleted file mode 100644
index 0eee8c6..0000000
--- a/examples/src/main/java/org/apache/ignite/examples/portable/computegrid/ComputeClientTask.java
+++ /dev/null
@@ -1,116 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.ignite.examples.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/71379a80/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
deleted file mode 100644
index 469128c..0000000
--- a/examples/src/main/java/org/apache/ignite/examples/portable/computegrid/package-info.java
+++ /dev/null
@@ -1,21 +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.
- */
-
-/**
- * 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/71379a80/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
deleted file mode 100644
index 77c5d95..0000000
--- a/examples/src/main/java/org/apache/ignite/examples/portable/datagrid/CacheClientPortablePutGetExample.java
+++ /dev/null
@@ -1,230 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.ignite.examples.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();
-            }
-            finally {
-                // Delete cache with its content completely.
-                ignite.destroyCache(CACHE_NAME);
-            }
-        }
-    }
-
-    /**
-     * 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/71379a80/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
deleted file mode 100644
index 3170864..0000000
--- a/examples/src/main/java/org/apache/ignite/examples/portable/datagrid/CacheClientPortableQueryExample.java
+++ /dev/null
@@ -1,325 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.ignite.examples.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 {@link ExamplePortableNodeStartup} which starts a 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();
-            }
-            finally {
-                // Delete caches with their content completely.
-                ignite.destroyCache(ORGANIZATION_CACHE_NAME);
-                ignite.destroyCache(EMPLOYEE_CACHE_NAME);
-            }
-        }
-    }
-
-    /**
-     * 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/71379a80/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
deleted file mode 100644
index b24f233..0000000
--- a/examples/src/main/java/org/apache/ignite/examples/portable/datagrid/package-info.java
+++ /dev/null
@@ -1,21 +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.
- */
-
-/**
- * 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/71379a80/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
deleted file mode 100644
index 4301027..0000000
--- a/examples/src/main/java/org/apache/ignite/examples/portable/package-info.java
+++ /dev/null
@@ -1,21 +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.
- */
-
-/**
- * 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/71379a80/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
deleted file mode 100644
index 6ea1484..0000000
--- a/examples/src/test/java/org/apache/ignite/examples/CacheClientPortableExampleTest.java
+++ /dev/null
@@ -1,46 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.ignite.examples;
-
-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/71379a80/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
deleted file mode 100644
index 2223aec..0000000
--- a/examples/src/test/java/org/apache/ignite/examples/ComputeClientPortableExampleTest.java
+++ /dev/null
@@ -1,37 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.ignite.examples;
-
-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/71379a80/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 baa23fc..4669ae4 100644
--- a/examples/src/test/java/org/apache/ignite/testsuites/IgniteExamplesSelfTestSuite.java
+++ b/examples/src/test/java/org/apache/ignite/testsuites/IgniteExamplesSelfTestSuite.java
@@ -20,12 +20,10 @@ 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;
@@ -95,10 +93,6 @@ 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

http://git-wip-us.apache.org/repos/asf/ignite/blob/71379a80/modules/core/pom.xml
----------------------------------------------------------------------
diff --git a/modules/core/pom.xml b/modules/core/pom.xml
index 2f0dde7..9162afe 100644
--- a/modules/core/pom.xml
+++ b/modules/core/pom.xml
@@ -34,13 +34,6 @@
     <version>1.4.0-SNAPSHOT</version>
     <url>http://ignite.apache.org</url>
 
-    <repositories>
-        <repository>
-            <id>ignite-portables-test-repo</id>
-            <url>file://${basedir}/src/test/portables/repo</url>
-        </repository>
-    </repositories>
-
     <properties>
         <ignite.update.notifier.product>apache-ignite</ignite.update.notifier.product>
     </properties>
@@ -176,20 +169,6 @@
             <version>2.4</version>
             <scope>test</scope>
         </dependency>
-
-        <dependency>
-            <groupId>org.apache.ignite.portable</groupId>
-            <artifactId>test1</artifactId>
-            <version>1.1</version>
-            <scope>test</scope>
-        </dependency>
-
-        <dependency>
-            <groupId>org.apache.ignite.portable</groupId>
-            <artifactId>test2</artifactId>
-            <version>1.1</version>
-            <scope>test</scope>
-        </dependency>
     </dependencies>
 
     <build>

http://git-wip-us.apache.org/repos/asf/ignite/blob/71379a80/modules/core/src/main/java/org/apache/ignite/Ignite.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/Ignite.java b/modules/core/src/main/java/org/apache/ignite/Ignite.java
index 62fd020..0afccd0 100644
--- a/modules/core/src/main/java/org/apache/ignite/Ignite.java
+++ b/modules/core/src/main/java/org/apache/ignite/Ignite.java
@@ -459,13 +459,6 @@ public interface Ignite extends AutoCloseable {
     public <T extends IgnitePlugin> T plugin(String name) throws PluginNotFoundException;
 
     /**
-     * Gets an instance of {@link IgnitePortables} interface.
-     *
-     * @return Instance of {@link IgnitePortables} interface.
-     */
-    public IgnitePortables portables();
-
-    /**
      * Closes {@code this} instance of grid. This method is identical to calling
      * {@link G#stop(String, boolean) G.stop(gridName, true)}.
      * <p>

http://git-wip-us.apache.org/repos/asf/ignite/blob/71379a80/modules/core/src/main/java/org/apache/ignite/IgniteCache.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/IgniteCache.java b/modules/core/src/main/java/org/apache/ignite/IgniteCache.java
index e0f9f55..5558a26 100644
--- a/modules/core/src/main/java/org/apache/ignite/IgniteCache.java
+++ b/modules/core/src/main/java/org/apache/ignite/IgniteCache.java
@@ -18,12 +18,9 @@
 package org.apache.ignite;
 
 import java.io.Serializable;
-import java.sql.Timestamp;
 import java.util.Collection;
-import java.util.Date;
 import java.util.Map;
 import java.util.Set;
-import java.util.UUID;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.locks.Lock;
 import javax.cache.Cache;
@@ -55,7 +52,6 @@ import org.apache.ignite.lang.IgniteAsyncSupported;
 import org.apache.ignite.lang.IgniteBiInClosure;
 import org.apache.ignite.lang.IgniteBiPredicate;
 import org.apache.ignite.lang.IgniteFuture;
-import org.apache.ignite.marshaller.portable.PortableMarshaller;
 import org.apache.ignite.mxbean.CacheMetricsMXBean;
 import org.jetbrains.annotations.Nullable;
 
@@ -132,44 +128,6 @@ public interface IgniteCache<K, V> extends javax.cache.Cache<K, V>, IgniteAsyncS
     public IgniteCache<K, V> withNoRetries();
 
     /**
-     * Returns cache that will operate with portable objects.
-     * <p>
-     * Cache returned by this method will not be forced to deserialize portable objects,
-     * so keys and values will be returned from cache API methods without changes. Therefore,
-     * signature of the cache can contain only following types:
-     * <ul>
-     *     <li><code>org.apache.ignite.portable.PortableObject</code> for portable classes</li>
-     *     <li>All primitives (byte, int, ...) and there boxed versions (Byte, Integer, ...)</li>
-     *     <li>Arrays of primitives (byte[], int[], ...)</li>
-     *     <li>{@link String} and array of {@link String}s</li>
-     *     <li>{@link UUID} and array of {@link UUID}s</li>
-     *     <li>{@link Date} and array of {@link Date}s</li>
-     *     <li>{@link Timestamp} and array of {@link Timestamp}s</li>
-     *     <li>Enums and array of enums</li>
-     *     <li>
-     *         Maps, collections and array of objects (but objects inside
-     *         them will still be converted if they are portable)
-     *     </li>
-     * </ul>
-     * <p>
-     * For example, if you use {@link Integer} as a key and {@code Value} class as a value
-     * (which will be stored in portable format), you should acquire following projection
-     * to avoid deserialization:
-     * <pre>
-     * IgniteCache<Integer, PortableObject> prj = cache.withKeepPortable();
-     *
-     * // Value is not deserialized and returned in portable format.
-     * PortableObject po = prj.get(1);
-     * </pre>
-     * <p>
-     * Note that this method makes sense only if cache is working in portable mode ({@link PortableMarshaller} is used).
-     * If not, this method is no-op and will return current cache.
-     *
-     * @return New cache instance for portable objects.
-     */
-    public <K1, V1> IgniteCache<K1, V1> withKeepPortable();
-
-    /**
      * Executes {@link #localLoadCache(IgniteBiPredicate, Object...)} on all cache nodes.
      *
      * @param p Optional predicate (may be {@code null}). If provided, will be used to
@@ -661,4 +619,4 @@ public interface IgniteCache<K, V> extends javax.cache.Cache<K, V>, IgniteAsyncS
      * @return MxBean.
      */
     public CacheMetricsMXBean mxBean();
-}
\ No newline at end of file
+}