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/08/02 18:50:26 UTC

[1/3] ignite git commit: IGNITE-5542 CacheGroup configuration from cluster is merged with local settings

Repository: ignite
Updated Branches:
  refs/heads/ignite-5578 1091daaa5 -> 48330d36c


IGNITE-5542 CacheGroup configuration from cluster is merged with local settings


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

Branch: refs/heads/ignite-5578
Commit: 88818ecc5ad882eb7a6b0b1bb4a7b1131b724a78
Parents: b277682
Author: Sergey Chugunov <se...@gmail.com>
Authored: Wed Aug 2 18:14:46 2017 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Wed Aug 2 18:14:46 2017 +0300

----------------------------------------------------------------------
 .../processors/cache/CacheGroupDescriptor.java  |  20 ++-
 .../processors/cache/ClusterCachesInfo.java     |  25 +++
 .../processors/cache/GridCacheAttributes.java   |  35 +++++
 .../processors/cache/GridCacheUtils.java        |   1 +
 .../CacheGroupLocalConfigurationSelfTest.java   | 155 +++++++++++++++++++
 .../testsuites/IgniteCacheTestSuite2.java       |   2 +
 6 files changed, 237 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/88818ecc/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheGroupDescriptor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheGroupDescriptor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheGroupDescriptor.java
index c4976e5..20301a6 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheGroupDescriptor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheGroupDescriptor.java
@@ -83,7 +83,7 @@ public class CacheGroupDescriptor {
         this.rcvdFrom = rcvdFrom;
         this.startTopVer = startTopVer;
         this.deploymentId = deploymentId;
-        this.cacheCfg = cacheCfg;
+        this.cacheCfg = new CacheConfiguration<>(cacheCfg);
         this.caches = caches;
     }
 
