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/10/30 11:44:30 UTC

[02/19] ignite git commit: ignite-950-new WIP

http://git-wip-us.apache.org/repos/asf/ignite/blob/35b6d61f/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
index 2f7bdb0..360ecec 100644
--- 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
@@ -31,11 +31,11 @@ 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.igniteobject.IgniteObjectException;
+import org.apache.ignite.igniteobject.IgniteObjectMarshalAware;
+import org.apache.ignite.igniteobject.IgniteObject;
+import org.apache.ignite.igniteobject.IgniteObjectReader;
+import org.apache.ignite.igniteobject.IgniteObjectWriter;
 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;
@@ -221,12 +221,12 @@ public class GridDataStreamerImplSelfTest extends GridCommonAbstractTest {
             }
 
             // Read random keys. Take values as PortableObject.
-            IgniteCache<Integer, PortableObject> c2 = ((IgniteCacheProxy)c).keepPortable();
+            IgniteCache<Integer, IgniteObject> c2 = ((IgniteCacheProxy)c).keepPortable();
 
             for (int i = 0; i < 100; i ++) {
                 Integer k = rnd.nextInt(KEYS_COUNT);
 
-                PortableObject v = c2.get(k);
+                IgniteObject v = c2.get(k);
 
                 assertEquals(k, v.field("val"));
             }
