You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by sb...@apache.org on 2017/12/11 07:49:38 UTC

[2/6] ignite git commit: IGNITE-7085 When PDS is enabled and IGNITE_BINARY_SORT_OBJECT_FIELDS is set and IgniteCache#put is called node hangs - Fixes #3121.

IGNITE-7085 When PDS is enabled and IGNITE_BINARY_SORT_OBJECT_FIELDS is set and IgniteCache#put is called node hangs - Fixes #3121.

Signed-off-by: Alexey Goncharuk <al...@gmail.com>


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

Branch: refs/heads/ignite-zk
Commit: 3c20b0e4e32443977cbc18b61ef03d94b1cd5407
Parents: c769838
Author: Alexander Fedotov <al...@gmail.com>
Authored: Fri Dec 8 15:32:27 2017 +0300
Committer: Alexey Goncharuk <al...@gmail.com>
Committed: Fri Dec 8 15:32:27 2017 +0300

----------------------------------------------------------------------
 .../internal/binary/BinaryClassDescriptor.java  |   3 +-
 .../ignite/internal/binary/BinaryContext.java   |   1 +
 .../IgnitePdsBinarySortObjectFieldsTest.java    | 150 +++++++++++++++++++
 .../ignite/testframework/GridTestUtils.java     |  41 +++++
 .../IgnitePdsWithIndexingCoreTestSuite.java     |   2 +
 5 files changed, 196 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/3c20b0e4/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryClassDescriptor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryClassDescriptor.java b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryClassDescriptor.java