@@ -197,6 +197,24 @@ public class CacheGroupDescriptor {
     }
 
     /**
+     * Method to merge this CacheGroup descriptor with another one.
+     *
+     * @param otherDesc CacheGroup descriptor that must be merged with this one.
+     */
+    void mergeWith(CacheGroupDescriptor otherDesc) {
+        assert otherDesc != null && otherDesc.config() != null: otherDesc;
+
+        CacheConfiguration otherCfg = otherDesc.config();
+
+        cacheCfg.setRebalanceDelay(otherCfg.getRebalanceDelay());
+        cacheCfg.setRebalanceBatchesPrefetchCount(otherCfg.getRebalanceBatchesPrefetchCount());
+        cacheCfg.setRebalanceBatchSize(otherCfg.getRebalanceBatchSize());
+        cacheCfg.setRebalanceOrder(otherCfg.getRebalanceOrder());
+        cacheCfg.setRebalanceThrottle(otherCfg.getRebalanceThrottle());
+        cacheCfg.setRebalanceTimeout(otherCfg.getRebalanceTimeout());
+    }
+
+    /**
      * @return Start version for dynamically started group.
      */
     @Nullable public AffinityTopologyVersion startTopologyVersion() {

http://git-wip-us.apache.org/repos/asf/ignite/blob/88818ecc/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ClusterCachesInfo.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ClusterCachesInfo.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ClusterCachesInfo.java
index 1a05b96..bb51a3b 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ClusterCachesInfo.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ClusterCachesInfo.java
@@ -285,6 +285,22 @@ class ClusterCachesInfo {
             CU.checkAttributeMismatch(log, rmtAttr.cacheName(), rmt, "preloadBatchSize",
                 "Preload batch size", locAttr.rebalanceBatchSize(), rmtAttr.rebalanceBatchSize(), false);
 
+            CU.checkAttributeMismatch(log, rmtAttr.cacheName(), rmt, "rebalanceDelay",
+                "Rebalance delay", locAttr.rebalanceDelay(), rmtAttr.rebalanceDelay(), false);
+
+            CU.checkAttributeMismatch(log, rmtAttr.cacheName(), rmt, "rebalanceBatchesPrefetchCount",
+                "Rebalance batches prefetch count", locAttr.rebalanceBatchesPrefetchCount(),
+                rmtAttr.rebalanceBatchesPrefetchCount(), false);
+
+            CU.checkAttributeMismatch(log, rmtAttr.cacheName(), rmt, "rebalanceOrder",
+                "Rebalance order", locAttr.rebalanceOrder(), rmtAttr.rebalanceOrder(), false);
+
+            CU.checkAttributeMismatch(log, rmtAttr.cacheName(), rmt, "rebalanceThrottle",
+                "Rebalance throttle", locAttr.rebalanceThrottle(), rmtAttr.rebalanceThrottle(), false);
+
+            CU.checkAttributeMismatch(log, rmtAttr.cacheName(), rmt, "rebalanceTimeout",
+                "Rebalance timeout", locAttr.rebalanceTimeout(), rmtAttr.rebalanceTimeout(), false);
+
             CU.checkAttributeMismatch(log, rmtAttr.cacheName(), rmt, "writeSynchronizationMode",
                 "Write synchronization mode", locAttr.writeSynchronization(), rmtAttr.writeSynchronization(),
                 true);
@@ -941,6 +957,9 @@ class ClusterCachesInfo {
 
         CacheNodeCommonDiscoveryData cachesData = (CacheNodeCommonDiscoveryData)data.commonData();
 
+        // CacheGroup configurations that were created from local node configuration.
+        Map<Integer, CacheGroupDescriptor> locCacheGrps = new HashMap<>(registeredCacheGroups());
+
         // Replace locally registered data with actual data received from cluster.
         registeredCaches.clear();
         registeredCacheGrps.clear();
@@ -956,6 +975,12 @@ class ClusterCachesInfo {
                 grpData.deploymentId(),
                 grpData.caches());
 
+            if (locCacheGrps.containsKey(grpDesc.groupId())) {
+                CacheGroupDescriptor locGrpCfg = locCacheGrps.get(grpDesc.groupId());
+
+                grpDesc.mergeWith(locGrpCfg);
+            }
+
             CacheGroupDescriptor old = registeredCacheGrps.put(grpDesc.groupId(), grpDesc);
 
             assert old == null : old;

http://git-wip-us.apache.org/repos/asf/ignite/blob/88818ecc/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAttributes.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAttributes.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAttributes.java
index 32871ea..d64ee8b 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAttributes.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAttributes.java
@@ -203,6 +203,41 @@ public class GridCacheAttributes implements Serializable {
     }
 
     /**
+     * @return Rebalance delay.
+     */
+    public long rebalanceDelay() {
+        return ccfg.getRebalanceDelay();
+    }
+
+    /**
+     * @return Rebalance prefetch count.
+     */
+    public long rebalanceBatchesPrefetchCount() {
+        return ccfg.getRebalanceBatchesPrefetchCount();
+    }
+
+    /**
+     * @return Rebalance order.
+     */
+    public int rebalanceOrder() {
+        return ccfg.getRebalanceOrder();
+    }
+
+    /**
+     * @return Rebalance throttle.
+     */
+    public long rebalanceThrottle() {
+        return ccfg.getRebalanceThrottle();
+    }
+
+    /**
+     * @return Rebalance timeout.
+     */
+    public long rebalanceTimeout() {
+        return ccfg.getRebalanceTimeout();
+    }
+
+    /**
      * @return Synchronization mode.
      */
     public CacheWriteSynchronizationMode writeSynchronization() {

http://git-wip-us.apache.org/repos/asf/ignite/blob/88818ecc/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheUtils.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheUtils.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheUtils.java
index a07b166..f0b8170 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheUtils.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheUtils.java
@@ -992,6 +992,7 @@ public class GridCacheUtils {
             }
         }
     }
+
     /**
      * @param cfg1 Existing configuration.
      * @param cfg2 Cache configuration to start.

http://git-wip-us.apache.org/repos/asf/ignite/blob/88818ecc/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheGroupLocalConfigurationSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheGroupLocalConfigurationSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheGroupLocalConfigurationSelfTest.java
new file mode 100644
index 0000000..51f9001
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheGroupLocalConfigurationSelfTest.java
@@ -0,0 +1,155 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.ignite.internal.processors.cache;
+
+import java.util.Map;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.internal.IgniteKernal;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ *
+ */
+public class CacheGroupLocalConfigurationSelfTest extends GridCommonAbstractTest {
+    /** */
+    private static final String SECOND_NODE_NAME = "secondNode";
+
+    /** */
+    private static final int NON_STANDARD_REBALANCE_VALUE = 101;
+
+    /** */
+    private static final String NON_DEFAULT_GROUP_NAME = "cacheGroup";
+
+    /** */
+    private boolean useNonDfltCacheGrp = true;
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(igniteInstanceName);
+
+        if (igniteInstanceName.equals(SECOND_NODE_NAME)) {
+            CacheConfiguration ccfg = new CacheConfiguration()
+                .setName(DEFAULT_CACHE_NAME)
+                .setRebalanceDelay(NON_STANDARD_REBALANCE_VALUE)
+                .setRebalanceBatchesPrefetchCount(NON_STANDARD_REBALANCE_VALUE)
+                .setRebalanceBatchSize(NON_STANDARD_REBALANCE_VALUE)
+                .setRebalanceOrder(NON_STANDARD_REBALANCE_VALUE)
+                .setRebalanceThrottle(NON_STANDARD_REBALANCE_VALUE)
+                .setRebalanceTimeout(NON_STANDARD_REBALANCE_VALUE);
+
+            if (useNonDfltCacheGrp)
+                ccfg.setGroupName(NON_DEFAULT_GROUP_NAME);
+
+            cfg.setCacheConfiguration(ccfg);
+        }
+        else {
+            CacheConfiguration ccfg = new CacheConfiguration()
+                .setName(DEFAULT_CACHE_NAME);
+
+            if (useNonDfltCacheGrp)
+                ccfg.setGroupName(NON_DEFAULT_GROUP_NAME);
+
+            cfg.setCacheConfiguration(ccfg);
+        }
+
+
+        return cfg;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTest() throws Exception {
+        stopAllGrids();
+    }
+
+    /**
+     * Test validates that all cache group configuration attributes from local config
+     * that must not be overwritten by grid config are preserved for default cache group.
+     *
+     * @throws Exception If failed.
+     */
+    public void testDefaultGroupLocalAttributesPreserved() throws Exception {
+        useNonDfltCacheGrp = false;
+
+        executeTest();
+    }
+
+    /**
+     * Test validates that all cache group configuration attributes from local config
+     * that must not be overwritten by grid config are preserved for non-default cache group.
+     *
+     * @throws Exception If failed.
+     */
+    public void testNonDefaultGroupLocalAttributesPreserved() throws Exception {
+        useNonDfltCacheGrp = true;
+
+        executeTest();
+    }
+
+    /**
+     * Executes actual test logic.
+     *
+     * @throws Exception If failed.
+     */
+    private void executeTest() throws Exception {
+        startGrid(0);
+
+        IgniteKernal ignite = (IgniteKernal) startGrid("secondNode");
+
+        GridCacheProcessor cacheProc = ignite.context().cache();
+
+        Map<Integer, CacheGroupContext> cacheGrps = U.field(cacheProc, "cacheGrps");
+
+        CacheConfiguration cacheGroupCfg = findGroupConfig(cacheGrps,
+            useNonDfltCacheGrp ? NON_DEFAULT_GROUP_NAME : DEFAULT_CACHE_NAME);
+
+        assertNotNull("Default cache group must be presented", cacheGroupCfg);
+
+        assertEquals("Rebalance delay", cacheGroupCfg.getRebalanceDelay(), NON_STANDARD_REBALANCE_VALUE);
+
+        assertEquals("Rebalance batches prefetch count",
+            cacheGroupCfg.getRebalanceBatchesPrefetchCount(),
+            NON_STANDARD_REBALANCE_VALUE);
+
+        assertEquals("Rebalance batch size", cacheGroupCfg.getRebalanceBatchSize(), NON_STANDARD_REBALANCE_VALUE);
+
+        assertEquals("Rebalance order", cacheGroupCfg.getRebalanceOrder(), NON_STANDARD_REBALANCE_VALUE);
+
+        assertEquals("Rebalance throttle", cacheGroupCfg.getRebalanceThrottle(), NON_STANDARD_REBALANCE_VALUE);
+
+        assertEquals("Rebalance timeout", cacheGroupCfg.getRebalanceTimeout(), NON_STANDARD_REBALANCE_VALUE);
+    }
+
+    /**
+     * @param cacheGrps All configured cache groups.
+     * @param groupName Name of group to find.
+     * @return Cache configuration.
+     */
+    private CacheConfiguration findGroupConfig(Map<Integer, CacheGroupContext> cacheGrps, @Nullable String groupName) {
+        if (groupName == null)
+            groupName = DEFAULT_CACHE_NAME;
+
+        for (CacheGroupContext grpCtx : cacheGrps.values()) {
+            if (groupName.equals(grpCtx.cacheOrGroupName()))
+                return grpCtx.config();
+        }
+
+        return null;
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/88818ecc/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite2.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite2.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite2.java
index 9ed7ee3..513f909 100644
--- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite2.java
+++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite2.java
@@ -28,6 +28,7 @@ import org.apache.ignite.internal.processors.cache.CacheDhtLocalPartitionAfterRe
 import org.apache.ignite.internal.processors.cache.CacheEnumOperationsSingleNodeTest;
 import org.apache.ignite.internal.processors.cache.CacheEnumOperationsTest;
 import org.apache.ignite.internal.processors.cache.CacheExchangeMessageDuplicatedStateTest;
+import org.apache.ignite.internal.processors.cache.CacheGroupLocalConfigurationSelfTest;
 import org.apache.ignite.internal.processors.cache.CacheMemoryPolicyConfigurationTest;
 import org.apache.ignite.internal.processors.cache.CacheOptimisticTransactionsWithFilterSingleServerTest;
 import org.apache.ignite.internal.processors.cache.CacheOptimisticTransactionsWithFilterTest;
@@ -259,6 +260,7 @@ public class IgniteCacheTestSuite2 extends TestSuite {
         suite.addTest(new TestSuite(MemoryPolicyConfigValidationTest.class));
         suite.addTest(new TestSuite(MemoryPolicyInitializationTest.class));
         suite.addTest(new TestSuite(CacheMemoryPolicyConfigurationTest.class));
+        suite.addTest(new TestSuite(CacheGroupLocalConfigurationSelfTest.class));
         suite.addTest(new TestSuite(CacheEnumOperationsSingleNodeTest.class));
         suite.addTest(new TestSuite(CacheEnumOperationsTest.class));
         suite.addTest(new TestSuite(IgniteCacheIncrementTxTest.class));


[3/3] ignite git commit: 5578

Posted by sb...@apache.org.
5578


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

Branch: refs/heads/ignite-5578
Commit: 48330d36cf66e959016a513e6a1e1561490865dc
Parents: 509afde
Author: sboikov <sb...@gridgain.com>
Authored: Wed Aug 2 21:50:13 2017 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Wed Aug 2 21:50:13 2017 +0300

----------------------------------------------------------------------
 .../processors/cache/ExchangeContext.java       |  3 ++
 .../distributed/dht/GridDhtTopologyFuture.java  | 32 ++++++++++++++++----
 .../GridDhtPartitionsExchangeFuture.java        |  8 +++--
 3 files changed, 34 insertions(+), 9 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/48330d36/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ExchangeContext.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ExchangeContext.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ExchangeContext.java
index 0916961..ab0ff1e 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ExchangeContext.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ExchangeContext.java
@@ -116,6 +116,9 @@ public class ExchangeContext {
         return requestGrpsAffOnJoin;
     }
 
+    /**
+     * @return {@code True} if exchanges merge is allowed during current exchange.
+     */
     public boolean mergeExchanges() {
         return merge;
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/48330d36/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTopologyFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTopologyFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTopologyFuture.java
index 59ce1dd..cc12960 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTopologyFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTopologyFuture.java
@@ -22,6 +22,9 @@ import org.apache.ignite.cache.affinity.Affinity;
 import org.apache.ignite.internal.IgniteInternalFuture;
 import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
 import org.apache.ignite.internal.processors.cache.GridCacheContext;
+import org.apache.ignite.internal.processors.cache.GridCachePartitionExchangeManager;
+import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionsExchangeFuture;
+import org.apache.ignite.internal.util.future.GridFutureAdapter;
 import org.jetbrains.annotations.Nullable;
 
 /**
@@ -29,22 +32,39 @@ import org.jetbrains.annotations.Nullable;
  * safe to use when all transactions that involve moving primary partitions are completed and partition map
  * exchange is also completed.
  * <p/>
- * When new new transaction is started, it will wait for this future before acquiring new locks on particular
+ * When new new cache operation is started, it will wait for this future before acquiring new locks on particular
  * topology version.
  */
 public interface GridDhtTopologyFuture extends IgniteInternalFuture<AffinityTopologyVersion> {
     /**
-     * Gets topology version of this future.
+     * Returns topology version when exchange started. It can differ from result topology version if exchanges for
+     * multiple discovery events are merged. Initial version should not be used as version for cache operation
+     * since it is possible affinity for this version is never calculated.
      *
-     * @return Topology version.
+     * @return Topology version when exchange started.
      */
-    public AffinityTopologyVersion topologyVersion();
+    public AffinityTopologyVersion initialVersion();
 
     /**
-     * @return Version when exchange started.
+     * Gets result topology version of this future. Result version can differ from initial exchange version
+     * if excanges for multiple discovery events are merged, in this case result version is version of last
+     * discovery event.
+     * <p>
+     * This method should be called only for finished topology future
+     * since result version is not known before exchange finished.
+     *
+     * @return Result topology version.
      */
-    public AffinityTopologyVersion initialVersion();
+    public AffinityTopologyVersion topologyVersion();
 
+    /**
+     * Ready affinity future ({@link GridCachePartitionExchangeManager#affinityReadyFuture(AffinityTopologyVersion)}
+     * is completed before {@link GridFutureAdapter#onDone(Object, Throwable)} is called on
+     * {@link GridDhtPartitionsExchangeFuture}, it is guaranteed that this method will return {@code true}
+     * if affinity ready future is finished.
+     *
+     * @return {@code True} if exchange is finished and result topology version can be used.
+     */
     public boolean exchangeDone();
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/48330d36/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsExchangeFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsExchangeFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsExchangeFuture.java
index cf8d15d..df72d30 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsExchangeFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsExchangeFuture.java
@@ -1884,10 +1884,12 @@ public class GridDhtPartitionsExchangeFuture extends GridDhtTopologyFutureAdapte
     /**
      * @return {@code False} if interrupted.
      */
-    private synchronized boolean awaitSingleMapUpdates() {
+    private boolean awaitSingleMapUpdates() {
         try {
-            while (pendingSingleUpdates > 0)
-                U.wait(mux);
+            synchronized (mux) {
+                while (pendingSingleUpdates > 0)
+                    U.wait(mux);
+            }
 
             return true;
         }


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

Posted by sb...@apache.org.
Merge remote-tracking branch 'remotes/origin/master' into ignite-5578


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

Branch: refs/heads/ignite-5578
Commit: 509afde0b3f886745d212e07383125b317e8404b
Parents: 1091daa 88818ec
Author: sboikov <sb...@gridgain.com>
Authored: Wed Aug 2 21:26:44 2017 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Wed Aug 2 21:26:44 2017 +0300

----------------------------------------------------------------------
 .../processors/cache/CacheGroupDescriptor.java  |  20 ++-
 .../processors/cache/ClusterCachesInfo.java     |  25 +++
 .../processors/cache/GridCacheAttributes.java   |  35 +++++
 .../processors/cache/GridCacheUtils.java        |   1 +
 .../CacheGroupLocalConfigurationSelfTest.java   | 155 +++++++++++++++++++
 .../testsuites/IgniteCacheTestSuite2.java       |   2 +
 6 files changed, 237 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/509afde0/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite2.java
----------------------------------------------------------------------