You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by vo...@apache.org on 2016/03/10 12:05:57 UTC

[1/2] ignite git commit: IGNITE-2753: Removed unnecessary deserialization of keys and values passed to store manager. This closes #537.

Repository: ignite
Updated Branches:
  refs/heads/master 648b10c8c -> 8cf71d463


IGNITE-2753: Removed unnecessary deserialization of keys and values passed to store manager. This closes #537.


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

Branch: refs/heads/master
Commit: b46ed10acfc46117dd597c1e7e986e4216bfd047
Parents: a8d16bd
Author: dkarachentsev <dk...@gridgain.com>
Authored: Thu Mar 10 14:05:40 2016 +0300
Committer: vozerov-gridgain <vo...@gridgain.com>
Committed: Thu Mar 10 14:05:40 2016 +0300

----------------------------------------------------------------------
 .../processors/cache/GridCacheMapEntry.java     |  25 +-
 .../distributed/dht/GridDhtCacheEntry.java      |   2 +-
 ...ridCacheStoreManagerDeserializationTest.java | 345 +++++++++++++++++++
 ...calCacheStoreManagerDeserializationTest.java | 101 ++++++
 .../testsuites/IgniteCacheTestSuite4.java       |  11 +-
 5 files changed, 468 insertions(+), 16 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/b46ed10a/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 1658b85..16ff153 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
@@ -208,7 +208,7 @@ public abstract class GridCacheMapEntry extends GridMetadataAwareAdapter impleme
 
         // In case we deal with IGFS cache, count updated data
         if (cctx.cache().isIgfsDataCache() &&
-            cctx.kernalContext().igfsHelper().isIgfsBlockKey(key.value(cctx.cacheObjectContext(), false))) {
+            cctx.kernalContext().igfsHelper().isIgfsBlockKey(keyValue(false))) {
             int newSize = valueLength0(val, null);
             int oldSize = valueLength0(this.val, (this.val == null && hasOffHeapPointer()) ? valueBytes0() : null);
 
@@ -591,7 +591,7 @@ public abstract class GridCacheMapEntry extends GridMetadataAwareAdapter impleme
                 }
 
                 keyClsLdrId = cctx.deploy().getClassLoaderId(
-                    U.detectObjectClassLoader(key.value(cctx.cacheObjectContext(), false)));
+                    U.detectObjectClassLoader(keyValue(false)));
             }
 
             IgniteBiTuple<byte[], Byte> valBytes = valueBytes0();
@@ -1250,7 +1250,7 @@ public abstract class GridCacheMapEntry extends GridMetadataAwareAdapter impleme
         // Persist outside of synchronization. The correctness of the
         // value will be handled by current transaction.
         if (writeThrough)
-            cctx.store().put(tx, keyValue(false), CU.value(val, cctx, false), newVer);
+            cctx.store().put(tx, key, val, newVer);
 
         if (intercept)
             cctx.config().getInterceptor().onAfterPut(new CacheLazyEntry(cctx, key, key0, val, val0, keepBinary));
@@ -1718,7 +1718,8 @@ public abstract class GridCacheMapEntry extends GridMetadataAwareAdapter impleme
 
                 if (writeThrough)
                     // Must persist inside synchronization in non-tx mode.
-                    cctx.store().put(null, keyValue(false), CU.value(updated, cctx, false), ver);
+                    cctx.store().put(null, key, updated, ver);
+
 
                 // Update index inside synchronization since it can be updated
                 // in load methods without actually holding entry lock.