index f9be1db..2056566 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryClassDescriptor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryClassDescriptor.java
@@ -741,7 +741,8 @@ public class BinaryClassDescriptor {
                         postWrite(writer);
 
                         // Check whether we need to update metadata.
-                        if (obj.getClass() != BinaryMetadata.class) {
+                        // The reason for this check is described in https://issues.apache.org/jira/browse/IGNITE-7138.
+                        if (obj.getClass() != BinaryMetadata.class && obj.getClass() != BinaryTreeMap.class) {
                             int schemaId = writer.schemaId();
 
                             if (schemaReg.schema(schemaId) == null) {

http://git-wip-us.apache.org/repos/asf/ignite/blob/3c20b0e4/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryContext.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryContext.java b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryContext.java
index dd192bf..5be1d39 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryContext.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryContext.java
@@ -342,6 +342,7 @@ public class BinaryContext {
         registerPredefinedType(BinaryMetadataKey.class, 0);
         registerPredefinedType(BinaryMetadata.class, 0);
         registerPredefinedType(BinaryEnumObjectImpl.class, 0);
+        registerPredefinedType(BinaryTreeMap.class, 0);
 
         registerPredefinedType(PlatformDotNetSessionData.class, 0);
         registerPredefinedType(PlatformDotNetSessionLockResult.class, 0);

http://git-wip-us.apache.org/repos/asf/ignite/blob/3c20b0e4/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgnitePdsBinarySortObjectFieldsTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgnitePdsBinarySortObjectFieldsTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgnitePdsBinarySortObjectFieldsTest.java
new file mode 100644
index 0000000..7c8981e
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgnitePdsBinarySortObjectFieldsTest.java
@@ -0,0 +1,150 @@
+/*
+ * 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.persistence;
+
+import java.util.concurrent.TimeUnit;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.configuration.PersistentStoreConfiguration;
+import org.apache.ignite.internal.IgniteEx;
+import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi;
+import org.apache.ignite.spi.discovery.tcp.ipfinder.TcpDiscoveryIpFinder;
+import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder;
+import org.apache.ignite.testframework.GridTestUtils;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+
+/**
+ *
+ */
+public class IgnitePdsBinarySortObjectFieldsTest extends GridCommonAbstractTest {
+    /** */
+    private static final String CACHE_NAME = "ignitePdsBinarySortObjectFieldsTestCache";
+
+    /**
+     * Value.
+     */
+    public static class Value {
+        /** */
+        private Long val;
+
+        /**
+         * Default constructor.
+         */
+        public Value() {
+            // No-op.
+        }
+
+        /**
+         * @param val Value.
+         */
+        public Value(Long val) {
+            this.val = val;
+        }
+
+        /**
+         * Returns the value.
+         *
+         * @return Value.
+         */
+        public Long getVal() {
+            return val;
+        }
+
+        /**
+         * Sets the value.
+         *
+         * @param val Value.
+         */
+        public void setVal(Long val) {
+            this.val = val;
+        }
+
+        /** {@inheritDoc} */
+        @Override public String toString() {
+            return "Value [val=" + val + ']';
+        }
+    }
+
+    /** */
+    private static final TcpDiscoveryIpFinder IP_FINDER = new TcpDiscoveryVmIpFinder(true);
+
+    /** {@inheritDoc} */
+    @Override protected void afterTestsStopped() throws Exception {
+        GridTestUtils.deleteDbFiles();
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTest() throws Exception {
+        super.beforeTest();
+
+        GridTestUtils.deleteDbFiles();
+    }
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(gridName);
+
+        cfg.setConsistentId(gridName);
+
+        ((TcpDiscoverySpi)cfg.getDiscoverySpi()).setIpFinder(IP_FINDER);
+
+        cfg.setPersistentStoreConfiguration(new PersistentStoreConfiguration());
+
+        return cfg;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTest() throws Exception {
+        stopAllGrids();
+
+        GridTestUtils.deleteDbFiles();
+
+        super.afterTest();
+    }
+
+    /**
+     * @throws Exception if failed.
+     */
+    public void testGivenCacheWithPojoValueAndPds_WhenPut_ThenNoHangup() throws Exception {
+        System.setProperty("IGNITE_BINARY_SORT_OBJECT_FIELDS", "true");
+
+        GridTestUtils.assertTimeout(5, TimeUnit.SECONDS, new Runnable() {
+            @Override public void run() {
+                IgniteEx ignite;
+
+                try {
+                    ignite = startGrid(0);
+                }
+                catch (Exception e) {
+                    throw new RuntimeException(e);
+                }
+
+                ignite.active(true);
+
+                IgniteCache<Long, Value> cache = ignite.getOrCreateCache(
+                    new CacheConfiguration<Long, Value>(CACHE_NAME)
+                );
+
+                cache.put(1L, new Value(1L));
+
+                assertEquals(1, cache.size());
+            }
+        });
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/3c20b0e4/modules/core/src/test/java/org/apache/ignite/testframework/GridTestUtils.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testframework/GridTestUtils.java b/modules/core/src/test/java/org/apache/ignite/testframework/GridTestUtils.java
index 380284f..53ca516 100644
--- a/modules/core/src/test/java/org/apache/ignite/testframework/GridTestUtils.java
+++ b/modules/core/src/test/java/org/apache/ignite/testframework/GridTestUtils.java
@@ -52,6 +52,11 @@ import java.util.concurrent.Callable;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.ConcurrentLinkedQueue;
 import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
 import java.util.concurrent.atomic.AtomicBoolean;
 import javax.cache.CacheException;
 import javax.cache.configuration.Factory;
@@ -453,6 +458,42 @@ public final class GridTestUtils {
     }
 
     /**
+     * Asserts that the specified runnable completes within the specified timeout.
+     *
+     * @param msg Assertion message in case of timeout.
+     * @param timeout Timeout.
+     * @param timeUnit Timeout {@link TimeUnit}.
+     * @param runnable {@link Runnable} to check.
+     * @throws Exception In case of any exception distinct from {@link TimeoutException}.
+     */
+    public static void assertTimeout(String msg, long timeout, TimeUnit timeUnit, Runnable runnable) throws Exception {
+        ExecutorService executorSvc = Executors.newSingleThreadExecutor();
+        Future<?> fut = executorSvc.submit(runnable);
+
+        try {
+            fut.get(timeout, timeUnit);
+        }
+        catch (TimeoutException ignored) {
+            fail(msg, null);
+        }
+        finally {
+            executorSvc.shutdownNow();
+        }
+    }
+
+    /**
+     * Asserts that the specified runnable completes within the specified timeout.
+     *
+     * @param timeout Timeout.
+     * @param timeUnit Timeout {@link TimeUnit}.
+     * @param runnable {@link Runnable} to check.
+     * @throws Exception In case of any exception distinct from {@link TimeoutException}.
+     */
+    public static void assertTimeout(long timeout, TimeUnit timeUnit, Runnable runnable) throws Exception {
+        assertTimeout("Timeout occurred.", timeout, timeUnit, runnable);
+    }
+
+    /**
      * Throw assertion error with specified error message and initialized cause.
      *
      * @param msg Error message.

http://git-wip-us.apache.org/repos/asf/ignite/blob/3c20b0e4/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgnitePdsWithIndexingCoreTestSuite.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgnitePdsWithIndexingCoreTestSuite.java b/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgnitePdsWithIndexingCoreTestSuite.java
index ea30ce2..8308fd3 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgnitePdsWithIndexingCoreTestSuite.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgnitePdsWithIndexingCoreTestSuite.java
@@ -19,6 +19,7 @@ package org.apache.ignite.testsuites;
 import junit.framework.TestSuite;
 import org.apache.ignite.internal.processors.cache.persistence.IgnitePdsAtomicCacheRebalancingTest;
 import org.apache.ignite.internal.processors.cache.persistence.IgnitePdsBinaryMetadataOnClusterRestartTest;
+import org.apache.ignite.internal.processors.cache.persistence.IgnitePdsBinarySortObjectFieldsTest;
 import org.apache.ignite.internal.processors.cache.persistence.IgnitePdsMarshallerMappingRestoreOnNodeStartTest;
 import org.apache.ignite.internal.processors.cache.persistence.IgnitePdsTxCacheRebalancingTest;
 import org.apache.ignite.internal.processors.cache.persistence.IgnitePersistentStoreCacheGroupsTest;
@@ -60,6 +61,7 @@ public class IgnitePdsWithIndexingCoreTestSuite extends TestSuite {
         suite.addTestSuite(IgnitePdsBinaryMetadataOnClusterRestartTest.class);
         suite.addTestSuite(IgnitePdsMarshallerMappingRestoreOnNodeStartTest.class);
         suite.addTestSuite(IgnitePdsThreadInterruptionTest.class);
+        suite.addTestSuite(IgnitePdsBinarySortObjectFieldsTest.class);
 
         return suite;
     }