@@ -256,7 +256,7 @@ public class GridDataStreamerImplSelfTest extends GridCommonAbstractTest {
 
     /**
      */
-    private static class TestObject implements PortableMarshalAware, Serializable {
+    private static class TestObject implements IgniteObjectMarshalAware, Serializable {
         /** */
         private int val;
 
@@ -289,19 +289,19 @@ public class GridDataStreamerImplSelfTest extends GridCommonAbstractTest {
         }
 
         /** {@inheritDoc} */
-        @Override public void writePortable(PortableWriter writer) throws PortableException {
+        @Override public void writePortable(IgniteObjectWriter writer) throws IgniteObjectException {
             writer.writeInt("val", val);
         }
 
         /** {@inheritDoc} */
-        @Override public void readPortable(PortableReader reader) throws PortableException {
+        @Override public void readPortable(IgniteObjectReader reader) throws IgniteObjectException {
             val = reader.readInt("val");
         }
     }
 
     /**
      */
-    private static class TestObject2 implements PortableMarshalAware, Serializable {
+    private static class TestObject2 implements IgniteObjectMarshalAware, Serializable {
         /** */
         private int val;
 
@@ -333,12 +333,12 @@ public class GridDataStreamerImplSelfTest extends GridCommonAbstractTest {
         }
 
         /** {@inheritDoc} */
-        @Override public void writePortable(PortableWriter writer) throws PortableException {
+        @Override public void writePortable(IgniteObjectWriter writer) throws IgniteObjectException {
             writer.writeInt("val", val);
         }
 
         /** {@inheritDoc} */
-        @Override public void readPortable(PortableReader reader) throws PortableException {
+        @Override public void readPortable(IgniteObjectReader reader) throws IgniteObjectException {
             val = reader.readInt("val");
         }
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/35b6d61f/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/distributed/dht/GridCacheAffinityRoutingPortableSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/distributed/dht/GridCacheAffinityRoutingPortableSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/distributed/dht/GridCacheAffinityRoutingPortableSelfTest.java
index 155ba48..9964168 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/distributed/dht/GridCacheAffinityRoutingPortableSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/distributed/dht/GridCacheAffinityRoutingPortableSelfTest.java
@@ -18,10 +18,11 @@
 package org.apache.ignite.internal.processors.cache.portable.distributed.dht;
 
 import java.util.Collections;
+import org.apache.ignite.cache.CacheKeyConfiguration;
 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;
+import org.apache.ignite.igniteobject.IgniteObjectConfiguration;
 
 /**
  *
@@ -31,10 +32,13 @@ public class GridCacheAffinityRoutingPortableSelfTest extends GridCacheAffinityR
     @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
         IgniteConfiguration cfg = super.getConfiguration(gridName);
 
-        PortableTypeConfiguration typeCfg = new PortableTypeConfiguration();
+        IgniteObjectConfiguration typeCfg = new IgniteObjectConfiguration();
 
         typeCfg.setClassName(AffinityTestKey.class.getName());
-        typeCfg.setAffinityKeyFieldName("affKey");
+
+        CacheKeyConfiguration keyCfg = new CacheKeyConfiguration(AffinityTestKey.class.getName(), "affKey");
+
+        cfg.setCacheKeyCfg(keyCfg);
 
         PortableMarshaller marsh = new PortableMarshaller();
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/35b6d61f/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
index 3f8cd1c..21a2085 100644
--- 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
@@ -20,7 +20,7 @@ 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 org.apache.ignite.internal.processors.cache.portable.GridCacheIgniteObjectsAbstractDataStreamerSelfTest;
 
 import static org.apache.ignite.cache.CacheAtomicityMode.ATOMIC;
 import static org.apache.ignite.cache.CacheMode.PARTITIONED;
@@ -29,7 +29,7 @@ import static org.apache.ignite.cache.CacheMode.PARTITIONED;
  *
  */
 public class GridCacheAtomicPartitionedOnlyPortableDataStreamerMultithreadedSelfTest extends
-    GridCachePortableObjectsAbstractDataStreamerSelfTest {
+    GridCacheIgniteObjectsAbstractDataStreamerSelfTest {
     /** {@inheritDoc} */
     @Override protected CacheMode cacheMode() {
         return PARTITIONED;

http://git-wip-us.apache.org/repos/asf/ignite/blob/35b6d61f/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
index 8f3a05f..5033204 100644
--- 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
@@ -20,7 +20,7 @@ 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 org.apache.ignite.internal.processors.cache.portable.GridCacheIgniteObjectsAbstractMultiThreadedSelfTest;
 
 import static org.apache.ignite.cache.CacheAtomicityMode.ATOMIC;
 import static org.apache.ignite.cache.CacheMode.PARTITIONED;
@@ -29,7 +29,7 @@ import static org.apache.ignite.cache.CacheMode.PARTITIONED;
  *
  */
 public class GridCacheAtomicPartitionedOnlyPortableMultithreadedSelfTest extends
-    GridCachePortableObjectsAbstractMultiThreadedSelfTest {
+    GridCacheIgniteObjectsAbstractMultiThreadedSelfTest {
     /** {@inheritDoc} */
     @Override protected CacheMode cacheMode() {
         return PARTITIONED;

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

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

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

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

http://git-wip-us.apache.org/repos/asf/ignite/blob/35b6d61f/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/distributed/dht/GridCacheIgniteObjectsPartitionedSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/distributed/dht/GridCacheIgniteObjectsPartitionedSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/distributed/dht/GridCacheIgniteObjectsPartitionedSelfTest.java
new file mode 100644
index 0000000..c708f1b
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/distributed/dht/GridCacheIgniteObjectsPartitionedSelfTest.java
@@ -0,0 +1,51 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT 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.GridCacheIgniteObjectsAbstractSelfTest;
+
+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 GridCacheIgniteObjectsPartitionedSelfTest extends GridCacheIgniteObjectsAbstractSelfTest {
+    /** {@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/35b6d61f/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
index 1a0d601..b45a88a 100644
--- 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
@@ -21,7 +21,7 @@ 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;
+import org.apache.ignite.igniteobject.IgniteObject;
 
 /**
  *
@@ -66,7 +66,7 @@ public class GridCacheOffHeapTieredEvictionAtomicPortableSelfTest extends GridCa
 
         /** {@inheritDoc} */
         @Override public void checkValue(Object val) {
-            PortableObject obj = (PortableObject)val;
+            IgniteObject obj = (IgniteObject)val;
 
             assertEquals(expVal, obj.field("val"));
         }
@@ -87,7 +87,7 @@ public class GridCacheOffHeapTieredEvictionAtomicPortableSelfTest extends GridCa
 
         /** {@inheritDoc} */
         @Override public void checkValue(Object val) {
-            PortableObject obj = (PortableObject)val;
+            IgniteObject obj = (IgniteObject)val;
 
             assertEquals(expVal, obj.field("val"));
         }

http://git-wip-us.apache.org/repos/asf/ignite/blob/35b6d61f/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
index 60eed45..838109d 100644
--- 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
@@ -21,7 +21,7 @@ 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;
+import org.apache.ignite.igniteobject.IgniteObject;
 
 /**
  *
@@ -66,7 +66,7 @@ public class GridCacheOffHeapTieredEvictionPortableSelfTest extends GridCacheOff
 
         /** {@inheritDoc} */
         @Override public void checkValue(Object val) {
-            PortableObject obj = (PortableObject)val;
+            IgniteObject obj = (IgniteObject)val;
 
             assertEquals(expVal, obj.field("val"));
         }
@@ -87,7 +87,7 @@ public class GridCacheOffHeapTieredEvictionPortableSelfTest extends GridCacheOff
 
         /** {@inheritDoc} */
         @Override public void checkValue(Object val) {
-            PortableObject obj = (PortableObject)val;
+            IgniteObject obj = (IgniteObject)val;
 
             assertEquals(expVal, obj.field("val"));
         }

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

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

http://git-wip-us.apache.org/repos/asf/ignite/blob/35b6d61f/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/local/GridCacheIgniteObjectsLocalSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/local/GridCacheIgniteObjectsLocalSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/local/GridCacheIgniteObjectsLocalSelfTest.java
new file mode 100644
index 0000000..a3c5373
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/local/GridCacheIgniteObjectsLocalSelfTest.java
@@ -0,0 +1,51 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT 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.GridCacheIgniteObjectsAbstractSelfTest;
+
+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 GridCacheIgniteObjectsLocalSelfTest extends GridCacheIgniteObjectsAbstractSelfTest {
+    /** {@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/35b6d61f/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/35b6d61f/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/35b6d61f/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/35b6d61f/modules/core/src/test/java/org/apache/ignite/platform/PlatformComputePortableArgTask.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/platform/PlatformComputePortableArgTask.java b/modules/core/src/test/java/org/apache/ignite/platform/PlatformComputePortableArgTask.java
index 0e8b825..bd15f1f 100644
--- a/modules/core/src/test/java/org/apache/ignite/platform/PlatformComputePortableArgTask.java
+++ b/modules/core/src/test/java/org/apache/ignite/platform/PlatformComputePortableArgTask.java
@@ -25,8 +25,8 @@ import org.apache.ignite.compute.ComputeJobAdapter;
 import org.apache.ignite.compute.ComputeJobResult;
 import org.apache.ignite.compute.ComputeTaskAdapter;
 import org.apache.ignite.internal.util.typedef.F;
-import org.apache.ignite.portable.PortableMetadata;
-import org.apache.ignite.portable.PortableObject;
+import org.apache.ignite.igniteobject.IgniteObjectMetadata;
+import org.apache.ignite.igniteobject.IgniteObject;
 import org.apache.ignite.resources.IgniteInstanceResource;
 import org.jetbrains.annotations.Nullable;
 
@@ -87,9 +87,9 @@ public class PlatformComputePortableArgTask extends ComputeTaskAdapter<Object, I
 
         /** {@inheritDoc} */
         @Nullable @Override public Object execute() {
-            PortableObject arg0 = ((PortableObject)arg);
+            IgniteObject arg0 = ((IgniteObject)arg);
 
-            PortableMetadata meta = ignite.portables().metadata(arg0.typeId());
+            IgniteObjectMetadata meta = ignite.portables().metadata(arg0.typeId());
 
             if (meta == null)
                 throw new IgniteException("Metadata doesn't exist.");

http://git-wip-us.apache.org/repos/asf/ignite/blob/35b6d61f/modules/core/src/test/java/org/apache/ignite/platform/PlatformEventsWriteEventTask.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/platform/PlatformEventsWriteEventTask.java b/modules/core/src/test/java/org/apache/ignite/platform/PlatformEventsWriteEventTask.java
index d9dee9d..f4e3ae8 100644
--- a/modules/core/src/test/java/org/apache/ignite/platform/PlatformEventsWriteEventTask.java
+++ b/modules/core/src/test/java/org/apache/ignite/platform/PlatformEventsWriteEventTask.java
@@ -32,7 +32,7 @@ import org.apache.ignite.events.DiscoveryEvent;
 import org.apache.ignite.events.JobEvent;
 import org.apache.ignite.events.SwapSpaceEvent;
 import org.apache.ignite.events.TaskEvent;
-import org.apache.ignite.internal.portable.PortableRawWriterEx;
+import org.apache.ignite.internal.portable.IgniteObjectRawWriterEx;
 import org.apache.ignite.internal.processors.platform.PlatformContext;
 import org.apache.ignite.internal.processors.platform.memory.PlatformMemory;
 import org.apache.ignite.internal.processors.platform.memory.PlatformOutputStream;
@@ -41,8 +41,6 @@ import org.apache.ignite.internal.util.typedef.F;
 import org.apache.ignite.lang.IgniteUuid;
 import org.apache.ignite.events.*;
 
-import org.apache.ignite.plugin.security.SecurityPermission;
-import org.apache.ignite.plugin.security.SecuritySubjectType;
 import org.apache.ignite.resources.IgniteInstanceResource;
 import org.jetbrains.annotations.Nullable;
 
@@ -97,7 +95,7 @@ public class PlatformEventsWriteEventTask extends ComputeTaskAdapter<Long, Objec
 
             try (PlatformMemory mem = ctx.memory().get(ptr)) {
                 PlatformOutputStream out = mem.output();
-                PortableRawWriterEx writer = ctx.writer(out);
+                IgniteObjectRawWriterEx writer = ctx.writer(out);
 
                 int evtType = EventType.EVT_SWAP_SPACE_CLEARED;
                 String msg = "msg";

http://git-wip-us.apache.org/repos/asf/ignite/blob/35b6d61f/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..a4a05aa 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,7 @@ 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.IgniteObjects;
 import org.apache.ignite.IgniteQueue;
 import org.apache.ignite.IgniteScheduler;
 import org.apache.ignite.IgniteServices;
@@ -271,7 +271,7 @@ public class IgniteMock implements Ignite {
     }
 
     /** {@inheritDoc} */
-    @Override public IgnitePortables portables() {
+    @Override public IgniteObjects portables() {
         return null;
     }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/35b6d61f/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 d44f9c8..d2e17d8 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
@@ -424,7 +424,7 @@ public class IgniteCacheProcessProxy<K, V> implements IgniteCache<K, V> {
     }
 
     /** {@inheritDoc} */
-    @Override public <K1, V1> IgniteCache<K1, V1> withKeepPortable() {
+    @Override public <K1, V1> IgniteCache<K1, V1> withKeepBinary() {
         throw new UnsupportedOperationException("Method should be supported.");
     }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/35b6d61f/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 35e2dde..6c3d91d 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.IgniteObjects;
 import org.apache.ignite.IgniteQueue;
 import org.apache.ignite.IgniteScheduler;
 import org.apache.ignite.IgniteServices;
@@ -546,7 +546,7 @@ public class IgniteProcessProxy implements IgniteEx {
     }
 
     /** {@inheritDoc} */
-    @Override public IgnitePortables portables() {
+    @Override public IgniteObjects portables() {
         throw new UnsupportedOperationException("Operation isn't supported yet.");
     }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/35b6d61f/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
index ecd25e1..01bc296 100644
--- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgnitePortableObjectsTestSuite.java
+++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgnitePortableObjectsTestSuite.java
@@ -19,31 +19,31 @@ 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.GridIgniteObjectBuilderAdditionalSelfTest;
+import org.apache.ignite.internal.portable.GridIgniteObjectBuilderSelfTest;
+import org.apache.ignite.internal.portable.GridIgniteObjectBuilderStringAsCharsAdditionalSelfTest;
+import org.apache.ignite.internal.portable.GridIgniteObjectBuilderStringAsCharsSelfTest;
 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.GridCacheClientNodeIgniteObjectMetadataMultinodeTest;
+import org.apache.ignite.internal.processors.cache.portable.GridCacheClientNodeIgniteObjectMetadataTest;
 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;
+import org.apache.ignite.internal.processors.cache.portable.distributed.dht.GridCacheIgniteObjectsAtomicNearDisabledOffheapTieredSelfTest;
+import org.apache.ignite.internal.processors.cache.portable.distributed.dht.GridCacheIgniteObjectsAtomicNearDisabledSelfTest;
+import org.apache.ignite.internal.processors.cache.portable.distributed.dht.GridCacheIgniteObjectsAtomicOffheapTieredSelfTest;
+import org.apache.ignite.internal.processors.cache.portable.distributed.dht.GridCacheIgniteObjectsAtomicSelfTest;
+import org.apache.ignite.internal.processors.cache.portable.distributed.dht.GridCacheIgniteObjectsPartitionedNearDisabledOffheapTieredSelfTest;
+import org.apache.ignite.internal.processors.cache.portable.distributed.dht.GridCacheIgniteObjectsPartitionedNearDisabledSelfTest;
+import org.apache.ignite.internal.processors.cache.portable.distributed.dht.GridCacheIgniteObjectsPartitionedOffheapTieredSelfTest;
+import org.apache.ignite.internal.processors.cache.portable.distributed.dht.GridCacheIgniteObjectsPartitionedSelfTest;
+import org.apache.ignite.internal.processors.cache.portable.distributed.replicated.GridCacheIgniteObjectsReplicatedSelfTest;
+import org.apache.ignite.internal.processors.cache.portable.local.GridCacheIgniteObjectsAtomicLocalSelfTest;
+import org.apache.ignite.internal.processors.cache.portable.local.GridCacheIgniteObjectsLocalOffheapTieredSelfTest;
+import org.apache.ignite.internal.processors.cache.portable.local.GridCacheIgniteObjectsLocalSelfTest;
 
 /**
  * Test for portable objects stored in cache.
@@ -58,34 +58,34 @@ public class IgnitePortableObjectsTestSuite extends TestSuite {
 
         suite.addTestSuite(GridPortableMarshallerSelfTest.class);
         suite.addTestSuite(GridPortableMarshallerCtxDisabledSelfTest.class);
-        suite.addTestSuite(GridPortableBuilderSelfTest.class);
-        suite.addTestSuite(GridPortableBuilderStringAsCharsSelfTest.class);
+        suite.addTestSuite(GridIgniteObjectBuilderSelfTest.class);
+        suite.addTestSuite(GridIgniteObjectBuilderStringAsCharsSelfTest.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(GridIgniteObjectBuilderAdditionalSelfTest.class);
+        suite.addTestSuite(GridIgniteObjectBuilderStringAsCharsAdditionalSelfTest.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(GridCacheIgniteObjectsLocalSelfTest.class);
+        suite.addTestSuite(GridCacheIgniteObjectsAtomicLocalSelfTest.class);
+        suite.addTestSuite(GridCacheIgniteObjectsReplicatedSelfTest.class);
+        suite.addTestSuite(GridCacheIgniteObjectsPartitionedSelfTest.class);
+        suite.addTestSuite(GridCacheIgniteObjectsPartitionedNearDisabledSelfTest.class);
+        suite.addTestSuite(GridCacheIgniteObjectsAtomicSelfTest.class);
+        suite.addTestSuite(GridCacheIgniteObjectsAtomicNearDisabledSelfTest.class);
 
-        suite.addTestSuite(GridCachePortableObjectsLocalOffheapTieredSelfTest.class);
-        suite.addTestSuite(GridCachePortableObjectsAtomicOffheapTieredSelfTest.class);
-        suite.addTestSuite(GridCachePortableObjectsAtomicNearDisabledOffheapTieredSelfTest.class);
-        suite.addTestSuite(GridCachePortableObjectsPartitionedOffheapTieredSelfTest.class);
-        suite.addTestSuite(GridCachePortableObjectsPartitionedNearDisabledOffheapTieredSelfTest.class);
+        suite.addTestSuite(GridCacheIgniteObjectsLocalOffheapTieredSelfTest.class);
+        suite.addTestSuite(GridCacheIgniteObjectsAtomicOffheapTieredSelfTest.class);
+        suite.addTestSuite(GridCacheIgniteObjectsAtomicNearDisabledOffheapTieredSelfTest.class);
+        suite.addTestSuite(GridCacheIgniteObjectsPartitionedOffheapTieredSelfTest.class);
+        suite.addTestSuite(GridCacheIgniteObjectsPartitionedNearDisabledOffheapTieredSelfTest.class);
 
         suite.addTestSuite(GridCachePortableStoreObjectsSelfTest.class);
         suite.addTestSuite(GridCachePortableStorePortablesSelfTest.class);
 
-        suite.addTestSuite(GridCacheClientNodePortableMetadataTest.class);
-        suite.addTestSuite(GridCacheClientNodePortableMetadataMultinodeTest.class);
+        suite.addTestSuite(GridCacheClientNodeIgniteObjectMetadataTest.class);
+        suite.addTestSuite(GridCacheClientNodeIgniteObjectMetadataMultinodeTest.class);
 
         return suite;
     }