@@ -1752,7 +1753,7 @@ public abstract class GridCacheMapEntry extends GridMetadataAwareAdapter impleme
             else {
                 if (writeThrough)
                     // Must persist inside synchronization in non-tx mode.
-                    cctx.store().remove(null, keyValue(false));
+                    cctx.store().remove(null, key);
 
                 boolean hasValPtr = hasOffHeapPointer();
 
@@ -1978,10 +1979,10 @@ public abstract class GridCacheMapEntry extends GridMetadataAwareAdapter impleme
                             if (val == null) {
                                 assert deletedUnlocked();
 
-                                cctx.store().remove(null, keyValue(false));
+                                cctx.store().remove(null, key);
                             }
                             else
-                                cctx.store().put(null, keyValue(false), CU.value(val, cctx, false), ver);
+                                cctx.store().put(null, key, val, ver);
                         }
 
                         return new GridCacheUpdateAtomicResult(false,
@@ -2031,10 +2032,10 @@ public abstract class GridCacheMapEntry extends GridMetadataAwareAdapter impleme
                             if (val == null) {
                                 assert deletedUnlocked();
 
-                                cctx.store().remove(null, keyValue(false));
+                                cctx.store().remove(null, key);
                             }
                             else
-                                cctx.store().put(null, keyValue(false), CU.value(val, cctx, false), ver);
+                                cctx.store().put(null, key, val, ver);
                         }
                         else {
                             if (log.isDebugEnabled())
@@ -3295,7 +3296,7 @@ public abstract class GridCacheMapEntry extends GridMetadataAwareAdapter impleme
 
                 if (cctx.store().isLocal()) {
                     if (val != null)
-                        cctx.store().put(null, keyValue(false), CU.value(val, cctx, false), ver);
+                        cctx.store().put(null, key, val, ver);
                 }
 
                 return true;
@@ -3358,7 +3359,7 @@ public abstract class GridCacheMapEntry extends GridMetadataAwareAdapter impleme
 
         CacheObject val = isNew ? unswap(true) : rawGetOrUnmarshalUnlocked(false);
 
-        return new GridCachePlainVersionedEntry<>(key.value(cctx.cacheObjectContext(), true),
+        return new GridCachePlainVersionedEntry<>(keyValue(true),
             CU.value(val, cctx, true),
             ttlExtras(),
             expireTimeExtras(),
@@ -4106,7 +4107,7 @@ public abstract class GridCacheMapEntry extends GridMetadataAwareAdapter impleme
                         }
 
                         keyClsLdrId = cctx.deploy().getClassLoaderId(
-                            U.detectObjectClassLoader(key.value(cctx.cacheObjectContext(), false)));
+                            U.detectObjectClassLoader(keyValue(false)));
                     }
 
                     IgniteBiTuple<byte[], Byte> valBytes = valueBytes0();

http://git-wip-us.apache.org/repos/asf/ignite/blob/b46ed10a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtCacheEntry.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtCacheEntry.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtCacheEntry.java
index fae8219..ab51bdb 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtCacheEntry.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtCacheEntry.java
@@ -590,7 +590,7 @@ public class GridDhtCacheEntry extends GridDistributedCacheEntry {
                 }
 
                 if (cctx.store().isLocal())
-                    cctx.store().remove(null, keyValue(false));
+                    cctx.store().remove(null, key);
 
                 rmv = true;
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/b46ed10a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheStoreManagerDeserializationTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheStoreManagerDeserializationTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheStoreManagerDeserializationTest.java
new file mode 100644
index 0000000..9fe60e0
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheStoreManagerDeserializationTest.java
@@ -0,0 +1,345 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.cache;
+
+import org.apache.ignite.Ignite;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.IgniteDataStreamer;
+import org.apache.ignite.binary.BinaryObject;
+import org.apache.ignite.binary.BinaryObjectBuilder;
+import org.apache.ignite.cache.CacheAtomicWriteOrderMode;
+import org.apache.ignite.cache.CacheAtomicityMode;
+import org.apache.ignite.cache.CacheMode;
+import org.apache.ignite.cache.CacheWriteSynchronizationMode;
+import org.apache.ignite.cache.store.CacheStoreAdapter;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.internal.binary.BinaryMarshaller;
+import org.apache.ignite.internal.processors.cache.extras.GridCacheObsoleteEntryExtras;
+import org.apache.ignite.internal.processors.cache.store.CacheLocalStore;
+import org.apache.ignite.internal.processors.cache.version.GridCacheVersion;
+import org.apache.ignite.marshaller.optimized.OptimizedMarshaller;
+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.jsr166.ConcurrentHashMap8;
+
+import javax.cache.Cache;
+import javax.cache.integration.CacheLoaderException;
+import javax.cache.integration.CacheWriterException;
+import java.io.Serializable;
+import java.util.Map;
+
+import static org.apache.ignite.cache.CacheRebalanceMode.SYNC;
+
+/**
+ * Checks whether storing to local store doesn't cause binary objects unmarshalling,
+ * and as a consequence {@link ClassNotFoundException} to be thrown.
+ *
+ * @see <a href="https://issues.apache.org/jira/browse/IGNITE-2753">
+ *     https://issues.apache.org/jira/browse/IGNITE-2753
+ *     </a>
+ */
+public class GridCacheStoreManagerDeserializationTest extends GridCommonAbstractTest {
+    /** IP finder. */
+    protected static final TcpDiscoveryIpFinder IP_FINDER = new TcpDiscoveryVmIpFinder(true);
+
+    /** Cache store. */
+    protected static final GridCacheLocalTestStore store = new GridCacheLocalTestStore();
+
+    /** Test cache name. */
+    protected static final String CACHE_NAME = "cache_name";
+
+    /** Cache mode. */
+    protected CacheMode cacheMode() {
+        return CacheMode.PARTITIONED;
+    }
+
+    /** Cache write order mode. */
+    protected CacheAtomicWriteOrderMode cacheAtomicWriteOrderMode() {
+        return CacheAtomicWriteOrderMode.PRIMARY;
+    }
+
+    /** Cache synchronization mode. */
+    private CacheWriteSynchronizationMode cacheWriteSynchronizationMode() {
+        return CacheWriteSynchronizationMode.FULL_SYNC;
+    }
+
+    /** {@inheritDoc} */
+    @SuppressWarnings("unchecked")
+    @Override protected IgniteConfiguration getConfiguration(final String gridName) throws Exception {
+        IgniteConfiguration c = super.getConfiguration(gridName);
+
+        if (gridName != null && gridName.toLowerCase().startsWith("binary"))
+            c.setMarshaller(new BinaryMarshaller());
+        else
+            c.setMarshaller(new OptimizedMarshaller());
+
+        TcpDiscoverySpi disco = new TcpDiscoverySpi();
+
+        disco.setIpFinder(IP_FINDER);
+
+        c.setDiscoverySpi(disco);
+
+        c.setCacheConfiguration(cacheConfiguration());
+
+        return c;
+    }
+
+    /**
+     * @return Cache configuration.
+     */
+    @SuppressWarnings("unchecked")
+    protected CacheConfiguration cacheConfiguration() {
+        CacheConfiguration cc = defaultCacheConfiguration();
+
+        cc.setSwapEnabled(false);
+        cc.setRebalanceMode(SYNC);
+
+        cc.setCacheStoreFactory(singletonFactory(store));
+        cc.setReadThrough(true);
+        cc.setWriteThrough(true);
+        cc.setLoadPreviousValue(true);
+        cc.setStoreKeepBinary(true);
+
+        cc.setCacheMode(cacheMode());
+        cc.setAtomicWriteOrderMode(cacheAtomicWriteOrderMode());
+        cc.setWriteSynchronizationMode(cacheWriteSynchronizationMode());
+
+        cc.setBackups(0);
+
+        cc.setAtomicityMode(CacheAtomicityMode.ATOMIC);
+
+        return cc;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTest() throws Exception {
+        store.map.clear();
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTest() throws Exception {
+        stopAllGrids();
+    }
+
+    /**
+     * Check whether test objects are stored correctly via stream API.
+     *
+     * @throws Exception
+     */
+    public void testStream() throws Exception {
+        final Ignite grid = startGrid();
+
+        final IgniteCache<TestObj, TestObj> cache = grid.createCache(CACHE_NAME);
+
+        final TestObj testObj = streamData(grid);
+
+        cache.destroy();
+        cache.close();
+
+        assert store.map.containsKey(testObj);
+
+        final IgniteCache<TestObj, TestObj> cache2 = grid.createCache(CACHE_NAME);
+
+        assert testObj.equals(cache2.get(testObj));
+        assert store.map.containsKey(testObj);
+    }
+
+    /**
+     * Simulate case where is called
+     * {@link org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtCacheEntry#clearInternal(
+     * GridCacheVersion, boolean, GridCacheObsoleteEntryExtras)}
+     *
+     * @throws Exception
+     */
+    public void testPartitionMove() throws Exception {
+        final Ignite grid = startGrid("binaryGrid1");
+
+        grid.createCache(CACHE_NAME);
+
+        final BinaryObjectBuilder builder = grid.binary().builder("custom_type");
+
+        final IgniteDataStreamer<BinaryObject, BinaryObject> streamer = grid.dataStreamer(CACHE_NAME);
+
+        streamer.keepBinary(true);
+
+        final int itemsNum = 10_000;
+
+        for (int i = 0; i < itemsNum; i++) {
+            final BinaryObject key = builder.setField("id", i).build();
+
+            streamer.addData(key, key);
+        }
+
+        streamer.close();
+
+        streamer.future().get();
+
+        assert store.map.size() == itemsNum;
+
+        startGrid("binaryGrid2");
+        startGrid("binaryGrid3");
+        startGrid("binaryGrid4");
+
+        Thread.sleep(10_000);
+    }
+
+    /**
+     * Check whether binary objects are stored without unmarshalling via stream API.
+     *
+     * @throws Exception
+     */
+    public void testBinaryStream() throws Exception {
+        final Ignite grid = startGrid("binaryGrid");
+
+        final IgniteCache<BinaryObject, BinaryObject> cache = grid.createCache(CACHE_NAME).withKeepBinary();
+
+        final BinaryObject key = streamBinaryData(grid);
+
+        assert cache.containsKey(key);
+        assert store.map.containsKey(key);
+
+        cache.destroy();
+        cache.close();
+
+        assert store.map.containsKey(key);
+
+        final IgniteCache<BinaryObject, BinaryObject> cache2 = grid.createCache(CACHE_NAME).withKeepBinary();
+
+        final BinaryObject loaded = cache2.get(key);
+
+        assert loaded == key;
+        assert store.map.containsKey(key);
+    }
+
+    /**
+     * Create and add test data via Streamer API.
+     *
+     * @param grid to get streamer.
+     * @return test object (it is key and val).
+     */
+    private TestObj streamData(final Ignite grid) {
+        final IgniteDataStreamer<TestObj, TestObj> streamer = grid.dataStreamer(CACHE_NAME);
+
+        TestObj entity = null;
+
+        for (int i = 0; i < 1; i++) {
+            entity = new TestObj(i);
+
+            streamer.addData(entity, entity);
+        }
+
+        streamer.flush();
+        streamer.close();
+        streamer.future().get();
+
+        return entity;
+    }
+
+    /**
+     * Create and add binary data via Streamer API.
+     *
+     * @param grid to get streamer.
+     * @return test object (it is key and val).
+     */
+    private BinaryObject streamBinaryData(final Ignite grid) {
+        final IgniteDataStreamer<BinaryObject, BinaryObject> streamer = grid.dataStreamer(CACHE_NAME);
+
+        streamer.keepBinary(true);
+
+        final BinaryObjectBuilder builder = grid.binary().builder("custom_type");
+
+        BinaryObject entity = null;
+
+        for (int i = 0; i < 1; i++) {
+            builder.setField("id", i);
+
+            entity = builder.build();
+
+            streamer.addData(entity, entity);
+        }
+
+        streamer.flush();
+        streamer.close();
+        streamer.future().get();
+
+        return entity;
+    }
+
+    /**
+     * Local store mock.
+     *
+     * @param <K>
+     * @param <V>
+     */
+    @CacheLocalStore
+    protected static class GridCacheLocalTestStore<K, V> extends CacheStoreAdapter<K, V> {
+        /** */
+        public final Map<K, V> map = new ConcurrentHashMap8<>();
+
+        /** {@inheritDoc} */
+        @Override public V load(final K key) throws CacheLoaderException {
+            return map.get(key);
+        }
+
+        /** {@inheritDoc} */
+        @Override public void write(final Cache.Entry<? extends K, ? extends V> entry) throws CacheWriterException {
+            map.put(entry.getKey(), entry.getValue());
+        }
+
+        /** {@inheritDoc} */
+        @Override public void delete(final Object key) throws CacheWriterException {
+            map.remove(key);
+        }
+    }
+
+    /**
+     * Test object.
+     */
+    static class TestObj implements Serializable {
+        /** */
+        Integer val;
+
+        /** */
+        public TestObj() {
+        }
+
+        /** */
+        public TestObj(final Integer val) {
+            this.val = val;
+        }
+
+        /** {@inheritDoc} */
+        @Override public boolean equals(final Object o) {
+            if (this == o) return true;
+            if (o == null || getClass() != o.getClass()) return false;
+
+            final TestObj testObj = (TestObj) o;
+
+            return val != null ? val.equals(testObj.val) : testObj.val == null;
+
+        }
+
+        /** {@inheritDoc} */
+        @Override public int hashCode() {
+            return val != null ? val.hashCode() : 0;
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/b46ed10a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridLocalCacheStoreManagerDeserializationTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridLocalCacheStoreManagerDeserializationTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridLocalCacheStoreManagerDeserializationTest.java
new file mode 100644
index 0000000..827b3cf
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridLocalCacheStoreManagerDeserializationTest.java
@@ -0,0 +1,101 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+
+package org.apache.ignite.internal.processors.cache;
+
+import org.apache.ignite.Ignite;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.binary.BinaryObject;
+import org.apache.ignite.binary.BinaryObjectBuilder;
+import org.apache.ignite.cache.CacheMode;
+import org.apache.ignite.internal.processors.cache.version.GridCacheVersion;
+
+import javax.cache.expiry.ExpiryPolicy;
+import java.util.UUID;
+
+/**
+ * Checks whether storing to local store doesn't cause binary objects unmarshalling,
+ * and as a consequence {@link ClassNotFoundException} to be thrown.
+ *
+ * @see <a href="https://issues.apache.org/jira/browse/IGNITE-2753">
+ *     https://issues.apache.org/jira/browse/IGNITE-2753
+ *     </a>
+ */
+public class GridLocalCacheStoreManagerDeserializationTest extends GridCacheStoreManagerDeserializationTest {
+    /** {@inheritDoc} */
+    @Override protected CacheMode cacheMode() {
+        return CacheMode.LOCAL;
+    }
+
+    /**
+     * Checks no additional unmarshalling happens in calling
+     * {@link GridCacheMapEntry#innerUpdateLocal(GridCacheVersion, GridCacheOperation, Object, Object[],
+     * boolean, boolean, boolean, boolean, ExpiryPolicy, boolean, boolean, CacheEntryPredicate[],
+     * boolean, UUID, String)}.
+     *
+     * @throws Exception
+     */
+    public void testUpdate() throws Exception {
+        // Goal is to check correct saving to store (no exception must be thrown)
+
+        final Ignite grid = startGrid();
+
+        final IgniteCache<TestObj, TestObj> cache = grid.createCache(CACHE_NAME);
+
+        final TestObj testObj = new TestObj(0);
+
+        cache.put(testObj, testObj);
+
+        assert testObj.equals(cache.get(testObj));
+        assert store.map.containsKey(testObj);
+
+        cache.remove(testObj);
+
+        assert cache.get(testObj) == null;
+        assert !store.map.containsKey(testObj);
+    }
+
+    /**
+     * Checks no additional unmarshalling happens in calling
+     * {@link GridCacheMapEntry#innerUpdateLocal(GridCacheVersion, GridCacheOperation, Object, Object[],
+     * boolean, boolean, boolean, boolean, ExpiryPolicy, boolean, boolean, CacheEntryPredicate[],
+     * boolean, UUID, String)} for binary objects.
+     *
+     * @throws Exception
+     */
+    public void testBinaryUpdate() throws Exception {
+        // Goal is to check correct saving to store (no exception must be thrown)
+        final Ignite grid = startGrid("binaryGrid");
+
+        final IgniteCache<BinaryObject, BinaryObject> cache = grid.createCache(CACHE_NAME).withKeepBinary();
+
+        final BinaryObjectBuilder builder = grid.binary().builder("custom_type");
+
+        final BinaryObject entity = builder.setField("id", 0).build();
+
+        cache.put(entity, entity);
+
+        assert entity.equals(cache.get(entity));
+        assert store.map.containsKey(entity);
+
+        cache.remove(entity);
+
+        assert cache.get(entity) == null;
+        assert !store.map.containsKey(entity);
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/b46ed10a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite4.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite4.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite4.java
index 45679dd..7aab990 100644
--- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite4.java
+++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite4.java
@@ -19,15 +19,14 @@ package org.apache.ignite.testsuites;
 
 import junit.framework.TestSuite;
 import org.apache.ignite.cache.store.jdbc.CacheJdbcStoreSessionListenerSelfTest;
+import org.apache.ignite.internal.processors.GridCacheTxLoadFromStoreOnLockSelfTest;
+import org.apache.ignite.internal.processors.cache.CacheClientStoreSelfTest;
 import org.apache.ignite.internal.processors.cache.CacheGetEntryOptimisticReadCommittedSeltTest;
 import org.apache.ignite.internal.processors.cache.CacheGetEntryOptimisticRepeatableReadSeltTest;
 import org.apache.ignite.internal.processors.cache.CacheGetEntryOptimisticSerializableSeltTest;
 import org.apache.ignite.internal.processors.cache.CacheGetEntryPessimisticReadCommittedSeltTest;
 import org.apache.ignite.internal.processors.cache.CacheGetEntryPessimisticRepeatableReadSeltTest;
 import org.apache.ignite.internal.processors.cache.CacheGetEntryPessimisticSerializableSeltTest;
-import org.apache.ignite.internal.processors.cache.IgniteCacheGetCustomCollectionsSelfTest;
-import org.apache.ignite.internal.processors.GridCacheTxLoadFromStoreOnLockSelfTest;
-import org.apache.ignite.internal.processors.cache.CacheClientStoreSelfTest;
 import org.apache.ignite.internal.processors.cache.CacheOffheapMapEntrySelfTest;
 import org.apache.ignite.internal.processors.cache.CacheReadThroughAtomicRestartSelfTest;
 import org.apache.ignite.internal.processors.cache.CacheReadThroughLocalAtomicRestartSelfTest;
@@ -49,7 +48,9 @@ import org.apache.ignite.internal.processors.cache.GridCacheMultinodeUpdateAtomi
 import org.apache.ignite.internal.processors.cache.GridCacheMultinodeUpdateNearEnabledNoBackupsSelfTest;
 import org.apache.ignite.internal.processors.cache.GridCacheMultinodeUpdateNearEnabledSelfTest;
 import org.apache.ignite.internal.processors.cache.GridCacheMultinodeUpdateSelfTest;
+import org.apache.ignite.internal.processors.cache.GridCacheStoreManagerDeserializationTest;
 import org.apache.ignite.internal.processors.cache.GridCacheVersionMultinodeTest;
+import org.apache.ignite.internal.processors.cache.GridLocalCacheStoreManagerDeserializationTest;
 import org.apache.ignite.internal.processors.cache.IgniteCacheAtomicCopyOnReadDisabledTest;
 import org.apache.ignite.internal.processors.cache.IgniteCacheAtomicLocalPeekModesTest;
 import org.apache.ignite.internal.processors.cache.IgniteCacheAtomicLocalStoreValueTest;
@@ -63,6 +64,7 @@ import org.apache.ignite.internal.processors.cache.IgniteCacheAtomicStoreValueTe
 import org.apache.ignite.internal.processors.cache.IgniteCacheConfigurationDefaultTemplateTest;
 import org.apache.ignite.internal.processors.cache.IgniteCacheConfigurationTemplateTest;
 import org.apache.ignite.internal.processors.cache.IgniteCacheDynamicStopSelfTest;
+import org.apache.ignite.internal.processors.cache.IgniteCacheGetCustomCollectionsSelfTest;
 import org.apache.ignite.internal.processors.cache.IgniteCacheInvokeReadThroughTest;
 import org.apache.ignite.internal.processors.cache.IgniteCacheLoadRebalanceEvictionSelfTest;
 import org.apache.ignite.internal.processors.cache.IgniteCacheTxCopyOnReadDisabledTest;
@@ -266,6 +268,9 @@ public class IgniteCacheTestSuite4 extends TestSuite {
         suite.addTestSuite(CacheStoreUsageMultinodeDynamicStartAtomicTest.class);
         suite.addTestSuite(CacheStoreUsageMultinodeDynamicStartTxTest.class);
 
+        suite.addTestSuite(GridCacheStoreManagerDeserializationTest.class);
+        suite.addTestSuite(GridLocalCacheStoreManagerDeserializationTest.class);
+
         suite.addTestSuite(IgniteStartCacheInTransactionSelfTest.class);
         suite.addTestSuite(IgniteStartCacheInTransactionAtomicSelfTest.class);
 


[2/2] ignite git commit: Merge remote-tracking branch 'origin/master'

Posted by vo...@apache.org.
Merge remote-tracking branch 'origin/master'


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

Branch: refs/heads/master
Commit: 8cf71d463d9f2baa7651286a6524ec5d6d286b5b
Parents: b46ed10 648b10c
Author: vozerov-gridgain <vo...@gridgain.com>
Authored: Thu Mar 10 14:05:53 2016 +0300
Committer: vozerov-gridgain <vo...@gridgain.com>
Committed: Thu Mar 10 14:05:53 2016 +0300

----------------------------------------------------------------------
 .../install-package.cmd                         | 11 ++-
 .../Apache.Ignite.Core.nuspec                   |  6 +-
 .../NuGet/LINQPad/ComputeExample.linq           |  2 +-
 .../NuGet/LINQPad/PutGetExample.linq            |  2 +-
 .../NuGet/LINQPad/QueryExample.linq             |  2 +-
 modules/web/ignite-weblogic-test/pom.xml        | 74 +++++++-------------
 .../src/main/webapp/WEB-INF/web.xml             | 17 -----
 .../src/main/webapp/index.jsp                   | 17 -----
 8 files changed, 40 insertions(+), 91 deletions(-)
----------------------------------------------------------------------