This is an automated email from the ASF dual-hosted git repository. dgovorukhin pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/ignite.git
The following commit(s) were added to refs/heads/master by this push: new 9a3de134 IGNITE-8571 Added Baseline auto-adjust feature - Fixes #5907. 9a3de134 is described below commit 9a3de134a5dfdb4385d0f5b8f7b1b16eda93b604 Author: Anton Kalashnikov <kaa....@yandex.ru> AuthorDate: Tue Feb 19 11:35:14 2019 +0300 IGNITE-8571 Added Baseline auto-adjust feature - Fixes #5907. Signed-off-by: Dmitriy Govorukhin <dmitriy.govoruk...@gmail.com> --- .../rest/JettyRestProcessorBaselineSelfTest.java | 10 + .../org/apache/ignite/IgniteSystemProperties.java | 5 + .../ignite/configuration/IgniteConfiguration.java | 77 ----- .../cluster/DistributedBaselineConfiguration.java | 76 +++-- .../ignite/internal/cluster/IgniteClusterImpl.java | 21 +- .../cache/GridCachePartitionExchangeManager.java | 4 +- .../dht/GridDhtTopologyFutureAdapter.java | 8 + .../preloader/GridDhtPartitionsExchangeFuture.java | 3 +- .../cluster/BaselineAdjustForbiddenException.java | 41 +++ .../cluster/ChangeGlobalStateFinishMessage.java | 5 +- .../cluster/ChangeGlobalStateMessage.java | 4 +- .../processors/cluster/ClusterProcessor.java | 10 +- .../cluster/GridClusterStateProcessor.java | 27 +- .../autoadjust/BaselineAutoAdjustData.java | 75 +++++ .../autoadjust/BaselineAutoAdjustExecutor.java | 84 +++++ .../autoadjust/BaselineAutoAdjustScheduler.java | 63 ++++ .../baseline/autoadjust/ChangeTopologyWatcher.java | 124 ++++++++ .../distributed/DistributedBooleanProperty.java | 2 +- .../DistributedConfigurationProcessor.java | 4 +- .../distributed/DistributedLongProperty.java | 2 +- .../distributed/DistributedProperty.java | 47 ++- ...erty.java => NotWritablePropertyException.java} | 22 +- .../distributed/PropertyUpdateClosure.java} | 23 +- .../metastorage/DistributedMetaStorage.java | 26 ++ .../persistence/DistributedMetaStorageImpl.java | 60 ++-- .../platform/utils/PlatformConfigurationUtils.java | 12 - .../apache/ignite/internal/util/IgniteUtils.java | 13 + .../ClusterBaselineNodesMetricsSelfTest.java | 16 + ...usterActivateDeactivateTestWithPersistence.java | 15 + ...teDynamicCacheStartFailWithPersistenceTest.java | 6 + .../distributed/CacheBaselineTopologyTest.java | 15 + .../CacheDataLossOnPartitionMoveTest.java | 15 + .../distributed/CacheRentingStateRepairTest.java | 15 + .../GridCacheRebalancingWithAsyncClearingTest.java | 16 + .../processors/cache/mvcc/CacheMvccVacuumTest.java | 15 + ...niteBaselineAffinityTopologyActivationTest.java | 15 + .../persistence/IgnitePdsCorruptedIndexTest.java | 16 + .../IgnitePdsPartitionFilesDestroyTest.java | 15 + ...gniteRebalanceScheduleResendPartitionsTest.java | 15 + ...ocalWalModeChangeDuringRebalancingSelfTest.java | 5 + .../ClientAffinityAssignmentWithBaselineTest.java | 15 + .../IgniteAbsentEvictionNodeOutOfBaselineTest.java | 11 + ...hangingBaselineDownCachePutAllFailoverTest.java | 6 + ...hangingBaselineDownCacheRemoveFailoverTest.java | 5 + ...eChangingBaselineUpCachePutAllFailoverTest.java | 6 + ...eChangingBaselineUpCacheRemoveFailoverTest.java | 5 + .../persistence/db/IgniteLogicalRecoveryTest.java | 16 + ...IgnitePdsCacheWalDisabledOnRebalancingTest.java | 15 + .../db/IgnitePdsDataRegionMetricsTest.java | 14 + .../cache/persistence/db/IgnitePdsWithTtlTest.java | 16 + .../SlowHistoricalRebalanceSmallHistoryTest.java | 16 + ...eWalFlushMultiNodeFailoverAbstractSelfTest.java | 16 + .../persistence/db/wal/WalCompactionTest.java | 16 + .../cluster/ChangeTopologyWatcherTest.java | 350 +++++++++++++++++++++ .../distributed/DistributedConfigurationTest.java | 25 +- .../service/IgniteServiceReassignmentTest.java | 16 + .../ServiceDeploymentOutsideBaselineTest.java | 18 ++ .../testsuites/IgniteKernalSelfTestSuite.java | 3 + .../apache/ignite/util/GridCommandHandlerTest.java | 10 + ...ngingBaselineCacheQueryNodeRestartSelfTest.java | 6 + .../Cache/PersistenceTest.cs | 4 + .../IgniteConfigurationTest.cs | 4 - .../Apache.Ignite.Core/IgniteConfiguration.cs | 60 ---- .../IgniteConfigurationSection.xsd | 15 - 64 files changed, 1410 insertions(+), 285 deletions(-) diff --git a/modules/clients/src/test/java/org/apache/ignite/internal/processors/rest/JettyRestProcessorBaselineSelfTest.java b/modules/clients/src/test/java/org/apache/ignite/internal/processors/rest/JettyRestProcessorBaselineSelfTest.java index 1a83a43..aa20c50 100644 --- a/modules/clients/src/test/java/org/apache/ignite/internal/processors/rest/JettyRestProcessorBaselineSelfTest.java +++ b/modules/clients/src/test/java/org/apache/ignite/internal/processors/rest/JettyRestProcessorBaselineSelfTest.java @@ -32,6 +32,7 @@ import org.apache.ignite.internal.util.typedef.internal.U; import org.jetbrains.annotations.Nullable; import org.junit.Test; +import static org.apache.ignite.IgniteSystemProperties.IGNITE_BASELINE_AUTO_ADJUST_ENABLED; import static org.apache.ignite.configuration.WALMode.NONE; import static org.apache.ignite.internal.processors.rest.GridRestResponse.STATUS_SUCCESS; @@ -41,6 +42,8 @@ import static org.apache.ignite.internal.processors.rest.GridRestResponse.STATUS public class JettyRestProcessorBaselineSelfTest extends JettyRestProcessorCommonSelfTest { /** {@inheritDoc} */ @Override protected void beforeTestsStarted() throws Exception { + System.setProperty(IGNITE_BASELINE_AUTO_ADJUST_ENABLED, "false"); + U.resolveWorkDirectory(U.defaultWorkDirectory(), "db", true); super.beforeTestsStarted(); @@ -76,6 +79,13 @@ public class JettyRestProcessorBaselineSelfTest extends JettyRestProcessorCommon return cfg; } + /** {@inheritDoc} */ + @Override protected void afterTestsStopped() throws Exception { + super.afterTestsStopped(); + + System.clearProperty(IGNITE_BASELINE_AUTO_ADJUST_ENABLED); + } + /** * @param nodes Collection of grid nodes. * @return Collection of node consistent IDs for given collection of grid nodes. diff --git a/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java b/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java index 64982d4..6398e2d 100644 --- a/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java +++ b/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java @@ -1127,6 +1127,11 @@ public final class IgniteSystemProperties { public static final String IGNITE_NODE_IDS_HISTORY_SIZE = "IGNITE_NODE_IDS_HISTORY_SIZE"; /** + * Flag to enable baseline auto-adjust by default. + */ + public static final String IGNITE_BASELINE_AUTO_ADJUST_ENABLED = "IGNITE_BASELINE_AUTO_ADJUST_ENABLED"; + + /** * Enforces singleton. */ private IgniteSystemProperties() { diff --git a/modules/core/src/main/java/org/apache/ignite/configuration/IgniteConfiguration.java b/modules/core/src/main/java/org/apache/ignite/configuration/IgniteConfiguration.java index 7da24fbcc..1ded486 100644 --- a/modules/core/src/main/java/org/apache/ignite/configuration/IgniteConfiguration.java +++ b/modules/core/src/main/java/org/apache/ignite/configuration/IgniteConfiguration.java @@ -225,18 +225,6 @@ public class IgniteConfiguration { /** Default time interval between MVCC vacuum runs in milliseconds. */ public static final long DFLT_MVCC_VACUUM_FREQUENCY = 5000; - /** Default of initial value of manual baseline control or auto adjusting baseline. */ - public static final boolean DFLT_INIT_BASELINE_AUTO_ADJUST_ENABLED = false; - - /** - * Initial value of time which we would wait before the actual topology change since last discovery event(node - * join/exit). - */ - public static final long DFLT_INIT_BASELINE_AUTO_ADJUST_TIMEOUT = 0; - - /** Initial value of time which we would wait from the first discovery event in the chain(node join/exit). */ - public static final long DFLT_INIT_BASELINE_AUTO_ADJUST_MAX_TIMEOUT = 0; - /** Default SQL query history size. */ public static final int DFLT_SQL_QUERY_HISTORY_SIZE = 1000; @@ -542,18 +530,6 @@ public class IgniteConfiguration { /** SQL schemas to be created on node start. */ private String[] sqlSchemas; - /** Initial value of manual baseline control or auto adjusting baseline. */ - private boolean initBaselineAutoAdjustEnabled = DFLT_INIT_BASELINE_AUTO_ADJUST_ENABLED; - - /** - * Initial value of time which we would wait before the actual topology change since last discovery event(node - * join/exit). - */ - private long initBaselineAutoAdjustTimeout = DFLT_INIT_BASELINE_AUTO_ADJUST_TIMEOUT; - - /** Initial value of time which we would wait from the first discovery event in the chain(node join/exit). */ - private long initBaselineAutoAdjustMaxTimeout = DFLT_INIT_BASELINE_AUTO_ADJUST_MAX_TIMEOUT; - /** * Creates valid grid configuration with all default values. */ @@ -3222,59 +3198,6 @@ public class IgniteConfiguration { return this; } - /** - * Gets initial value of manual baseline control or auto adjusting baseline. This value would be used only if it - * have not been changed earlier in real time. - * - * @return {@code true} if auto adjusting baseline enabled. - */ - public boolean isInitBaselineAutoAdjustEnabled() { - return initBaselineAutoAdjustEnabled; - } - - /** - * Sets initial value of manual baseline control or auto adjusting baseline. - */ - public void setInitBaselineAutoAdjustEnabled(boolean initBaselineAutoAdjustEnabled) { - this.initBaselineAutoAdjustEnabled = initBaselineAutoAdjustEnabled; - } - - /** - * Gets initial value of time which we would wait before the actual topology change. But it would be reset if new - * discovery event happened. (node join/exit). This value would be used only if it have not been changed earlier in - * real time. - * - * @return Timeout of wait the actual topology change. - */ - public long getInitBaselineAutoAdjustTimeout() { - return initBaselineAutoAdjustTimeout; - } - - /** - * Sets initial value of time which we would wait before the actual topology change. - */ - public void setInitBaselineAutoAdjustTimeout(long initBaselineAutoAdjustTimeout) { - this.initBaselineAutoAdjustTimeout = initBaselineAutoAdjustTimeout; - } - - /** - * Gets initial value of time which we would wait from the first discovery event in the chain. If we achieved it - * than we would change BLAT right away (no matter were another node join/exit happened or not). This value would be - * used only if it have not been changed earlier in real time. - * - * @return Timeout of wait the actual topology change. - */ - public long getInitBaselineAutoAdjustMaxTimeout() { - return initBaselineAutoAdjustMaxTimeout; - } - - /** - * Sets initial value of time which we would wait from the first discovery event in the chain. - */ - public void setInitBaselineAutoAdjustMaxTimeout(long initBaselineAutoAdjustMaxTimeout) { - this.initBaselineAutoAdjustMaxTimeout = initBaselineAutoAdjustMaxTimeout; - } - /** {@inheritDoc} */ @Override public String toString() { return S.toString(IgniteConfiguration.class, this); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/cluster/DistributedBaselineConfiguration.java b/modules/core/src/main/java/org/apache/ignite/internal/cluster/DistributedBaselineConfiguration.java index 99d8929..9686a3a 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/cluster/DistributedBaselineConfiguration.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/cluster/DistributedBaselineConfiguration.java @@ -18,43 +18,67 @@ package org.apache.ignite.internal.cluster; import org.apache.ignite.IgniteCheckedException; +import org.apache.ignite.IgniteLogger; +import org.apache.ignite.IgniteSystemProperties; import org.apache.ignite.configuration.IgniteConfiguration; import org.apache.ignite.internal.processors.configuration.distributed.DistributedBooleanProperty; import org.apache.ignite.internal.processors.configuration.distributed.DistributedLongProperty; import org.apache.ignite.internal.processors.subscription.GridInternalSubscriptionProcessor; +import org.apache.ignite.internal.util.future.GridFutureAdapter; +import org.apache.ignite.internal.util.typedef.internal.CU; -import static org.apache.ignite.internal.processors.configuration.distributed.DistributedBooleanProperty.detachedProperty; -import static org.apache.ignite.internal.processors.configuration.distributed.DistributedLongProperty.detachedProperty; +import static org.apache.ignite.IgniteSystemProperties.IGNITE_BASELINE_AUTO_ADJUST_ENABLED; +import static org.apache.ignite.IgniteSystemProperties.IGNITE_RECOVERY_SEMAPHORE_PERMITS; +import static org.apache.ignite.IgniteSystemProperties.getBoolean; +import static org.apache.ignite.internal.processors.configuration.distributed.DistributedBooleanProperty.detachedBooleanProperty; +import static org.apache.ignite.internal.processors.configuration.distributed.DistributedLongProperty.detachedLongProperty; /** * Distributed baseline configuration. */ public class DistributedBaselineConfiguration { + /** Default auto-adjust timeout for persistence grid. */ + private static final int DEFAULT_PERSISTENCE_TIMEOUT = 5 * 60_000; + /** Default auto-adjust timeout for in-memory grid. */ + private static final int DEFAULT_IN_MEMORY_TIMEOUT = 0; + /** Message of baseline auto-adjust configuration. */ + private static final String AUTO_ADJUST_CONFIGURED_MESSAGE = "Baseline auto-adjust is '%s' with timeout='%d' ms"; /** Value of manual baseline control or auto adjusting baseline. */ - private DistributedBooleanProperty baselineAutoAdjustEnabled; + private volatile DistributedBooleanProperty baselineAutoAdjustEnabled = + detachedBooleanProperty("baselineAutoAdjustEnabled", false); /** * Value of time which we would wait before the actual topology change since last discovery event(node join/exit). */ - private DistributedLongProperty baselineAutoAdjustTimeout; - - /** Value of time which we would wait from the first discovery event in the chain(node join/exit). */ - private DistributedLongProperty baselineAutoAdjustMaxTimeout; + private volatile DistributedLongProperty baselineAutoAdjustTimeout = + detachedLongProperty("baselineAutoAdjustTimeout", -1L); /** * @param cfg Static config. * @param isp Subscription processor. */ - public DistributedBaselineConfiguration(IgniteConfiguration cfg, GridInternalSubscriptionProcessor isp) { - baselineAutoAdjustEnabled = detachedProperty("baselineAutoAdjustEnabled", cfg.isInitBaselineAutoAdjustEnabled()); - baselineAutoAdjustTimeout = detachedProperty("baselineAutoAdjustTimeout", cfg.getInitBaselineAutoAdjustTimeout()); - baselineAutoAdjustMaxTimeout = detachedProperty("baselineAutoAdjustMaxTimeout", cfg.getInitBaselineAutoAdjustMaxTimeout()); - + public DistributedBaselineConfiguration( + IgniteConfiguration cfg, + GridInternalSubscriptionProcessor isp, + IgniteLogger log) { isp.registerDistributedConfigurationListener( dispatcher -> { + boolean persistenceEnabled = cfg != null && CU.isPersistenceEnabled(cfg); + + long timeout = persistenceEnabled ? DEFAULT_PERSISTENCE_TIMEOUT : DEFAULT_IN_MEMORY_TIMEOUT; + + //It set default value locally only. + baselineAutoAdjustEnabled.localUpdate(getBoolean(IGNITE_BASELINE_AUTO_ADJUST_ENABLED, true)); + baselineAutoAdjustTimeout.localUpdate(timeout); + dispatcher.registerProperty(baselineAutoAdjustEnabled); dispatcher.registerProperty(baselineAutoAdjustTimeout); - dispatcher.registerProperty(baselineAutoAdjustMaxTimeout); + + log.info( + String.format(AUTO_ADJUST_CONFIGURED_MESSAGE, + (baselineAutoAdjustEnabled.value() ? "enabled" : "disabled"), + baselineAutoAdjustTimeout.value() + )); } ); } @@ -70,8 +94,9 @@ public class DistributedBaselineConfiguration { * @param baselineAutoAdjustEnabled Value of manual baseline control or auto adjusting baseline. * @throws IgniteCheckedException if failed. */ - public void setBaselineAutoAdjustEnabled(boolean baselineAutoAdjustEnabled) throws IgniteCheckedException { - this.baselineAutoAdjustEnabled.propagate(baselineAutoAdjustEnabled); + public GridFutureAdapter<?> updateBaselineAutoAdjustEnabledAsync(boolean baselineAutoAdjustEnabled) + throws IgniteCheckedException { + return this.baselineAutoAdjustEnabled.propagateAsync(baselineAutoAdjustEnabled); } /** @@ -87,23 +112,8 @@ public class DistributedBaselineConfiguration { * discovery event(node join/exit). * @throws IgniteCheckedException If failed. */ - public void setBaselineAutoAdjustTimeout(long baselineAutoAdjustTimeout) throws IgniteCheckedException { - this.baselineAutoAdjustTimeout.propagate(baselineAutoAdjustTimeout); - } - - /** - * @return Value of time which we would wait from the first discovery event in the chain(node join/exit). - */ - public long getBaselineAutoAdjustMaxTimeout() { - return baselineAutoAdjustMaxTimeout.value(); - } - - /** - * @param baselineAutoAdjustMaxTimeout Value of time which we would wait from the first discovery event in the - * chain(node join/exit). - * @throws IgniteCheckedException If failed. - */ - public void setBaselineAutoAdjustMaxTimeout(long baselineAutoAdjustMaxTimeout) throws IgniteCheckedException { - this.baselineAutoAdjustMaxTimeout.propagate(baselineAutoAdjustMaxTimeout); + public GridFutureAdapter<?> updateBaselineAutoAdjustTimeoutAsync( + long baselineAutoAdjustTimeout) throws IgniteCheckedException { + return this.baselineAutoAdjustTimeout.propagateAsync(baselineAutoAdjustTimeout); } } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/cluster/IgniteClusterImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/cluster/IgniteClusterImpl.java index 2f5e63a..2eaf715 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/cluster/IgniteClusterImpl.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/cluster/IgniteClusterImpl.java @@ -113,7 +113,9 @@ public class IgniteClusterImpl extends ClusterGroupAdapter implements IgniteClus nodeLoc = new ClusterNodeLocalMapImpl(ctx); - distributedBaselineConfiguration = new DistributedBaselineConfiguration(cfg, ctx.internalSubscriptionProcessor()); + distributedBaselineConfiguration = new DistributedBaselineConfiguration( + cfg, ctx.internalSubscriptionProcessor(), ctx.log(DistributedBaselineConfiguration.class) + ); } /** {@inheritDoc} */ @@ -368,6 +370,17 @@ public class IgniteClusterImpl extends ClusterGroupAdapter implements IgniteClus } } + /** + * Sets baseline topology constructed from the cluster topology of the given version (the method succeeds only if + * the cluster topology has not changed). All client and daemon nodes will be filtered out of the resulting + * baseline. + * + * @param topVer Topology version to set. + */ + public void triggerBaselineAutoAdjust(long topVer) { + setBaselineTopology(topVer, true); + } + /** */ private boolean isInMemoryMode() { return !CU.isPersistenceEnabled(cfg); @@ -460,6 +473,10 @@ public class IgniteClusterImpl extends ClusterGroupAdapter implements IgniteClus /** {@inheritDoc} */ @Override public void setBaselineTopology(long topVer) { + setBaselineTopology(topVer, false); + } + + private void setBaselineTopology(long topVer, boolean isBaselineAutoAdjust) { guard(); try { @@ -480,7 +497,7 @@ public class IgniteClusterImpl extends ClusterGroupAdapter implements IgniteClus validateBeforeBaselineChange(target); - ctx.state().changeGlobalState(true, target, true).get(); + ctx.state().changeGlobalState(true, target, true, isBaselineAutoAdjust).get(); } catch (IgniteCheckedException e) { throw U.convertException(e); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePartitionExchangeManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePartitionExchangeManager.java index 553af8a..2568802 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePartitionExchangeManager.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePartitionExchangeManager.java @@ -892,7 +892,7 @@ public class GridCachePartitionExchangeManager<K, V> extends GridCacheSharedMana /** * @return Last finished topology future. */ - @Nullable public GridDhtTopologyFuture lastFinishedFuture() { + @Nullable public GridDhtPartitionsExchangeFuture lastFinishedFuture() { return lastFinishedFut.get(); } @@ -918,7 +918,7 @@ public class GridCachePartitionExchangeManager<K, V> extends GridCacheSharedMana * @param ver Topology version. * @return Future or {@code null} is future is already completed. */ - @Nullable public IgniteInternalFuture<AffinityTopologyVersion> affinityReadyFuture(AffinityTopologyVersion ver) { + @NotNull public IgniteInternalFuture<AffinityTopologyVersion> affinityReadyFuture(AffinityTopologyVersion ver) { GridDhtPartitionsExchangeFuture lastInitializedFut0 = lastInitializedFut; if (lastInitializedFut0 != null && lastInitializedFut0.initialVersion().compareTo(ver) == 0 diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTopologyFutureAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTopologyFutureAdapter.java index 8a6a5ee..88b99b7 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTopologyFutureAdapter.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTopologyFutureAdapter.java @@ -128,6 +128,14 @@ public abstract class GridDhtTopologyFutureAdapter extends GridFutureAdapter<Aff } /** + * @return {@code true} If any lost partitions was detected. + */ + public boolean hasLostPartitions() { + return grpValidRes.values().stream() + .anyMatch(CacheGroupValidation::hasLostPartitions); + } + + /** * Cache group validation result. */ protected static class CacheGroupValidation { 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 dd3a180..57de875 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 @@ -3576,7 +3576,8 @@ public class GridDhtPartitionsExchangeFuture extends GridDhtTopologyFutureAdapte ChangeGlobalStateFinishMessage stateFinishMsg = new ChangeGlobalStateFinishMessage( req.requestId(), active, - !stateChangeErr); + !stateChangeErr + ); cctx.discovery().sendCustomEvent(stateFinishMsg); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cluster/BaselineAdjustForbiddenException.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cluster/BaselineAdjustForbiddenException.java new file mode 100644 index 0000000..d99a53f --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cluster/BaselineAdjustForbiddenException.java @@ -0,0 +1,41 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.ignite.internal.processors.cluster; + +import org.apache.ignite.IgniteException; + +/** + * Exception which would be throw during force changing baseline if cluster has incorrect configuration. + */ +public class BaselineAdjustForbiddenException extends IgniteException { + /** */ + private static final long serialVersionUID = 0L; + /** */ + private static final String BASELINE_AUTO_ADJUST_ENABLED = + "Baseline auto-adjust is enabled, please turn-off it before try to adjust baseline manually"; + /** */ + private static final String BASELINE_AUTO_ADJUST_DISABLED = + "Baseline auto-adjust is disabled"; + + /** + * @param isBaselneAutoAdjustEnabled {@code true} if baseline auto-adjust enabled. + */ + public BaselineAdjustForbiddenException(boolean isBaselneAutoAdjustEnabled) { + super(isBaselneAutoAdjustEnabled ? BASELINE_AUTO_ADJUST_ENABLED : BASELINE_AUTO_ADJUST_DISABLED); + } +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cluster/ChangeGlobalStateFinishMessage.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cluster/ChangeGlobalStateFinishMessage.java index bbbd999..25dfadb 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cluster/ChangeGlobalStateFinishMessage.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cluster/ChangeGlobalStateFinishMessage.java @@ -49,7 +49,10 @@ public class ChangeGlobalStateFinishMessage implements DiscoveryCustomMessage { * @param reqId State change request ID. * @param clusterActive New cluster state. */ - public ChangeGlobalStateFinishMessage(UUID reqId, boolean clusterActive, Boolean transitionRes) { + public ChangeGlobalStateFinishMessage( + UUID reqId, + boolean clusterActive, + Boolean transitionRes) { assert reqId != null; this.reqId = reqId; diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cluster/ChangeGlobalStateMessage.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cluster/ChangeGlobalStateMessage.java index b1ff048..23f02ac 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cluster/ChangeGlobalStateMessage.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cluster/ChangeGlobalStateMessage.java @@ -163,8 +163,8 @@ public class ChangeGlobalStateMessage implements DiscoveryCustomMessage { } /** - * @return Node initiated state change. - */ + * @return Node initiated state change. + */ public UUID initiatorNodeId() { return initiatingNodeId; } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cluster/ClusterProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cluster/ClusterProcessor.java index 6d0710b..f9e5efe 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cluster/ClusterProcessor.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cluster/ClusterProcessor.java @@ -48,6 +48,7 @@ import org.apache.ignite.internal.managers.discovery.IgniteClusterNode; import org.apache.ignite.internal.managers.eventstorage.GridLocalEventListener; import org.apache.ignite.internal.processors.GridProcessorAdapter; import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion; +import org.apache.ignite.internal.processors.cluster.baseline.autoadjust.ChangeTopologyWatcher; import org.apache.ignite.internal.processors.timeout.GridTimeoutObject; import org.apache.ignite.internal.util.GridTimerTask; import org.apache.ignite.internal.util.future.GridFinishedFuture; @@ -72,6 +73,7 @@ import static org.apache.ignite.IgniteSystemProperties.IGNITE_DIAGNOSTIC_ENABLED import static org.apache.ignite.IgniteSystemProperties.IGNITE_UPDATE_NOTIFIER; import static org.apache.ignite.IgniteSystemProperties.getBoolean; import static org.apache.ignite.events.EventType.EVT_NODE_FAILED; +import static org.apache.ignite.events.EventType.EVT_NODE_JOINED; import static org.apache.ignite.events.EventType.EVT_NODE_LEFT; import static org.apache.ignite.internal.GridComponent.DiscoveryDataExchangeType.CLUSTER_PROC; import static org.apache.ignite.internal.GridTopic.TOPIC_INTERNAL_DIAGNOSTIC; @@ -170,6 +172,8 @@ public class ClusterProcessor extends GridProcessorAdapter { } }, EVT_NODE_FAILED, EVT_NODE_LEFT); + ctx.event().addLocalEventListener(new ChangeTopologyWatcher(ctx), EVT_NODE_FAILED, EVT_NODE_LEFT, EVT_NODE_JOINED); + ctx.io().addMessageListener(TOPIC_INTERNAL_DIAGNOSTIC, new GridMessageListener() { @Override public void onMessage(UUID nodeId, Object msg, byte plc) { if (msg instanceof IgniteDiagnosticMessage) { @@ -340,7 +344,7 @@ public class ClusterProcessor extends GridProcessorAdapter { Map<String, Object> map = (Map<String, Object>)ser; if (map.containsKey(ATTR_UPDATE_NOTIFIER_STATUS)) - flag = (Boolean) map.get(ATTR_UPDATE_NOTIFIER_STATUS); + flag = (Boolean)map.get(ATTR_UPDATE_NOTIFIER_STATUS); } } @@ -539,8 +543,8 @@ public class ClusterProcessor extends GridProcessorAdapter { } /** - * Sends diagnostic message closure to remote node. When response received dumps - * remote message and local communication info about connection(s) with remote node. + * Sends diagnostic message closure to remote node. When response received dumps remote message and local + * communication info about connection(s) with remote node. * * @param nodeId Target node ID. * @param c Closure to send. diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cluster/GridClusterStateProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cluster/GridClusterStateProcessor.java index e04feff..6509b51 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cluster/GridClusterStateProcessor.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cluster/GridClusterStateProcessor.java @@ -755,13 +755,22 @@ public class GridClusterStateProcessor extends GridProcessorAdapter implements I Collection<? extends BaselineNode> baselineNodes, boolean forceChangeBaselineTopology ) { + return changeGlobalState(activate, baselineNodes, forceChangeBaselineTopology, false); + } + + public IgniteInternalFuture<?> changeGlobalState( + final boolean activate, + Collection<? extends BaselineNode> baselineNodes, + boolean forceChangeBaselineTopology, + boolean isAutoAdjust + ) { if (inMemoryMode) - return changeGlobalState0(activate, null, false); + return changeGlobalState0(activate, null, false, isAutoAdjust); BaselineTopology newBlt = (compatibilityMode && !forceChangeBaselineTopology) ? null : calculateNewBaselineTopology(activate, baselineNodes, forceChangeBaselineTopology); - return changeGlobalState0(activate, newBlt, forceChangeBaselineTopology); + return changeGlobalState0(activate, newBlt, forceChangeBaselineTopology, isAutoAdjust); } /** @@ -831,6 +840,17 @@ public class GridClusterStateProcessor extends GridProcessorAdapter implements I /** */ private IgniteInternalFuture<?> changeGlobalState0(final boolean activate, BaselineTopology blt, boolean forceChangeBaselineTopology) { + return changeGlobalState0(activate, blt, forceChangeBaselineTopology, false); + } + + /** */ + private IgniteInternalFuture<?> changeGlobalState0(final boolean activate, + BaselineTopology blt, boolean forceChangeBaselineTopology, boolean isAutoAdjust) { + boolean isBaselineAutoAdjustEnabled = ctx.cluster().get().baselineConfiguration().isBaselineAutoAdjustEnabled(); + + if (forceChangeBaselineTopology && isBaselineAutoAdjustEnabled != isAutoAdjust) + throw new BaselineAdjustForbiddenException(isBaselineAutoAdjustEnabled); + if (ctx.isDaemon() || ctx.clientNode()) { GridFutureAdapter<Void> fut = new GridFutureAdapter<>(); @@ -899,7 +919,8 @@ public class GridClusterStateProcessor extends GridProcessorAdapter implements I activate, blt, forceChangeBaselineTopology, - System.currentTimeMillis()); + System.currentTimeMillis() + ); IgniteInternalFuture<?> resFut = wrapStateChangeFuture(startedFut, msg); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cluster/baseline/autoadjust/BaselineAutoAdjustData.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cluster/baseline/autoadjust/BaselineAutoAdjustData.java new file mode 100644 index 0000000..a913d09 --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cluster/baseline/autoadjust/BaselineAutoAdjustData.java @@ -0,0 +1,75 @@ +/* + * 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.cluster.baseline.autoadjust; + +import org.apache.ignite.events.Event; + +/** + * Container of required data for changing baseline. + */ +class BaselineAutoAdjustData { + /** Event with which this data correspond to. For statistic only. */ + private final Event reasonEvent; + /** Topology version nodes of which should be set to baseline by this task. */ + private final long targetTopologyVersion; + + /** {@code true} If this data don't actual anymore and it setting should be skipped. */ + private volatile boolean invalidated = false; + + /** + * @param evt Event with which this data correspond to. For statistic only. + * @param targetTopologyVersion Topology version nodes of which should be set by this task. + */ + BaselineAutoAdjustData(Event evt, long targetTopologyVersion) { + reasonEvent = evt; + this.targetTopologyVersion = targetTopologyVersion; + } + + /** + * Mark that this data are invalid. + */ + private void onInvalidate() { + invalidated = true; + } + + /** + * @return Topology version nodes of which should be set to baseline by this task. + */ + public long getTargetTopologyVersion() { + return targetTopologyVersion; + } + + /** + * @return {@code true} If this data still actual and can be set. + */ + public boolean isInvalidated() { + return invalidated; + } + + /** + * Produce next set baseline data based on this data. + * + * @param evt New triggired event. + * @return New set baseline data. + */ + public BaselineAutoAdjustData next(Event evt, long targetTopologyVersion) { + onInvalidate(); + + return new BaselineAutoAdjustData(evt, targetTopologyVersion); + } +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cluster/baseline/autoadjust/BaselineAutoAdjustExecutor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cluster/baseline/autoadjust/BaselineAutoAdjustExecutor.java new file mode 100644 index 0000000..b560932 --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cluster/baseline/autoadjust/BaselineAutoAdjustExecutor.java @@ -0,0 +1,84 @@ +/* + * 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.cluster.baseline.autoadjust; + +import java.util.concurrent.ExecutorService; +import java.util.concurrent.locks.Lock; +import java.util.concurrent.locks.ReentrantLock; +import java.util.function.BooleanSupplier; +import org.apache.ignite.IgniteException; +import org.apache.ignite.IgniteLogger; +import org.apache.ignite.internal.cluster.IgniteClusterImpl; + +/** + * This executor try to set new baseline by given data. + */ +class BaselineAutoAdjustExecutor { + /** */ + private final IgniteLogger log; + /** */ + private final IgniteClusterImpl cluster; + /** Service for execute this task in async. */ + private final ExecutorService executorService; + /** {@code true} if baseline auto-adjust enabled. */ + private final BooleanSupplier isBaselineAutoAdjustEnabled; + /** This protect from execution more than one task at same moment. */ + private final Lock executionGuard = new ReentrantLock(); + + /** + * @param log Logger. + * @param cluster Ignite cluster. + * @param executorService Thread pool for changing baseline. + * @param enabledSupplier Supplier return {@code true} if baseline auto-adjust enabled. + */ + public BaselineAutoAdjustExecutor(IgniteLogger log, IgniteClusterImpl cluster, ExecutorService executorService, + BooleanSupplier enabledSupplier) { + this.log = log; + this.cluster = cluster; + this.executorService = executorService; + isBaselineAutoAdjustEnabled = enabledSupplier; + } + + /** + * Try to set baseline if all conditions it allowed. + * + * @param data Data for operation. + */ + public void execute(BaselineAutoAdjustData data) { + executorService.submit(() -> + { + if (data.isInvalidated() || !isBaselineAutoAdjustEnabled.getAsBoolean()) + return; + + executionGuard.lock(); + try { + if (data.isInvalidated() || !isBaselineAutoAdjustEnabled.getAsBoolean()) + return; + + cluster.triggerBaselineAutoAdjust(data.getTargetTopologyVersion()); + } + catch (IgniteException e) { + log.error("Error during baseline changing", e); + } + finally { + executionGuard.unlock(); + } + } + ); + } +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cluster/baseline/autoadjust/BaselineAutoAdjustScheduler.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cluster/baseline/autoadjust/BaselineAutoAdjustScheduler.java new file mode 100644 index 0000000..d89ba85 --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cluster/baseline/autoadjust/BaselineAutoAdjustScheduler.java @@ -0,0 +1,63 @@ +/* + * 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.cluster.baseline.autoadjust; + +import org.apache.ignite.internal.processors.timeout.GridTimeoutObject; +import org.apache.ignite.internal.processors.timeout.GridTimeoutObjectAdapter; +import org.apache.ignite.internal.processors.timeout.GridTimeoutProcessor; + +/** + * This class able to add task of set baseline with timeout to queue. In one time only one task can be in queue. Every + * next queuing task evicted previous one. + */ +class BaselineAutoAdjustScheduler { + /** Timeout processor. */ + private final GridTimeoutProcessor timeoutProcessor; + /** Executor of set baseline operation. */ + private final BaselineAutoAdjustExecutor baselineAutoAdjustExecutor; + /** Last set task for set new baseline. It needed for removing from queue. */ + private GridTimeoutObject baselineTimeoutObj; + + /** + * @param timeoutProcessor Timeout processor. + * @param baselineAutoAdjustExecutor Executor of set baseline operation. + */ + public BaselineAutoAdjustScheduler(GridTimeoutProcessor timeoutProcessor, BaselineAutoAdjustExecutor baselineAutoAdjustExecutor) { + this.timeoutProcessor = timeoutProcessor; + this.baselineAutoAdjustExecutor = baselineAutoAdjustExecutor; + } + + /** + * Adding task to queue with delay and remove previous one. + * + * @param baselineAutoAdjustData Data for changing baseline. + * @param delay Delay after which set baseline should be started. + */ + public synchronized void schedule(BaselineAutoAdjustData baselineAutoAdjustData, long delay) { + if (baselineTimeoutObj != null) + timeoutProcessor.removeTimeoutObject(baselineTimeoutObj); + + timeoutProcessor.addTimeoutObject( + baselineTimeoutObj = new GridTimeoutObjectAdapter(delay) { + @Override public void onTimeout() { + baselineAutoAdjustExecutor.execute(baselineAutoAdjustData); + } + } + ); + } +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cluster/baseline/autoadjust/ChangeTopologyWatcher.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cluster/baseline/autoadjust/ChangeTopologyWatcher.java new file mode 100644 index 0000000..1188fb4 --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cluster/baseline/autoadjust/ChangeTopologyWatcher.java @@ -0,0 +1,124 @@ +/* + * 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.cluster.baseline.autoadjust; + +import org.apache.ignite.IgniteCheckedException; +import org.apache.ignite.IgniteLogger; +import org.apache.ignite.events.DiscoveryEvent; +import org.apache.ignite.events.Event; +import org.apache.ignite.internal.GridKernalContext; +import org.apache.ignite.internal.IgniteInternalFuture; +import org.apache.ignite.internal.cluster.DistributedBaselineConfiguration; +import org.apache.ignite.internal.cluster.IgniteClusterImpl; +import org.apache.ignite.internal.managers.discovery.GridDiscoveryManager; +import org.apache.ignite.internal.managers.eventstorage.GridLocalEventListener; +import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion; +import org.apache.ignite.internal.processors.cache.GridCachePartitionExchangeManager; +import org.apache.ignite.internal.processors.cluster.GridClusterStateProcessor; +import org.apache.ignite.lang.IgniteInClosure; + +import static org.apache.ignite.internal.util.IgniteUtils.isLocalNodeCoordinator; + +/** + * Watcher of topology changes. It initiate to set new baseline after some timeout. + */ +public class ChangeTopologyWatcher implements GridLocalEventListener { + /** Task represented NULL value is using when normal task can not be created. */ + private static final BaselineAutoAdjustData NULL_BASELINE_DATA = new BaselineAutoAdjustData(null, -1); + /** */ + private final IgniteLogger log; + /** */ + private final IgniteClusterImpl cluster; + /** */ + private final GridCachePartitionExchangeManager<?, ?> exchangeManager; + /** Configuration of baseline. */ + private final DistributedBaselineConfiguration baselineConfiguration; + /** Discovery manager. */ + private final GridDiscoveryManager discoveryMgr; + /** */ + private final GridClusterStateProcessor stateProcessor; + /** Scheduler of specific task of baseline changing. */ + private final BaselineAutoAdjustScheduler baselineAutoAdjustScheduler; + + /** Last data for set new baseline. */ + private BaselineAutoAdjustData lastBaselineData = NULL_BASELINE_DATA; + + /** + * @param ctx Context. + */ + public ChangeTopologyWatcher(GridKernalContext ctx) { + this.log = ctx.log(ChangeTopologyWatcher.class); + this.cluster = ctx.cluster().get(); + this.baselineConfiguration = cluster.baselineConfiguration(); + this.exchangeManager = ctx.cache().context().exchange(); + this.stateProcessor = ctx.state(); + this.baselineAutoAdjustScheduler = new BaselineAutoAdjustScheduler(ctx.timeout(), new BaselineAutoAdjustExecutor( + ctx.log(BaselineAutoAdjustExecutor.class), + cluster, + ctx.getSystemExecutorService(), + this::isBaselineAutoAdjustEnabled + )); + this.discoveryMgr = ctx.discovery(); + } + + /** {@inheritDoc} */ + @Override public void onEvent(Event evt) { + if (!isBaselineAutoAdjustEnabled()) { + synchronized (this) { + lastBaselineData = NULL_BASELINE_DATA; + } + + return; + } + + DiscoveryEvent discoEvt = (DiscoveryEvent)evt; + + if (discoEvt.eventNode().isClient() || discoEvt.eventNode().isDaemon()) + return; + + synchronized (this) { + lastBaselineData = lastBaselineData.next(evt, discoEvt.topologyVersion()); + + if (isLocalNodeCoordinator(discoveryMgr)) { + exchangeManager.affinityReadyFuture(new AffinityTopologyVersion(discoEvt.topologyVersion())) + .listen((IgniteInClosure<IgniteInternalFuture<AffinityTopologyVersion>>)future -> { + + if (exchangeManager.lastFinishedFuture().hasLostPartitions()) { + log.warning("Baseline won't be changed cause the lost partitions were detected"); + + return; + } + + long timeout = baselineConfiguration.getBaselineAutoAdjustTimeout(); + + log.warning("Baseline will be changed in '" + timeout + "' ms"); + + baselineAutoAdjustScheduler.schedule(lastBaselineData, timeout); + }); + + } + } + } + + /** + * @return {@code true} if auto-adjust baseline enabled. + */ + private boolean isBaselineAutoAdjustEnabled() { + return stateProcessor.clusterState().active() && baselineConfiguration.isBaselineAutoAdjustEnabled(); + } +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/configuration/distributed/DistributedBooleanProperty.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/configuration/distributed/DistributedBooleanProperty.java index c459c82..d445d5b 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/configuration/distributed/DistributedBooleanProperty.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/configuration/distributed/DistributedBooleanProperty.java @@ -32,7 +32,7 @@ public class DistributedBooleanProperty extends DistributedProperty<Boolean> { * @param initVal Initial initVal of property. * @return Property detached from processor.(Distributed updating are not accessable). */ - public static DistributedBooleanProperty detachedProperty(String name, Boolean initVal) { + public static DistributedBooleanProperty detachedBooleanProperty(String name, Boolean initVal) { return new DistributedBooleanProperty(name, initVal); } } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/configuration/distributed/DistributedConfigurationProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/configuration/distributed/DistributedConfigurationProcessor.java index 9c8116d..1c977b3 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/configuration/distributed/DistributedConfigurationProcessor.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/configuration/distributed/DistributedConfigurationProcessor.java @@ -27,7 +27,6 @@ import org.apache.ignite.internal.processors.metastorage.DistributedMetaStorage; import org.apache.ignite.internal.processors.metastorage.DistributedMetastorageLifecycleListener; import org.apache.ignite.internal.processors.metastorage.ReadableDistributedMetaStorage; import org.apache.ignite.internal.processors.subscription.GridInternalSubscriptionProcessor; -import org.apache.ignite.internal.util.lang.IgniteThrowableBiConsumer; import static org.apache.ignite.internal.processors.configuration.distributed.DistributedConfigurationProcessor.AllowableAction.ACTUALIZE; import static org.apache.ignite.internal.processors.configuration.distributed.DistributedConfigurationProcessor.AllowableAction.CLUSTER_WIDE_UPDATE; @@ -253,8 +252,7 @@ public class DistributedConfigurationProcessor extends GridProcessorAdapter impl */ private void doClusterWideUpdate(DistributedProperty prop) { prop.onReadyForUpdate( - (IgniteThrowableBiConsumer<String, Serializable>)(key, value) -> - distributedMetastorage.write(toMetaStorageKey(key), value) + (key, newValue) -> distributedMetastorage.writeAsync(toMetaStorageKey(key), newValue) ); } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/configuration/distributed/DistributedLongProperty.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/configuration/distributed/DistributedLongProperty.java index dacd973..8e57891 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/configuration/distributed/DistributedLongProperty.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/configuration/distributed/DistributedLongProperty.java @@ -32,7 +32,7 @@ public class DistributedLongProperty extends DistributedComparableProperty<Long> * @param initVal Initial initVal of property. * @return Property detached from processor.(Distributed updating are not accessable). */ - public static DistributedLongProperty detachedProperty(String name, Long initVal) { + public static DistributedLongProperty detachedLongProperty(String name, Long initVal) { return new DistributedLongProperty(name, initVal); } } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/configuration/distributed/DistributedProperty.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/configuration/distributed/DistributedProperty.java index af67b55..3e3fc2e 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/configuration/distributed/DistributedProperty.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/configuration/distributed/DistributedProperty.java @@ -19,7 +19,7 @@ package org.apache.ignite.internal.processors.configuration.distributed; import java.io.Serializable; import org.apache.ignite.IgniteCheckedException; -import org.apache.ignite.internal.util.lang.IgniteThrowableBiConsumer; +import org.apache.ignite.internal.util.future.GridFutureAdapter; import org.apache.ignite.internal.util.tostring.GridToStringExclude; import org.apache.ignite.internal.util.typedef.internal.S; import org.jetbrains.annotations.NotNull; @@ -39,7 +39,7 @@ public class DistributedProperty<T extends Serializable> { * wide. */ @GridToStringExclude - private volatile IgniteThrowableBiConsumer<String, Serializable> clusterWideUpdater; + private volatile PropertyUpdateClosure clusterWideUpdater; /** * @param name Name of property. @@ -54,22 +54,51 @@ public class DistributedProperty<T extends Serializable> { * Change value across whole cluster. * * @param newVal Value which this property should be changed on. - * @return {@code true} if value was successfully updated and {@code false} if cluster wide update have not - * permitted yet. + * @return {@code true} if value was successfully updated and {@code false} if cluster wide update was failed, + * perhaps some concurrent operation was changed this value in same moment. * @throws DetachedPropertyException If this property have not been attached to processor yet, please call {@link * DistributedConfigurationProcessor#registerProperty(DistributedProperty)} before this method. + * @throws NotWritablePropertyException If this property don't ready to cluster wide update yet, perhaps cluster is + * not active yet. * @throws IgniteCheckedException If failed during cluster wide update. */ public boolean propagate(T newVal) throws IgniteCheckedException { + ensureClusterWideUpdateIsReady(); + + clusterWideUpdater.update(name, newVal).get(); + + return true; + } + + /** + * @throws DetachedPropertyException If this property have not been attached to processor yet, please call {@link + * DistributedConfigurationProcessor#registerProperty(DistributedProperty)} before this method. + * @throws NotWritablePropertyException If this property don't ready to cluster wide update yet, perhaps cluster is + * not active yet. + */ + private void ensureClusterWideUpdateIsReady() throws DetachedPropertyException, NotWritablePropertyException { if (!attached) throw new DetachedPropertyException(name); if (clusterWideUpdater == null) - return false; + throw new NotWritablePropertyException(name); + } - clusterWideUpdater.accept(name, newVal); + /** + * Change value across whole cluster. + * + * @param newVal Value which this property should be changed on. + * @return Future for update operation. + * @throws DetachedPropertyException If this property have not been attached to processor yet, please call {@link + * DistributedConfigurationProcessor#registerProperty(DistributedProperty)} before this method. + * @throws NotWritablePropertyException If this property don't ready to cluster wide update yet, perhaps cluster is + * not active yet. + * @throws IgniteCheckedException If failed during cluster wide update. + */ + public GridFutureAdapter<?> propagateAsync(T newVal) throws IgniteCheckedException { + ensureClusterWideUpdateIsReady(); - return true; + return clusterWideUpdater.update(name, newVal); } /** @@ -98,7 +127,7 @@ public class DistributedProperty<T extends Serializable> { * * @param updater Consumer for update value across cluster. */ - void onReadyForUpdate(@NotNull IgniteThrowableBiConsumer<String, Serializable> updater) { + void onReadyForUpdate(@NotNull PropertyUpdateClosure updater) { this.clusterWideUpdater = updater; } @@ -107,7 +136,7 @@ public class DistributedProperty<T extends Serializable> { * * @param newVal New value. */ - void localUpdate(Serializable newVal) { + public void localUpdate(Serializable newVal) { val = (T)newVal; } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/configuration/distributed/DistributedLongProperty.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/configuration/distributed/NotWritablePropertyException.java similarity index 60% copy from modules/core/src/main/java/org/apache/ignite/internal/processors/configuration/distributed/DistributedLongProperty.java copy to modules/core/src/main/java/org/apache/ignite/internal/processors/configuration/distributed/NotWritablePropertyException.java index dacd973..cd2780d 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/configuration/distributed/DistributedLongProperty.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/configuration/distributed/NotWritablePropertyException.java @@ -17,22 +17,18 @@ package org.apache.ignite.internal.processors.configuration.distributed; +import org.apache.ignite.IgniteCheckedException; + /** - * Implementation of {@link DistributedProperty} for {@link Long}. + * Exception of distributed property still have not been allowed for write value. */ -public class DistributedLongProperty extends DistributedComparableProperty<Long> { - - /** {@inheritDoc} */ - DistributedLongProperty(String name, Long initVal) { - super(name, initVal); - } - +public class NotWritablePropertyException extends IgniteCheckedException { + /** */ + private static final long serialVersionUID = 0L; /** - * @param name Name of property. - * @param initVal Initial initVal of property. - * @return Property detached from processor.(Distributed updating are not accessable). + * @param name Name of detached property. */ - public static DistributedLongProperty detachedProperty(String name, Long initVal) { - return new DistributedLongProperty(name, initVal); + public NotWritablePropertyException(String name) { + super("Property '" + name + "' is not ready for cluster wide update, perhaps cluster is not active yet."); } } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/lang/IgniteThrowableBiConsumer.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/configuration/distributed/PropertyUpdateClosure.java similarity index 60% rename from modules/core/src/main/java/org/apache/ignite/internal/util/lang/IgniteThrowableBiConsumer.java rename to modules/core/src/main/java/org/apache/ignite/internal/processors/configuration/distributed/PropertyUpdateClosure.java index 2733759..59cd1e4 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/util/lang/IgniteThrowableBiConsumer.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/configuration/distributed/PropertyUpdateClosure.java @@ -15,24 +15,25 @@ * limitations under the License. */ -package org.apache.ignite.internal.util.lang; +package org.apache.ignite.internal.processors.configuration.distributed; import java.io.Serializable; import org.apache.ignite.IgniteCheckedException; +import org.apache.ignite.internal.util.future.GridFutureAdapter; /** - * Represents an operation that accepts a single input argument and returns no result. Unlike most other functional - * interfaces, {@code IgniteThrowableConsumer} is expected to operate via side-effects. - * - * @param <E> Type of closure parameter. - * @param <R> Type of result value. + * Closure of cluster wide update of distributed property. */ -public interface IgniteThrowableBiConsumer<E, R> extends Serializable { +@FunctionalInterface +public interface PropertyUpdateClosure { + /** - * Consumer body. + * Update property on cluster. * - * @param e Consumer parameter. - * @throws IgniteCheckedException if body execution was failed. + * @param key Property key. + * @param newValue New value. + * @return Future this boolean value. + * @throws IgniteCheckedException if failed. */ - public void accept(E e, R r) throws IgniteCheckedException; + public GridFutureAdapter<?> update(String key, Serializable newValue) throws IgniteCheckedException; } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/metastorage/DistributedMetaStorage.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/metastorage/DistributedMetaStorage.java index 47236cb..837f265 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/metastorage/DistributedMetaStorage.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/metastorage/DistributedMetaStorage.java @@ -19,6 +19,7 @@ package org.apache.ignite.internal.processors.metastorage; import java.io.Serializable; import org.apache.ignite.IgniteCheckedException; +import org.apache.ignite.internal.util.future.GridFutureAdapter; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; @@ -38,6 +39,15 @@ public interface DistributedMetaStorage extends ReadableDistributedMetaStorage { void write(@NotNull String key, @NotNull Serializable val) throws IgniteCheckedException; /** + * Write value into distributed metastorage asynchronously. + * + * @param key The key. + * @param val Value to write. Must not be null. + * @throws IgniteCheckedException If cluster is in deactivated state. + */ + GridFutureAdapter<?> writeAsync(@NotNull String key, @NotNull Serializable val) throws IgniteCheckedException; + + /** * Remove value from distributed metastorage. * * @param key The key. @@ -62,6 +72,22 @@ public interface DistributedMetaStorage extends ReadableDistributedMetaStorage { ) throws IgniteCheckedException; /** + * Write value into distributed metastorage asynchronously but only if current value matches the expected one. + * + * @param key The key. + * @param expVal Expected value. Might be null. + * @param newVal Value to write. Must not be null. + * @throws IgniteCheckedException If cluster is in deactivated state. + * @return {@code True} if expected value matched the actual one and write was completed successfully. + * {@code False} otherwise. + */ + GridFutureAdapter<Boolean> compareAndSetAsync( + @NotNull String key, + @Nullable Serializable expVal, + @NotNull Serializable newVal + ) throws IgniteCheckedException; + + /** * Remove value from distributed metastorage but only if current value matches the expected one. * * @param key The key. diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/metastorage/persistence/DistributedMetaStorageImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/metastorage/persistence/DistributedMetaStorageImpl.java index 1bfb1ec..a2acb7b 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/metastorage/persistence/DistributedMetaStorageImpl.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/metastorage/persistence/DistributedMetaStorageImpl.java @@ -72,12 +72,11 @@ import static org.apache.ignite.internal.processors.metastorage.persistence.Dist import static org.apache.ignite.internal.processors.metastorage.persistence.DistributedMetaStorageUtil.unmarshal; /** - * Implementation of {@link DistributedMetaStorage} based on {@link MetaStorage} for persistence and discovery SPI - * for communication. + * Implementation of {@link DistributedMetaStorage} based on {@link MetaStorage} for persistence and discovery SPI for + * communication. */ public class DistributedMetaStorageImpl extends GridProcessorAdapter - implements DistributedMetaStorage, IgniteChangeGlobalStateSupport -{ + implements DistributedMetaStorage, IgniteChangeGlobalStateSupport { /** Component ID required for {@link DiscoveryDataBag} instances. */ private static final int COMPONENT_ID = META_STORAGE.ordinal(); @@ -132,8 +131,8 @@ public class DistributedMetaStorageImpl extends GridProcessorAdapter private final ConcurrentMap<UUID, GridFutureAdapter<Boolean>> updateFuts = new ConcurrentHashMap<>(); /** - * Some extra values that are useful only when node is not active. Otherwise it is nullized to remove - * excessive data from the heap. + * Some extra values that are useful only when node is not active. Otherwise it is nullized to remove excessive data + * from the heap. * * @see StartupExtras */ @@ -274,8 +273,8 @@ public class DistributedMetaStorageImpl extends GridProcessorAdapter } /** - * Implementation for {@link MetastorageLifecycleListener#onReadyForRead(ReadOnlyMetastorage)} listener. - * Invoked after node was started but before it was activated (only in persistent clusters). + * Implementation for {@link MetastorageLifecycleListener#onReadyForRead(ReadOnlyMetastorage)} listener. Invoked + * after node was started but before it was activated (only in persistent clusters). * * @param metastorage Local metastorage instance available for reading. * @throws IgniteCheckedException If there were any issues while metastorage reading. @@ -363,12 +362,20 @@ public class DistributedMetaStorageImpl extends GridProcessorAdapter @Override public void write(@NotNull String key, @NotNull Serializable val) throws IgniteCheckedException { assert val != null : key; - startWrite(key, marshal(val)); + startWrite(key, marshal(val)).get(); + } + + /** {@inheritDoc} */ + @Override public GridFutureAdapter<?> writeAsync(@NotNull String key, @NotNull Serializable val) + throws IgniteCheckedException { + assert val != null : key; + + return startWrite(key, marshal(val)); } /** {@inheritDoc} */ @Override public void remove(@NotNull String key) throws IgniteCheckedException { - startWrite(key, null); + startWrite(key, null).get(); } /** {@inheritDoc} */ @@ -379,6 +386,17 @@ public class DistributedMetaStorageImpl extends GridProcessorAdapter ) throws IgniteCheckedException { assert newVal != null : key; + return compareAndSetAsync(key, expVal, newVal).get(); + } + + /** {@inheritDoc} */ + @Override public GridFutureAdapter<Boolean> compareAndSetAsync( + @NotNull String key, + @Nullable Serializable expVal, + @NotNull Serializable newVal + ) throws IgniteCheckedException { + assert newVal != null : key; + return startCas(key, marshal(expVal), marshal(newVal)); } @@ -389,7 +407,7 @@ public class DistributedMetaStorageImpl extends GridProcessorAdapter ) throws IgniteCheckedException { assert expVal != null : key; - return startCas(key, marshal(expVal), null); + return startCas(key, marshal(expVal), null).get(); } /** {@inheritDoc} */ @@ -690,8 +708,8 @@ public class DistributedMetaStorageImpl extends GridProcessorAdapter } /** - * Returns actual version from the local node. It is just a version for activated node or calculated future - * version otherwise. + * Returns actual version from the local node. It is just a version for activated node or calculated future version + * otherwise. */ private DistributedMetaStorageVersion getActualVersion() { assert Thread.holdsLock(innerStateLock); @@ -795,9 +813,9 @@ public class DistributedMetaStorageImpl extends GridProcessorAdapter * @param key The key. * @param valBytes Value bytes to write. Null if value needs to be removed. * @throws IgniteCheckedException If there was an error while sending discovery message or message was sent but - * cluster is not active. + * cluster is not active. */ - private void startWrite(String key, byte[] valBytes) throws IgniteCheckedException { + private GridFutureAdapter<?> startWrite(String key, byte[] valBytes) throws IgniteCheckedException { UUID reqId = UUID.randomUUID(); GridFutureAdapter<Boolean> fut = new GridFutureAdapter<>(); @@ -808,13 +826,14 @@ public class DistributedMetaStorageImpl extends GridProcessorAdapter ctx.discovery().sendCustomEvent(msg); - fut.get(); + return fut; } /** * Basically the same as {@link #startWrite(String, byte[])} but for CAS operations. */ - private boolean startCas(String key, byte[] expValBytes, byte[] newValBytes) throws IgniteCheckedException { + private GridFutureAdapter<Boolean> startCas(String key, byte[] expValBytes, byte[] newValBytes) + throws IgniteCheckedException { UUID reqId = UUID.randomUUID(); GridFutureAdapter<Boolean> fut = new GridFutureAdapter<>(); @@ -825,7 +844,7 @@ public class DistributedMetaStorageImpl extends GridProcessorAdapter ctx.discovery().sendCustomEvent(msg); - return fut.get(); + return fut; } /** @@ -895,7 +914,7 @@ public class DistributedMetaStorageImpl extends GridProcessorAdapter ctx.failure().process(new FailureContext(FailureType.CRITICAL_ERROR, e)); if (e instanceof Error) - throw (Error) e; + throw (Error)e; throw U.convertException((IgniteCheckedException)e); } @@ -905,7 +924,8 @@ public class DistributedMetaStorageImpl extends GridProcessorAdapter * * @param bridge Bridge to get the access to the storage. * @param histItem {@code <key, value>} pair to process. - * @param notifyListeners Whether listeners should be notified or not. {@code false} for data restore on activation. + * @param notifyListeners Whether listeners should be notified or not. {@code false} for data restore on + * activation. * @throws IgniteCheckedException In case of IO/unmarshalling errors. */ private void completeWrite( diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/utils/PlatformConfigurationUtils.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/utils/PlatformConfigurationUtils.java index 4e0140c..dfd7d57 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/utils/PlatformConfigurationUtils.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/utils/PlatformConfigurationUtils.java @@ -662,12 +662,6 @@ public class PlatformConfigurationUtils { if (in.readBoolean()) cfg.setSystemWorkerBlockedTimeout(in.readLong()); if (in.readBoolean()) - cfg.setInitBaselineAutoAdjustEnabled(in.readBoolean()); - if (in.readBoolean()) - cfg.setInitBaselineAutoAdjustTimeout(in.readLong()); - if (in.readBoolean()) - cfg.setInitBaselineAutoAdjustMaxTimeout(in.readLong()); - if (in.readBoolean()) cfg.setSqlQueryHistorySize(in.readInt()); int sqlSchemasCnt = in.readInt(); @@ -1260,12 +1254,6 @@ public class PlatformConfigurationUtils { w.writeBoolean(false); } w.writeBoolean(true); - w.writeBoolean(cfg.isInitBaselineAutoAdjustEnabled()); - w.writeBoolean(true); - w.writeLong(cfg.getInitBaselineAutoAdjustTimeout()); - w.writeBoolean(true); - w.writeLong(cfg.getInitBaselineAutoAdjustMaxTimeout()); - w.writeBoolean(true); w.writeInt(cfg.getSqlQueryHistorySize()); if (cfg.getSqlSchemas() == null) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/IgniteUtils.java b/modules/core/src/main/java/org/apache/ignite/internal/util/IgniteUtils.java index 6c93dc9..2686840 100755 --- a/modules/core/src/main/java/org/apache/ignite/internal/util/IgniteUtils.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/util/IgniteUtils.java @@ -205,6 +205,7 @@ import org.apache.ignite.internal.events.DiscoveryCustomEvent; import org.apache.ignite.internal.managers.communication.GridIoManager; import org.apache.ignite.internal.managers.communication.GridIoPolicy; import org.apache.ignite.internal.managers.deployment.GridDeploymentInfo; +import org.apache.ignite.internal.managers.discovery.GridDiscoveryManager; import org.apache.ignite.internal.mxbean.IgniteStandardMXBean; import org.apache.ignite.internal.processors.cache.CacheClassLoaderMarker; import org.apache.ignite.internal.processors.cache.GridCacheAttributes; @@ -254,6 +255,7 @@ import org.apache.ignite.spi.IgniteSpiException; import org.apache.ignite.spi.discovery.DiscoverySpi; import org.apache.ignite.spi.discovery.DiscoverySpiOrderSupport; import org.apache.ignite.transactions.TransactionAlreadyCompletedException; +import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi; import org.apache.ignite.transactions.TransactionDeadlockException; import org.apache.ignite.transactions.TransactionDuplicateKeyException; import org.apache.ignite.transactions.TransactionHeuristicException; @@ -10904,6 +10906,17 @@ public abstract class IgniteUtils { } /** + * @return {@code true} if local node is coordinator. + */ + public static boolean isLocalNodeCoordinator(GridDiscoveryManager discoveryManager) { + DiscoverySpi spi = discoveryManager.getInjectedDiscoverySpi(); + + return spi instanceof TcpDiscoverySpi + ? ((TcpDiscoverySpi)spi).isLocalNodeCoordinator() + : F.eq(discoveryManager.localNode(), U.oldest(discoveryManager.aliveServerNodes(), null)); + } + + /** * The batch of tasks with a batch index in global array. */ private static class Batch<T,R> { diff --git a/modules/core/src/test/java/org/apache/ignite/internal/ClusterBaselineNodesMetricsSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/ClusterBaselineNodesMetricsSelfTest.java index 331f92d..3fd6480 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/ClusterBaselineNodesMetricsSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/ClusterBaselineNodesMetricsSelfTest.java @@ -35,6 +35,8 @@ import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; import org.apache.ignite.testframework.junits.common.GridCommonTest; import org.junit.Test; +import static org.apache.ignite.IgniteSystemProperties.IGNITE_BASELINE_AUTO_ADJUST_ENABLED; + /** * Baseline nodes metrics self test. */ @@ -45,6 +47,20 @@ public class ClusterBaselineNodesMetricsSelfTest extends GridCommonAbstractTest stopAllGrids(); } + /** {@inheritDoc} */ + @Override protected void beforeTestsStarted() throws Exception { + System.setProperty(IGNITE_BASELINE_AUTO_ADJUST_ENABLED, "false"); + + super.beforeTestsStarted(); + } + + /** {@inheritDoc} */ + @Override protected void afterTestsStopped() throws Exception { + super.afterTestsStopped(); + + System.clearProperty(IGNITE_BASELINE_AUTO_ADJUST_ENABLED); + } + /** * @throws Exception If failed. */ diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteClusterActivateDeactivateTestWithPersistence.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteClusterActivateDeactivateTestWithPersistence.java index c38809f..3bd6819 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteClusterActivateDeactivateTestWithPersistence.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteClusterActivateDeactivateTestWithPersistence.java @@ -45,6 +45,7 @@ import org.apache.ignite.testframework.GridTestUtils; import org.junit.Assert; import org.junit.Test; +import static org.apache.ignite.IgniteSystemProperties.IGNITE_BASELINE_AUTO_ADJUST_ENABLED; import static org.apache.ignite.testframework.GridTestUtils.assertThrowsAnyCause; /** @@ -57,6 +58,20 @@ public class IgniteClusterActivateDeactivateTestWithPersistence extends IgniteCl } /** {@inheritDoc} */ + @Override protected void beforeTestsStarted() throws Exception { + System.setProperty(IGNITE_BASELINE_AUTO_ADJUST_ENABLED, "false"); + + super.beforeTestsStarted(); + } + + /** {@inheritDoc} */ + @Override protected void afterTestsStopped() throws Exception { + super.afterTestsStopped(); + + System.clearProperty(IGNITE_BASELINE_AUTO_ADJUST_ENABLED); + } + + /** {@inheritDoc} */ @Override protected void beforeTest() throws Exception { super.beforeTest(); diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteDynamicCacheStartFailWithPersistenceTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteDynamicCacheStartFailWithPersistenceTest.java index 8643b66..09268da 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteDynamicCacheStartFailWithPersistenceTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteDynamicCacheStartFailWithPersistenceTest.java @@ -25,6 +25,8 @@ import org.apache.ignite.configuration.IgniteConfiguration; import org.apache.ignite.configuration.WALMode; import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi; +import static org.apache.ignite.IgniteSystemProperties.IGNITE_BASELINE_AUTO_ADJUST_ENABLED; + /** * Tests the recovery after a dynamic cache start failure, with enabled persistence. */ @@ -57,6 +59,8 @@ public class IgniteDynamicCacheStartFailWithPersistenceTest extends IgniteAbstra /** {@inheritDoc} */ @Override protected void beforeTestsStarted() throws Exception { + System.setProperty(IGNITE_BASELINE_AUTO_ADJUST_ENABLED, "false"); + cleanPersistenceDir(); startGrids(gridCount()); @@ -73,6 +77,8 @@ public class IgniteDynamicCacheStartFailWithPersistenceTest extends IgniteAbstra super.afterTestsStopped(); cleanPersistenceDir(); + + System.clearProperty(IGNITE_BASELINE_AUTO_ADJUST_ENABLED); } /** {@inheritDoc} */ diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/CacheBaselineTopologyTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/CacheBaselineTopologyTest.java index 755aac3..4292ba5 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/CacheBaselineTopologyTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/CacheBaselineTopologyTest.java @@ -60,6 +60,7 @@ import org.apache.ignite.transactions.TransactionConcurrency; import org.apache.ignite.transactions.TransactionIsolation; import org.junit.Test; +import static org.apache.ignite.IgniteSystemProperties.IGNITE_BASELINE_AUTO_ADJUST_ENABLED; import static org.apache.ignite.IgniteSystemProperties.IGNITE_WAL_LOG_TX_RECORDS; import static org.apache.ignite.cache.CacheMode.PARTITIONED; import static org.apache.ignite.cache.CacheWriteSynchronizationMode.FULL_SYNC; @@ -88,6 +89,20 @@ public class CacheBaselineTopologyTest extends GridCommonAbstractTest { private static final String DATA_NODE = "dataNodeUserAttr"; /** {@inheritDoc} */ + @Override protected void beforeTestsStarted() throws Exception { + System.setProperty(IGNITE_BASELINE_AUTO_ADJUST_ENABLED, "false"); + + super.beforeTestsStarted(); + } + + /** {@inheritDoc} */ + @Override protected void afterTestsStopped() throws Exception { + super.afterTestsStopped(); + + System.clearProperty(IGNITE_BASELINE_AUTO_ADJUST_ENABLED); + } + + /** {@inheritDoc} */ @Override protected void beforeTest() throws Exception { super.beforeTest(); diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/CacheDataLossOnPartitionMoveTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/CacheDataLossOnPartitionMoveTest.java index 8b28834..60d8b42 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/CacheDataLossOnPartitionMoveTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/CacheDataLossOnPartitionMoveTest.java @@ -47,6 +47,7 @@ import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; import org.jetbrains.annotations.Nullable; import org.junit.Test; +import static org.apache.ignite.IgniteSystemProperties.IGNITE_BASELINE_AUTO_ADJUST_ENABLED; import static org.apache.ignite.internal.processors.cache.distributed.dht.topology.GridDhtPartitionState.EVICTED; import static org.apache.ignite.internal.processors.cache.distributed.dht.topology.GridDhtPartitionState.OWNING; @@ -97,6 +98,20 @@ public class CacheDataLossOnPartitionMoveTest extends GridCommonAbstractTest { return cfg; } + /** {@inheritDoc} */ + @Override protected void beforeTestsStarted() throws Exception { + System.setProperty(IGNITE_BASELINE_AUTO_ADJUST_ENABLED, "false"); + + super.beforeTestsStarted(); + } + + /** {@inheritDoc} */ + @Override protected void afterTestsStopped() throws Exception { + super.afterTestsStopped(); + + System.clearProperty(IGNITE_BASELINE_AUTO_ADJUST_ENABLED); + } + /** * @param name Name. */ diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/CacheRentingStateRepairTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/CacheRentingStateRepairTest.java index 4533559..140e002 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/CacheRentingStateRepairTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/CacheRentingStateRepairTest.java @@ -34,6 +34,7 @@ import org.apache.ignite.internal.processors.cache.distributed.dht.topology.Grid import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; import org.junit.Test; +import static org.apache.ignite.IgniteSystemProperties.IGNITE_BASELINE_AUTO_ADJUST_ENABLED; import static org.apache.ignite.testframework.GridTestUtils.waitForCondition; /** @@ -78,6 +79,20 @@ public class CacheRentingStateRepairTest extends GridCommonAbstractTest { } /** {@inheritDoc} */ + @Override protected void beforeTestsStarted() throws Exception { + System.setProperty(IGNITE_BASELINE_AUTO_ADJUST_ENABLED, "false"); + + super.beforeTestsStarted(); + } + + /** {@inheritDoc} */ + @Override protected void afterTestsStopped() throws Exception { + super.afterTestsStopped(); + + System.clearProperty(IGNITE_BASELINE_AUTO_ADJUST_ENABLED); + } + + /** {@inheritDoc} */ @Override public String getTestIgniteInstanceName(int idx) { return "node" + idx; } diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/rebalancing/GridCacheRebalancingWithAsyncClearingTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/rebalancing/GridCacheRebalancingWithAsyncClearingTest.java index 03764e0..739cc99 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/rebalancing/GridCacheRebalancingWithAsyncClearingTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/rebalancing/GridCacheRebalancingWithAsyncClearingTest.java @@ -41,6 +41,8 @@ import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; import org.junit.Assert; import org.junit.Test; +import static org.apache.ignite.IgniteSystemProperties.IGNITE_BASELINE_AUTO_ADJUST_ENABLED; + /** * */ @@ -79,6 +81,20 @@ public class GridCacheRebalancingWithAsyncClearingTest extends GridCommonAbstrac } /** {@inheritDoc} */ + @Override protected void beforeTestsStarted() throws Exception { + System.setProperty(IGNITE_BASELINE_AUTO_ADJUST_ENABLED, "false"); + + super.beforeTestsStarted(); + } + + /** {@inheritDoc} */ + @Override protected void afterTestsStopped() throws Exception { + super.afterTestsStopped(); + + System.clearProperty(IGNITE_BASELINE_AUTO_ADJUST_ENABLED); + } + + /** {@inheritDoc} */ @Override protected void beforeTest() throws Exception { System.clearProperty(IgniteSystemProperties.IGNITE_PDS_MAX_CHECKPOINT_MEMORY_HISTORY_SIZE); diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccVacuumTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccVacuumTest.java index 3f952d8..d1864cb 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccVacuumTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccVacuumTest.java @@ -35,6 +35,7 @@ import org.apache.ignite.testframework.GridTestUtils; import org.junit.Ignore; import org.junit.Test; +import static org.apache.ignite.IgniteSystemProperties.IGNITE_BASELINE_AUTO_ADJUST_ENABLED; import static org.apache.ignite.cache.CacheMode.PARTITIONED; /** @@ -42,6 +43,20 @@ import static org.apache.ignite.cache.CacheMode.PARTITIONED; */ public class CacheMvccVacuumTest extends CacheMvccAbstractTest { /** {@inheritDoc} */ + @Override protected void beforeTestsStarted() throws Exception { + System.setProperty(IGNITE_BASELINE_AUTO_ADJUST_ENABLED, "false"); + + super.beforeTestsStarted(); + } + + /** {@inheritDoc} */ + @Override protected void afterTestsStopped() throws Exception { + super.afterTestsStopped(); + + System.clearProperty(IGNITE_BASELINE_AUTO_ADJUST_ENABLED); + } + + /** {@inheritDoc} */ @Override protected CacheMode cacheMode() { return PARTITIONED; } diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgniteBaselineAffinityTopologyActivationTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgniteBaselineAffinityTopologyActivationTest.java index cb504bb..50e8a7d 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgniteBaselineAffinityTopologyActivationTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgniteBaselineAffinityTopologyActivationTest.java @@ -56,6 +56,7 @@ import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; import org.junit.Assert; import org.junit.Test; +import static org.apache.ignite.IgniteSystemProperties.IGNITE_BASELINE_AUTO_ADJUST_ENABLED; import static org.apache.ignite.cache.CacheMode.PARTITIONED; /** @@ -100,6 +101,20 @@ public class IgniteBaselineAffinityTopologyActivationTest extends GridCommonAbst } /** {@inheritDoc} */ + @Override protected void beforeTestsStarted() throws Exception { + System.setProperty(IGNITE_BASELINE_AUTO_ADJUST_ENABLED, "false"); + + super.beforeTestsStarted(); + } + + /** {@inheritDoc} */ + @Override protected void afterTestsStopped() throws Exception { + super.afterTestsStopped(); + + System.clearProperty(IGNITE_BASELINE_AUTO_ADJUST_ENABLED); + } + + /** {@inheritDoc} */ @Override protected void beforeTest() throws Exception { super.beforeTest(); diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgnitePdsCorruptedIndexTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgnitePdsCorruptedIndexTest.java index 2a789f8..085b9b0 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgnitePdsCorruptedIndexTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgnitePdsCorruptedIndexTest.java @@ -50,6 +50,8 @@ import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; import org.apache.ignite.testframework.junits.multijvm.IgniteProcessProxy; import org.junit.Test; +import static org.apache.ignite.IgniteSystemProperties.IGNITE_BASELINE_AUTO_ADJUST_ENABLED; + /** * Test to reproduce corrupted indexes problem after partition file eviction and truncation. */ @@ -98,6 +100,20 @@ public class IgnitePdsCorruptedIndexTest extends GridCommonAbstractTest { } /** {@inheritDoc} */ + @Override protected void beforeTestsStarted() throws Exception { + System.setProperty(IGNITE_BASELINE_AUTO_ADJUST_ENABLED, "false"); + + super.beforeTestsStarted(); + } + + /** {@inheritDoc} */ + @Override protected void afterTestsStopped() throws Exception { + super.afterTestsStopped(); + + System.clearProperty(IGNITE_BASELINE_AUTO_ADJUST_ENABLED); + } + + /** {@inheritDoc} */ @Override protected void beforeTest() throws Exception { stopAllGrids(); diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgnitePdsPartitionFilesDestroyTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgnitePdsPartitionFilesDestroyTest.java index d09f346..9123ef5 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgnitePdsPartitionFilesDestroyTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgnitePdsPartitionFilesDestroyTest.java @@ -47,6 +47,7 @@ import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; import org.junit.Assert; import org.junit.Test; +import static org.apache.ignite.IgniteSystemProperties.IGNITE_BASELINE_AUTO_ADJUST_ENABLED; import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.DFLT_STORE_DIR; /** @@ -90,6 +91,20 @@ public class IgnitePdsPartitionFilesDestroyTest extends GridCommonAbstractTest { } /** {@inheritDoc} */ + @Override protected void beforeTestsStarted() throws Exception { + System.setProperty(IGNITE_BASELINE_AUTO_ADJUST_ENABLED, "false"); + + super.beforeTestsStarted(); + } + + /** {@inheritDoc} */ + @Override protected void afterTestsStopped() throws Exception { + super.afterTestsStopped(); + + System.clearProperty(IGNITE_BASELINE_AUTO_ADJUST_ENABLED); + } + + /** {@inheritDoc} */ @Override protected void beforeTest() throws Exception { stopAllGrids(); diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgniteRebalanceScheduleResendPartitionsTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgniteRebalanceScheduleResendPartitionsTest.java index de9992d..5b5a2281 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgniteRebalanceScheduleResendPartitionsTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgniteRebalanceScheduleResendPartitionsTest.java @@ -49,6 +49,7 @@ import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; import org.junit.Assert; import org.junit.Test; +import static org.apache.ignite.IgniteSystemProperties.IGNITE_BASELINE_AUTO_ADJUST_ENABLED; import static org.apache.ignite.testframework.GridTestUtils.runAsync; /** @@ -85,6 +86,20 @@ public class IgniteRebalanceScheduleResendPartitionsTest extends GridCommonAbstr } /** {@inheritDoc} */ + @Override protected void beforeTestsStarted() throws Exception { + System.setProperty(IGNITE_BASELINE_AUTO_ADJUST_ENABLED, "false"); + + super.beforeTestsStarted(); + } + + /** {@inheritDoc} */ + @Override protected void afterTestsStopped() throws Exception { + super.afterTestsStopped(); + + System.clearProperty(IGNITE_BASELINE_AUTO_ADJUST_ENABLED); + } + + /** {@inheritDoc} */ @Override protected void beforeTest() throws Exception { super.beforeTest(); diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/LocalWalModeChangeDuringRebalancingSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/LocalWalModeChangeDuringRebalancingSelfTest.java index 8a505cb..78940a8 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/LocalWalModeChangeDuringRebalancingSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/LocalWalModeChangeDuringRebalancingSelfTest.java @@ -59,6 +59,7 @@ import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; import org.junit.Assert; import org.junit.Test; +import static org.apache.ignite.IgniteSystemProperties.IGNITE_BASELINE_AUTO_ADJUST_ENABLED; import static org.apache.ignite.testframework.GridTestUtils.waitForCondition; /** @@ -169,6 +170,8 @@ public class LocalWalModeChangeDuringRebalancingSelfTest extends GridCommonAbstr /** {@inheritDoc} */ @Override protected void beforeTestsStarted() throws Exception { + System.setProperty(IGNITE_BASELINE_AUTO_ADJUST_ENABLED, "false"); + super.beforeTestsStarted(); cleanPersistenceDir(); @@ -212,6 +215,8 @@ public class LocalWalModeChangeDuringRebalancingSelfTest extends GridCommonAbstr System.clearProperty(IgniteSystemProperties.IGNITE_DISABLE_WAL_DURING_REBALANCING); System.clearProperty(IgniteSystemProperties.IGNITE_PENDING_TX_TRACKER_ENABLED); + + System.clearProperty(IGNITE_BASELINE_AUTO_ADJUST_ENABLED); } /** diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/baseline/ClientAffinityAssignmentWithBaselineTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/baseline/ClientAffinityAssignmentWithBaselineTest.java index f606ad1..f081aac 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/baseline/ClientAffinityAssignmentWithBaselineTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/baseline/ClientAffinityAssignmentWithBaselineTest.java @@ -57,6 +57,7 @@ import org.apache.ignite.transactions.TransactionIsolation; import org.junit.Assume; import org.junit.Test; +import static org.apache.ignite.IgniteSystemProperties.IGNITE_BASELINE_AUTO_ADJUST_ENABLED; import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.DFLT_STORE_DIR; /** @@ -158,6 +159,20 @@ public class ClientAffinityAssignmentWithBaselineTest extends GridCommonAbstract } /** {@inheritDoc} */ + @Override protected void beforeTestsStarted() throws Exception { + System.setProperty(IGNITE_BASELINE_AUTO_ADJUST_ENABLED, "false"); + + super.beforeTestsStarted(); + } + + /** {@inheritDoc} */ + @Override protected void afterTestsStopped() throws Exception { + super.afterTestsStopped(); + + System.clearProperty(IGNITE_BASELINE_AUTO_ADJUST_ENABLED); + } + + /** {@inheritDoc} */ @Override protected void beforeTest() throws Exception { Assume.assumeFalse("https://issues.apache.org/jira/browse/IGNITE-10261", MvccFeatureChecker.forcedMvcc()); diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/baseline/IgniteAbsentEvictionNodeOutOfBaselineTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/baseline/IgniteAbsentEvictionNodeOutOfBaselineTest.java index 82e1f93..221d214 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/baseline/IgniteAbsentEvictionNodeOutOfBaselineTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/baseline/IgniteAbsentEvictionNodeOutOfBaselineTest.java @@ -29,6 +29,8 @@ import org.apache.ignite.internal.processors.cache.distributed.dht.topology.Grid import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; import org.junit.Test; +import static org.apache.ignite.IgniteSystemProperties.IGNITE_BASELINE_AUTO_ADJUST_ENABLED; + /** * Test absenting eviction for joined node if it is out of baseline. */ @@ -71,6 +73,15 @@ public class IgniteAbsentEvictionNodeOutOfBaselineTest extends GridCommonAbstrac stopAllGrids(); cleanPersistenceDir(); + + System.clearProperty(IGNITE_BASELINE_AUTO_ADJUST_ENABLED); + } + + /** {@inheritDoc} */ + @Override protected void beforeTestsStarted() throws Exception { + System.setProperty(IGNITE_BASELINE_AUTO_ADJUST_ENABLED, "false"); + + super.beforeTestsStarted(); } /** diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/baseline/IgniteChangingBaselineDownCachePutAllFailoverTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/baseline/IgniteChangingBaselineDownCachePutAllFailoverTest.java index c64284b..6d8548f 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/baseline/IgniteChangingBaselineDownCachePutAllFailoverTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/baseline/IgniteChangingBaselineDownCachePutAllFailoverTest.java @@ -32,6 +32,8 @@ import org.apache.ignite.internal.processors.cache.distributed.CachePutAllFailov import org.apache.ignite.internal.util.typedef.internal.U; import org.apache.ignite.testframework.GridTestUtils; +import static org.apache.ignite.IgniteSystemProperties.IGNITE_BASELINE_AUTO_ADJUST_ENABLED; + /** * Failover test for cache putAll operations when BaselineTopology is changed down * (existing node gets stopped and removed from BaselineTopology). @@ -60,6 +62,8 @@ public class IgniteChangingBaselineDownCachePutAllFailoverTest extends CachePutA /** {@inheritDoc} */ @Override protected void beforeTestsStarted() throws Exception { + System.setProperty(IGNITE_BASELINE_AUTO_ADJUST_ENABLED, "false"); + cleanPersistenceDir(); } @@ -79,6 +83,8 @@ public class IgniteChangingBaselineDownCachePutAllFailoverTest extends CachePutA stopAllGrids(); cleanPersistenceDir(); + + System.clearProperty(IGNITE_BASELINE_AUTO_ADJUST_ENABLED); } /** {@inheritDoc} */ diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/baseline/IgniteChangingBaselineDownCacheRemoveFailoverTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/baseline/IgniteChangingBaselineDownCacheRemoveFailoverTest.java index dfe1b1e..7c3d992 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/baseline/IgniteChangingBaselineDownCacheRemoveFailoverTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/baseline/IgniteChangingBaselineDownCacheRemoveFailoverTest.java @@ -37,6 +37,7 @@ import org.apache.ignite.internal.processors.cache.GridCacheAbstractRemoveFailur import org.apache.ignite.internal.util.typedef.internal.U; import org.apache.ignite.testframework.GridTestUtils; +import static org.apache.ignite.IgniteSystemProperties.IGNITE_BASELINE_AUTO_ADJUST_ENABLED; import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL; import static org.apache.ignite.cache.CacheMode.PARTITIONED; @@ -82,6 +83,8 @@ public class IgniteChangingBaselineDownCacheRemoveFailoverTest extends GridCache /** {@inheritDoc} */ @Override protected void beforeTestsStarted() throws Exception { + System.setProperty(IGNITE_BASELINE_AUTO_ADJUST_ENABLED, "false"); + cleanPersistenceDir(); } @@ -101,6 +104,8 @@ public class IgniteChangingBaselineDownCacheRemoveFailoverTest extends GridCache /** {@inheritDoc} */ @Override protected void afterTestsStopped() throws Exception { cleanPersistenceDir(); + + System.clearProperty(IGNITE_BASELINE_AUTO_ADJUST_ENABLED); } /** {@inheritDoc} */ diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/baseline/IgniteChangingBaselineUpCachePutAllFailoverTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/baseline/IgniteChangingBaselineUpCachePutAllFailoverTest.java index 2e78464..e40334c 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/baseline/IgniteChangingBaselineUpCachePutAllFailoverTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/baseline/IgniteChangingBaselineUpCachePutAllFailoverTest.java @@ -31,6 +31,8 @@ import org.apache.ignite.internal.processors.cache.distributed.CachePutAllFailov import org.apache.ignite.internal.util.typedef.internal.U; import org.apache.ignite.testframework.GridTestUtils; +import static org.apache.ignite.IgniteSystemProperties.IGNITE_BASELINE_AUTO_ADJUST_ENABLED; + /** * Failover test for cache putAll operations when BaselineTopology is changed up * (new node is added to BaselineTopology). @@ -59,6 +61,8 @@ public class IgniteChangingBaselineUpCachePutAllFailoverTest extends CachePutAll /** {@inheritDoc} */ @Override protected void beforeTestsStarted() throws Exception { + System.setProperty(IGNITE_BASELINE_AUTO_ADJUST_ENABLED, "false"); + cleanPersistenceDir(); } @@ -83,6 +87,8 @@ public class IgniteChangingBaselineUpCachePutAllFailoverTest extends CachePutAll /** {@inheritDoc} */ @Override protected void afterTestsStopped() throws Exception { cleanPersistenceDir(); + + System.clearProperty(IGNITE_BASELINE_AUTO_ADJUST_ENABLED); } /** {@inheritDoc} */ diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/baseline/IgniteChangingBaselineUpCacheRemoveFailoverTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/baseline/IgniteChangingBaselineUpCacheRemoveFailoverTest.java index 8f15d77..4730c0b 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/baseline/IgniteChangingBaselineUpCacheRemoveFailoverTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/baseline/IgniteChangingBaselineUpCacheRemoveFailoverTest.java @@ -37,6 +37,7 @@ import org.apache.ignite.internal.processors.cache.GridCacheAbstractRemoveFailur import org.apache.ignite.internal.util.typedef.internal.U; import org.apache.ignite.testframework.GridTestUtils; +import static org.apache.ignite.IgniteSystemProperties.IGNITE_BASELINE_AUTO_ADJUST_ENABLED; import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL; import static org.apache.ignite.cache.CacheMode.PARTITIONED; @@ -81,6 +82,8 @@ public class IgniteChangingBaselineUpCacheRemoveFailoverTest extends GridCacheAb /** {@inheritDoc} */ @Override protected void beforeTestsStarted() throws Exception { + System.setProperty(IGNITE_BASELINE_AUTO_ADJUST_ENABLED, "false"); + cleanPersistenceDir(); } @@ -98,6 +101,8 @@ public class IgniteChangingBaselineUpCacheRemoveFailoverTest extends GridCacheAb /** {@inheritDoc} */ @Override protected void afterTestsStopped() throws Exception { cleanPersistenceDir(); + + System.clearProperty(IGNITE_BASELINE_AUTO_ADJUST_ENABLED); } /** {@inheritDoc} */ diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/IgniteLogicalRecoveryTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/IgniteLogicalRecoveryTest.java index 6b23881..3c6806c 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/IgniteLogicalRecoveryTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/IgniteLogicalRecoveryTest.java @@ -63,6 +63,8 @@ import org.jetbrains.annotations.Nullable; import org.junit.Assert; import org.junit.Test; +import static org.apache.ignite.IgniteSystemProperties.IGNITE_BASELINE_AUTO_ADJUST_ENABLED; + /** * A set of tests that check correctness of logical recovery performed during node start. */ @@ -124,6 +126,20 @@ public class IgniteLogicalRecoveryTest extends GridCommonAbstractTest { return cfg; } + /** {@inheritDoc} */ + @Override protected void beforeTestsStarted() throws Exception { + System.setProperty(IGNITE_BASELINE_AUTO_ADJUST_ENABLED, "false"); + + super.beforeTestsStarted(); + } + + /** {@inheritDoc} */ + @Override protected void afterTestsStopped() throws Exception { + super.afterTestsStopped(); + + System.clearProperty(IGNITE_BASELINE_AUTO_ADJUST_ENABLED); + } + /** * @param name Name. * @param cacheMode Cache mode. diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/IgnitePdsCacheWalDisabledOnRebalancingTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/IgnitePdsCacheWalDisabledOnRebalancingTest.java index 92b572d..21ce122 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/IgnitePdsCacheWalDisabledOnRebalancingTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/IgnitePdsCacheWalDisabledOnRebalancingTest.java @@ -57,6 +57,7 @@ import org.apache.ignite.testframework.GridTestUtils; import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; import org.junit.Test; +import static org.apache.ignite.IgniteSystemProperties.IGNITE_BASELINE_AUTO_ADJUST_ENABLED; import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.DFLT_STORE_DIR; /** @@ -95,6 +96,20 @@ public class IgnitePdsCacheWalDisabledOnRebalancingTest extends GridCommonAbstra private boolean useBlockingFileIO; /** {@inheritDoc} */ + @Override protected void beforeTestsStarted() throws Exception { + System.setProperty(IGNITE_BASELINE_AUTO_ADJUST_ENABLED, "false"); + + super.beforeTestsStarted(); + } + + /** {@inheritDoc} */ + @Override protected void afterTestsStopped() throws Exception { + super.afterTestsStopped(); + + System.clearProperty(IGNITE_BASELINE_AUTO_ADJUST_ENABLED); + } + + /** {@inheritDoc} */ @Override protected void beforeTest() throws Exception { super.beforeTest(); diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/IgnitePdsDataRegionMetricsTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/IgnitePdsDataRegionMetricsTest.java index c627ef2..df52c91 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/IgnitePdsDataRegionMetricsTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/IgnitePdsDataRegionMetricsTest.java @@ -49,6 +49,7 @@ import org.junit.Assert; import org.junit.Test; import static java.nio.file.Files.newDirectoryStream; +import static org.apache.ignite.IgniteSystemProperties.IGNITE_BASELINE_AUTO_ADJUST_ENABLED; import static org.apache.ignite.configuration.DataStorageConfiguration.DFLT_DATA_REG_DEFAULT_NAME; import static org.apache.ignite.internal.processors.cache.GridCacheUtils.UTILITY_CACHE_NAME; import static org.apache.ignite.internal.processors.cache.mvcc.txlog.TxLog.TX_LOG_CACHE_NAME; @@ -100,6 +101,19 @@ public class IgnitePdsDataRegionMetricsTest extends GridCommonAbstractTest { return cfg; } + /** {@inheritDoc} */ + @Override protected void beforeTestsStarted() throws Exception { + System.setProperty(IGNITE_BASELINE_AUTO_ADJUST_ENABLED, "false"); + + super.beforeTestsStarted(); + } + + /** {@inheritDoc} */ + @Override protected void afterTestsStopped() throws Exception { + super.afterTestsStopped(); + + System.clearProperty(IGNITE_BASELINE_AUTO_ADJUST_ENABLED); + } /** * @return Ignite cache configuration. */ diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/IgnitePdsWithTtlTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/IgnitePdsWithTtlTest.java index c8d313f..e2db501 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/IgnitePdsWithTtlTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/IgnitePdsWithTtlTest.java @@ -43,6 +43,8 @@ import org.apache.ignite.testframework.MvccFeatureChecker; import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; import org.junit.Test; +import static org.apache.ignite.IgniteSystemProperties.IGNITE_BASELINE_AUTO_ADJUST_ENABLED; + /** * Test TTL worker with persistence enabled */ @@ -57,6 +59,20 @@ public class IgnitePdsWithTtlTest extends GridCommonAbstractTest { public static final int ENTRIES = 100_000; /** {@inheritDoc} */ + @Override protected void beforeTestsStarted() throws Exception { + System.setProperty(IGNITE_BASELINE_AUTO_ADJUST_ENABLED, "false"); + + super.beforeTestsStarted(); + } + + /** {@inheritDoc} */ + @Override protected void afterTestsStopped() throws Exception { + super.afterTestsStopped(); + + System.clearProperty(IGNITE_BASELINE_AUTO_ADJUST_ENABLED); + } + + /** {@inheritDoc} */ @Override protected void beforeTest() throws Exception { MvccFeatureChecker.skipIfNotSupported(MvccFeatureChecker.Feature.EXPIRATION); diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/SlowHistoricalRebalanceSmallHistoryTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/SlowHistoricalRebalanceSmallHistoryTest.java index 2cf1fac..cbd8a1b 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/SlowHistoricalRebalanceSmallHistoryTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/SlowHistoricalRebalanceSmallHistoryTest.java @@ -43,6 +43,8 @@ import org.apache.ignite.spi.communication.tcp.TcpCommunicationSpi; import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; import org.junit.Test; +import static org.apache.ignite.IgniteSystemProperties.IGNITE_BASELINE_AUTO_ADJUST_ENABLED; + /** * */ @@ -109,6 +111,20 @@ public class SlowHistoricalRebalanceSmallHistoryTest extends GridCommonAbstractT cleanPersistenceDir(); } + /** {@inheritDoc} */ + @Override protected void beforeTestsStarted() throws Exception { + System.setProperty(IGNITE_BASELINE_AUTO_ADJUST_ENABLED, "false"); + + super.beforeTestsStarted(); + } + + /** {@inheritDoc} */ + @Override protected void afterTestsStopped() throws Exception { + super.afterTestsStopped(); + + System.clearProperty(IGNITE_BASELINE_AUTO_ADJUST_ENABLED); + } + /** * Checks that we reserve and release the same WAL index on exchange. */ diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgniteWalFlushMultiNodeFailoverAbstractSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgniteWalFlushMultiNodeFailoverAbstractSelfTest.java index 0b3dd1b..5d25da2 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgniteWalFlushMultiNodeFailoverAbstractSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgniteWalFlushMultiNodeFailoverAbstractSelfTest.java @@ -51,6 +51,8 @@ import org.apache.ignite.transactions.TransactionIsolation; import org.junit.Assume; import org.junit.Test; +import static org.apache.ignite.IgniteSystemProperties.IGNITE_BASELINE_AUTO_ADJUST_ENABLED; + /** * Tests error recovery while node flushing */ @@ -94,6 +96,20 @@ public abstract class IgniteWalFlushMultiNodeFailoverAbstractSelfTest extends Gr } /** {@inheritDoc} */ + @Override protected void beforeTestsStarted() throws Exception { + System.setProperty(IGNITE_BASELINE_AUTO_ADJUST_ENABLED, "false"); + + super.beforeTestsStarted(); + } + + /** {@inheritDoc} */ + @Override protected void afterTestsStopped() throws Exception { + super.afterTestsStopped(); + + System.clearProperty(IGNITE_BASELINE_AUTO_ADJUST_ENABLED); + } + + /** {@inheritDoc} */ @Override protected long getTestTimeout() { return 120_000; } diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/WalCompactionTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/WalCompactionTest.java index b156669..89bf6b6 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/WalCompactionTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/WalCompactionTest.java @@ -45,6 +45,8 @@ import org.apache.ignite.internal.util.typedef.internal.U; import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; import org.junit.Test; +import static org.apache.ignite.IgniteSystemProperties.IGNITE_BASELINE_AUTO_ADJUST_ENABLED; + /** * */ @@ -100,6 +102,20 @@ public class WalCompactionTest extends GridCommonAbstractTest { } /** {@inheritDoc} */ + @Override protected void beforeTestsStarted() throws Exception { + System.setProperty(IGNITE_BASELINE_AUTO_ADJUST_ENABLED, "false"); + + super.beforeTestsStarted(); + } + + /** {@inheritDoc} */ + @Override protected void afterTestsStopped() throws Exception { + super.afterTestsStopped(); + + System.clearProperty(IGNITE_BASELINE_AUTO_ADJUST_ENABLED); + } + + /** {@inheritDoc} */ @Override protected void beforeTest() throws Exception { stopAllGrids(); diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cluster/ChangeTopologyWatcherTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cluster/ChangeTopologyWatcherTest.java new file mode 100644 index 0000000..1b97edf --- /dev/null +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cluster/ChangeTopologyWatcherTest.java @@ -0,0 +1,350 @@ +/* + * 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.cluster; + +import java.util.Arrays; +import java.util.Collection; +import java.util.Set; +import java.util.stream.Collectors; +import org.apache.ignite.Ignite; +import org.apache.ignite.IgniteCache; +import org.apache.ignite.cache.PartitionLossPolicy; +import org.apache.ignite.cluster.BaselineNode; +import org.apache.ignite.configuration.CacheConfiguration; +import org.apache.ignite.configuration.DataStorageConfiguration; +import org.apache.ignite.configuration.IgniteConfiguration; +import org.apache.ignite.internal.IgniteEx; +import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; +import org.apache.ignite.testframework.junits.common.GridCommonTest; +import org.junit.After; +import org.junit.Before; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +import static org.apache.ignite.testframework.GridTestUtils.waitForCondition; + +/** + * + */ +@GridCommonTest(group = "Kernal Self") +@RunWith(JUnit4.class) +public class ChangeTopologyWatcherTest extends GridCommonAbstractTest { + /** */ + private static final String TEST_NAME = "TEST_NAME"; + /** */ + private static int AUTO_ADJUST_TIMEOUT = 5000; + + /** + * @throws Exception if failed. + */ + @Before + public void before() throws Exception { + stopAllGrids(); + + cleanPersistenceDir(); + + AUTO_ADJUST_TIMEOUT = 5000; + } + + /** + * @throws Exception if failed. + */ + @After + public void after() throws Exception { + stopAllGrids(); + + cleanPersistenceDir(); + } + + /** {@inheritDoc} */ + @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception { + IgniteConfiguration cfg = super.getConfiguration(igniteInstanceName); + + cfg.setConsistentId(igniteInstanceName); + + DataStorageConfiguration storageCfg = new DataStorageConfiguration(); + + storageCfg.getDefaultDataRegionConfiguration() + .setPersistenceEnabled(true) + .setMaxSize(500L * 1024 * 1024); + + cfg.setDataStorageConfiguration(storageCfg); + + return cfg; + } + + /** + * @throws Exception if failed. + */ + @Test + public void testBaselineAutoAdjustAfterNodeLeft() throws Exception { + Ignite ignite0 = startGrids(2); + + ignite0.cluster().active(true); + + ignite0.cluster().baselineConfiguration().updateBaselineAutoAdjustTimeoutAsync(AUTO_ADJUST_TIMEOUT).get(); + + Set<Object> initBaseline = ignite0.cluster().currentBaselineTopology().stream() + .map(BaselineNode::consistentId) + .collect(Collectors.toSet()); + + stopGrid(1); + + Set<Object> nodeLeftBaseline = ignite0.cluster().currentBaselineTopology().stream() + .map(BaselineNode::consistentId) + .collect(Collectors.toSet()); + + assertEquals(initBaseline, nodeLeftBaseline); + + assertTrue(waitForCondition( + () -> isCurrentBaselineFromOneNode(ignite0), + AUTO_ADJUST_TIMEOUT * 2 + )); + } + + /** + * @throws Exception if failed. + */ + @Test + public void testBaselineAutoAdjustSinceSecondNodeLeft() throws Exception { + Ignite ignite0 = startGrids(3); + + ignite0.cluster().active(true); + + ignite0.cluster().baselineConfiguration().updateBaselineAutoAdjustTimeoutAsync(AUTO_ADJUST_TIMEOUT).get(); + + Set<Object> initBaseline = ignite0.cluster().currentBaselineTopology().stream() + .map(BaselineNode::consistentId) + .collect(Collectors.toSet()); + + stopGrid(1); + + doSleep(AUTO_ADJUST_TIMEOUT / 2); + + stopGrid(2); + + doSleep(AUTO_ADJUST_TIMEOUT / 2); + + Set<Object> twoNodeLeftBaseline = ignite0.cluster().currentBaselineTopology().stream() + .map(BaselineNode::consistentId) + .collect(Collectors.toSet()); + + assertEquals(initBaseline, twoNodeLeftBaseline); + + assertTrue(waitForCondition( + () -> isCurrentBaselineFromOneNode(ignite0), + AUTO_ADJUST_TIMEOUT * 2 + )); + } + + /** + * @throws Exception if failed. + */ + @Test + public void testBaselineAutoAdjustSinceCoordinatorLeft() throws Exception { + Ignite ignite0 = startGrids(3); + + ignite0.cluster().active(true); + + ignite0.cluster().baselineConfiguration().updateBaselineAutoAdjustTimeoutAsync(AUTO_ADJUST_TIMEOUT).get(); + + Set<Object> initBaseline = ignite0.cluster().currentBaselineTopology().stream() + .map(BaselineNode::consistentId) + .collect(Collectors.toSet()); + + stopGrid(1); + + doSleep(AUTO_ADJUST_TIMEOUT / 2); + + stopGrid(0); + + doSleep(AUTO_ADJUST_TIMEOUT / 2); + + Ignite ignite2 = ignite(2); + + Set<Object> twoNodeLeftBaseline = ignite2.cluster().currentBaselineTopology().stream() + .map(BaselineNode::consistentId) + .collect(Collectors.toSet()); + + assertEquals(initBaseline, twoNodeLeftBaseline); + + assertTrue(waitForCondition( + () -> isCurrentBaselineFromOneNode(ignite2), + AUTO_ADJUST_TIMEOUT + )); + } + + /** + * @throws Exception if failed. + */ + @Test + public void testBaselineAutoAdjustAfterNodeJoin() throws Exception { + IgniteEx ignite0 = startGrid(0); + + ignite0.cluster().active(true); + + ignite0.cluster().baselineConfiguration().updateBaselineAutoAdjustTimeoutAsync(AUTO_ADJUST_TIMEOUT).get(); + + assertTrue(isCurrentBaselineFromOneNode(ignite0)); + + startGrid(1); + + assertTrue(isCurrentBaselineFromOneNode(ignite0)); + + assertTrue(waitForCondition( + () -> ignite0.cluster().currentBaselineTopology().size() == 2, + AUTO_ADJUST_TIMEOUT * 2 + )); + } + + /** + * @param ignite0 Node to check. + * @return {@code true} if current baseline consist from one node. + */ + private boolean isCurrentBaselineFromOneNode(Ignite ignite0) { + return ignite0.cluster().currentBaselineTopology().stream() + .map(BaselineNode::consistentId) + .allMatch(((IgniteEx)ignite0).localNode().consistentId()::equals); + } + + /** + * @throws Exception if failed. + */ + @Test + public void testBaselineAutoAdjustDisabledAfterGridHasLostPart() throws Exception { + AUTO_ADJUST_TIMEOUT = 0; + + Ignite ignite0 = startGrids(2); + + ignite0.cluster().active(true); + + Set<Object> initBaseline = ignite0.cluster().currentBaselineTopology().stream() + .map(BaselineNode::consistentId) + .collect(Collectors.toSet()); + + ignite0.cluster().baselineConfiguration().updateBaselineAutoAdjustTimeoutAsync(AUTO_ADJUST_TIMEOUT).get(); + + IgniteCache<Object, Object> cache = ignite0.getOrCreateCache(new CacheConfiguration<>(TEST_NAME) + .setBackups(0) + .setPartitionLossPolicy(PartitionLossPolicy.READ_ONLY_SAFE) + ); + + for (int j = 0; j < 500; j++) + cache.put(j, "Value" + j); + + stopGrid(1); + + doSleep(3000); + + Set<Object> baselineAfterNodeLeft = ignite0.cluster().currentBaselineTopology().stream() + .map(BaselineNode::consistentId) + .collect(Collectors.toSet()); + + assertEquals(initBaseline, baselineAfterNodeLeft); + } + + /** + * @throws Exception if failed. + */ + @Test(expected = BaselineAdjustForbiddenException.class) + public void testBaselineAutoAdjustThrowExceptionWhenBaselineChangedManually() throws Exception { + Ignite ignite0 = startGrids(2); + + ignite0.cluster().active(true); + + ignite0.cluster().baselineConfiguration().updateBaselineAutoAdjustTimeoutAsync(AUTO_ADJUST_TIMEOUT).get(); + + Collection<BaselineNode> baselineNodes = ignite0.cluster().currentBaselineTopology(); + + assertEquals(2, baselineNodes.size()); + + stopGrid(1); + + ignite0.cluster().setBaselineTopology(Arrays.asList(((IgniteEx)ignite0).localNode())); + } + + /** + * @throws Exception if failed. + */ + @Test + public void testBaselineAutoAdjustTriggeredAfterFirstEventRegardlessInitBaseline() throws Exception { + AUTO_ADJUST_TIMEOUT = 3000; + + Ignite ignite0 = startGrids(3); + + ignite0.cluster().active(true); + + ignite0.cluster().baselineConfiguration().updateBaselineAutoAdjustTimeoutAsync(AUTO_ADJUST_TIMEOUT).get(); + + assertTrue(ignite0.cluster().baselineConfiguration().isBaselineAutoAdjustEnabled()); + + Set<Object> baselineNodes = ignite0.cluster().currentBaselineTopology().stream() + .map(BaselineNode::consistentId) + .collect(Collectors.toSet()); + + stopAllGrids(); + + ignite0 = startGrids(2); + + ignite0.cluster().active(true); + + Set<Object> baselineNodesAfterRestart = ignite0.cluster().currentBaselineTopology().stream() + .map(BaselineNode::consistentId) + .collect(Collectors.toSet()); + + assertEquals(baselineNodes, baselineNodesAfterRestart); + + stopGrid(1); + + Ignite finalIgnite = ignite0; + + assertTrue(waitForCondition( + () -> isCurrentBaselineFromOneNode(finalIgnite) + , + AUTO_ADJUST_TIMEOUT * 2 + )); + } + + /** + * @throws Exception if failed. + */ + @Test + public void testBaselineAutoAdjustIgnoreClientNodes() throws Exception { + IgniteEx ignite0 = startGrid(0); + startGrid(1); + + ignite0.cluster().active(true); + + ignite0.cluster().baselineConfiguration().updateBaselineAutoAdjustTimeoutAsync(AUTO_ADJUST_TIMEOUT).get(); + + assertTrue(ignite0.cluster().baselineConfiguration().isBaselineAutoAdjustEnabled()); + + stopGrid(1); + + doSleep(AUTO_ADJUST_TIMEOUT / 2); + + IgniteEx igniteClient = startGrid(getConfiguration(getTestIgniteInstanceName(2)).setClientMode(true)); + + doSleep(AUTO_ADJUST_TIMEOUT / 2); + + igniteClient.close(); + + assertTrue(isCurrentBaselineFromOneNode(ignite0)); + } +} \ No newline at end of file diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/configuration/distributed/DistributedConfigurationTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/configuration/distributed/DistributedConfigurationTest.java index c3100ea..c613a85 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/configuration/distributed/DistributedConfigurationTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/configuration/distributed/DistributedConfigurationTest.java @@ -17,7 +17,6 @@ package org.apache.ignite.internal.processors.configuration.distributed; -import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.configuration.DataStorageConfiguration; import org.apache.ignite.configuration.IgniteConfiguration; import org.apache.ignite.internal.IgniteEx; @@ -154,9 +153,6 @@ public class DistributedConfigurationTest extends GridCommonAbstractTest { long0 = ignite0.context().distributedConfiguration().registerLong(TEST_PROP, 0L); assertEquals(2, long0.value().longValue()); - - //Cluster wide update have not initialized yet. - assertFalse(long0.propagate(3L)); } /** @@ -186,7 +182,7 @@ public class DistributedConfigurationTest extends GridCommonAbstractTest { */ @Test(expected = DetachedPropertyException.class) public void testNotAttachedProperty() throws Exception { - DistributedLongProperty long0 = DistributedLongProperty.detachedProperty(TEST_PROP, 0L); + DistributedLongProperty long0 = DistributedLongProperty.detachedLongProperty(TEST_PROP, 0L); assertEquals(0, long0.value().longValue()); long0.propagate(1L); @@ -195,6 +191,18 @@ public class DistributedConfigurationTest extends GridCommonAbstractTest { /** * @throws Exception If failed. */ + @Test(expected = NotWritablePropertyException.class) + public void testPropagateValueOnInactiveGridShouldThrowException() throws Exception { + IgniteEx ignite0 = (IgniteEx)startGrids(2); + + DistributedLongProperty long0 = ignite0.context().distributedConfiguration().registerLong(TEST_PROP, 0L); + + long0.propagate(2L); + } + + /** + * @throws Exception If failed. + */ @Test public void testReadInitValueBeforeOnReadyForReady() throws Exception { IgniteEx ignite0 = startGrid(0); @@ -216,13 +224,6 @@ public class DistributedConfigurationTest extends GridCommonAbstractTest { //Read init value because onReadyForReady have not happened yet. assertEquals(-1, longProperty.value().longValue()); - - try { - assertFalse(longProperty.propagate(1L)); - } - catch (IgniteCheckedException e) { - throw new RuntimeException(e); - } }; ignite0 = startGrid(0); diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/service/IgniteServiceReassignmentTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/service/IgniteServiceReassignmentTest.java index e26e8b4..c054ae3 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/service/IgniteServiceReassignmentTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/service/IgniteServiceReassignmentTest.java @@ -38,6 +38,8 @@ import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; import org.junit.Assume; import org.junit.Test; +import static org.apache.ignite.IgniteSystemProperties.IGNITE_BASELINE_AUTO_ADJUST_ENABLED; + /** * */ @@ -72,6 +74,20 @@ public class IgniteServiceReassignmentTest extends GridCommonAbstractTest { } /** {@inheritDoc} */ + @Override protected void beforeTestsStarted() throws Exception { + System.setProperty(IGNITE_BASELINE_AUTO_ADJUST_ENABLED, "false"); + + super.beforeTestsStarted(); + } + + /** {@inheritDoc} */ + @Override protected void afterTestsStopped() throws Exception { + super.afterTestsStopped(); + + System.clearProperty(IGNITE_BASELINE_AUTO_ADJUST_ENABLED); + } + + /** {@inheritDoc} */ @Override protected void afterTest() throws Exception { stopAllGrids(); diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/service/ServiceDeploymentOutsideBaselineTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/service/ServiceDeploymentOutsideBaselineTest.java index b007c34..b693c57 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/service/ServiceDeploymentOutsideBaselineTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/service/ServiceDeploymentOutsideBaselineTest.java @@ -21,6 +21,7 @@ import java.util.concurrent.CountDownLatch; import java.util.concurrent.TimeUnit; import org.apache.ignite.Ignite; import org.apache.ignite.IgniteCluster; +import org.apache.ignite.IgniteSystemProperties; import org.apache.ignite.configuration.DataRegionConfiguration; import org.apache.ignite.configuration.DataStorageConfiguration; import org.apache.ignite.configuration.IgniteConfiguration; @@ -33,9 +34,12 @@ import org.apache.ignite.internal.util.typedef.internal.U; import org.apache.ignite.lang.IgniteFuture; import org.apache.ignite.services.ServiceConfiguration; import org.apache.ignite.testframework.GridTestUtils; +import org.apache.ignite.testframework.config.GridTestProperties; import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; import org.junit.Test; +import static org.apache.ignite.IgniteSystemProperties.IGNITE_BASELINE_AUTO_ADJUST_ENABLED; + /** */ public class ServiceDeploymentOutsideBaselineTest extends GridCommonAbstractTest { /** */ @@ -69,6 +73,20 @@ public class ServiceDeploymentOutsideBaselineTest extends GridCommonAbstractTest } /** {@inheritDoc} */ + @Override protected void beforeTestsStarted() throws Exception { + System.setProperty(IGNITE_BASELINE_AUTO_ADJUST_ENABLED, "false"); + + super.beforeTestsStarted(); + } + + /** {@inheritDoc} */ + @Override protected void afterTestsStopped() throws Exception { + super.afterTestsStopped(); + + System.clearProperty(IGNITE_BASELINE_AUTO_ADJUST_ENABLED); + } + + /** {@inheritDoc} */ @Override protected void beforeTest() throws Exception { persistence = false; srvcCfg = null; diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteKernalSelfTestSuite.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteKernalSelfTestSuite.java index 5c092a3..2bc7128 100644 --- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteKernalSelfTestSuite.java +++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteKernalSelfTestSuite.java @@ -47,8 +47,10 @@ import org.apache.ignite.internal.managers.discovery.GridDiscoveryManagerAliveCa import org.apache.ignite.internal.managers.discovery.GridDiscoveryManagerAttributesSelfTest; import org.apache.ignite.internal.managers.discovery.IgniteTopologyPrintFormatSelfTest; import org.apache.ignite.internal.managers.events.GridEventStorageManagerSelfTest; +import org.apache.ignite.internal.processors.cluster.ChangeTopologyWatcherTest; import org.apache.ignite.internal.processors.cluster.GridAddressResolverSelfTest; import org.apache.ignite.internal.processors.cluster.GridUpdateNotifierSelfTest; +import org.apache.ignite.internal.processors.cluster.baseline.autoadjust.ChangeTopologyWatcher; import org.apache.ignite.internal.processors.port.GridPortProcessorSelfTest; import org.apache.ignite.internal.util.GridStartupWithUndefinedIgniteHomeSelfTest; import org.apache.ignite.spi.communication.GridCacheMessageSelfTest; @@ -89,6 +91,7 @@ import org.junit.runners.Suite; IgniteConcurrentEntryProcessorAccessStopTest.class, GridUpdateNotifierSelfTest.class, GridAddressResolverSelfTest.class, + ChangeTopologyWatcherTest.class, IgniteUpdateNotifierPerClusterSettingSelfTest.class, GridLocalEventListenerSelfTest.class, IgniteTopologyPrintFormatSelfTest.class, diff --git a/modules/core/src/test/java/org/apache/ignite/util/GridCommandHandlerTest.java b/modules/core/src/test/java/org/apache/ignite/util/GridCommandHandlerTest.java index 304bdae..b4a4b46 100644 --- a/modules/core/src/test/java/org/apache/ignite/util/GridCommandHandlerTest.java +++ b/modules/core/src/test/java/org/apache/ignite/util/GridCommandHandlerTest.java @@ -112,6 +112,7 @@ import org.junit.Test; import static java.nio.file.Files.delete; import static java.nio.file.Files.newDirectoryStream; +import static org.apache.ignite.IgniteSystemProperties.IGNITE_BASELINE_AUTO_ADJUST_ENABLED; import static org.apache.ignite.IgniteSystemProperties.IGNITE_ENABLE_EXPERIMENTAL_COMMAND; import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL; import static org.apache.ignite.cache.CacheWriteSynchronizationMode.FULL_SYNC; @@ -157,6 +158,15 @@ public class GridCommandHandlerTest extends GridCommonAbstractTest { super.afterTestsStopped(); GridTestUtils.cleanIdleVerifyLogFiles(); + + System.clearProperty(IGNITE_BASELINE_AUTO_ADJUST_ENABLED); + } + + /** {@inheritDoc} */ + @Override protected void beforeTestsStarted() throws Exception { + System.setProperty(IGNITE_BASELINE_AUTO_ADJUST_ENABLED, "false"); + + super.beforeTestsStarted(); } /** {@inheritDoc} */ diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/database/baseline/IgniteChangingBaselineCacheQueryNodeRestartSelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/database/baseline/IgniteChangingBaselineCacheQueryNodeRestartSelfTest.java index b99745f..5345adb 100644 --- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/database/baseline/IgniteChangingBaselineCacheQueryNodeRestartSelfTest.java +++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/database/baseline/IgniteChangingBaselineCacheQueryNodeRestartSelfTest.java @@ -30,6 +30,8 @@ import org.apache.ignite.configuration.IgniteConfiguration; import org.apache.ignite.internal.IgniteInternalFuture; import org.apache.ignite.internal.processors.cache.distributed.near.IgniteCacheQueryNodeRestartSelfTest; +import static org.apache.ignite.IgniteSystemProperties.IGNITE_BASELINE_AUTO_ADJUST_ENABLED; + /** * */ @@ -52,6 +54,8 @@ public class IgniteChangingBaselineCacheQueryNodeRestartSelfTest extends IgniteC /** {@inheritDoc} */ @Override protected void beforeTestsStarted() throws Exception { + System.setProperty(IGNITE_BASELINE_AUTO_ADJUST_ENABLED, "false"); + cleanPersistenceDir(); startGrids(gridCount()); @@ -68,6 +72,8 @@ public class IgniteChangingBaselineCacheQueryNodeRestartSelfTest extends IgniteC stopAllGrids(); cleanPersistenceDir(); + + System.clearProperty(IGNITE_BASELINE_AUTO_ADJUST_ENABLED); } /** {@inheritDoc} */ diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/PersistenceTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/PersistenceTest.cs index d602802..e4cd038 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/PersistenceTest.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/PersistenceTest.cs @@ -224,6 +224,8 @@ namespace Apache.Ignite.Core.Tests.Cache [Test] public void TestBaselineTopology() { + Environment.SetEnvironmentVariable("IGNITE_BASELINE_AUTO_ADJUST_ENABLED", "false"); + var cfg1 = new IgniteConfiguration(GetPersistentConfiguration()) { ConsistentId = "node1" @@ -280,6 +282,8 @@ namespace Apache.Ignite.Core.Tests.Cache var res = cluster.GetBaselineTopology(); CollectionAssert.AreEquivalent(new[] { "node1", "node2" }, res.Select(x => x.ConsistentId)); } + + Environment.SetEnvironmentVariable("IGNITE_BASELINE_AUTO_ADJUST_ENABLED", null); } /// <summary> diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/IgniteConfigurationTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/IgniteConfigurationTest.cs index ae33a42..4cbf7ce 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/IgniteConfigurationTest.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/IgniteConfigurationTest.cs @@ -263,10 +263,6 @@ namespace Apache.Ignite.Core.Tests Assert.AreEqual(cfg.MvccVacuumThreadCount, resCfg.MvccVacuumThreadCount); Assert.AreEqual(cfg.SqlQueryHistorySize, resCfg.SqlQueryHistorySize); - Assert.AreEqual(cfg.InitBaselineAutoAdjustEnabled, resCfg.InitBaselineAutoAdjustEnabled); - Assert.AreEqual(cfg.InitBaselineAutoAdjustTimeout, resCfg.InitBaselineAutoAdjustTimeout); - Assert.AreEqual(cfg.InitBaselineAutoAdjustMaxTimeout, resCfg.InitBaselineAutoAdjustMaxTimeout); - Assert.IsNotNull(resCfg.SqlSchemas); Assert.AreEqual(2, resCfg.SqlSchemas.Count); Assert.IsTrue(resCfg.SqlSchemas.Contains("SCHEMA_3")); diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/IgniteConfiguration.cs b/modules/platforms/dotnet/Apache.Ignite.Core/IgniteConfiguration.cs index 425a232..34db906 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core/IgniteConfiguration.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Core/IgniteConfiguration.cs @@ -216,15 +216,6 @@ namespace Apache.Ignite.Core /** MVCC vacuum thread count. */ private int? _mvccVacuumThreadCnt; - /** */ - private bool? _initBaselineAutoAdjustEnabled; - - /** Initial value of time which we would wait before the actual topology change since last discovery event. */ - private long? _initBaselineAutoAdjustTimeout; - - /** Initial value of time which we would wait from the first discovery event in the chain(node join/exit). */ - private long? _initBaselineAutoAdjustMaxTimeout; - /** SQL query history size. */ private int? _sqlQueryHistorySize; @@ -264,21 +255,6 @@ namespace Apache.Ignite.Core public const int DefaultMvccVacuumThreadCount = 2; /// <summary> - /// Default value for <see cref="InitBaselineAutoAdjustEnabled"/> property. - /// </summary> - public const bool DefaultInitBaselineAutoAdjustEnabled = false; - - /// <summary> - /// Default value for <see cref="InitBaselineAutoAdjustTimeout"/> property. - /// </summary> - public const long DefaultInitBaselineAutoAdjustTimeout = 0; - - /// <summary> - /// Default value for <see cref="InitBaselineAutoAdjustMaxTimeout"/> property. - /// </summary> - public const long DefaultInitBaselineAutoAdjustMaxTimeout = 0; - - /// <summary> /// Default value for <see cref="SqlQueryHistorySize"/> property. /// </summary> public const int DefaultSqlQueryHistorySize = 1000; @@ -365,9 +341,6 @@ namespace Apache.Ignite.Core writer.WriteLongNullable(_mvccVacuumFreq); writer.WriteIntNullable(_mvccVacuumThreadCnt); writer.WriteTimeSpanAsLongNullable(_sysWorkerBlockedTimeout); - writer.WriteBooleanNullable(_initBaselineAutoAdjustEnabled); - writer.WriteLongNullable(_initBaselineAutoAdjustTimeout); - writer.WriteLongNullable(_initBaselineAutoAdjustMaxTimeout); writer.WriteIntNullable(_sqlQueryHistorySize); if (SqlSchemas == null) @@ -759,9 +732,6 @@ namespace Apache.Ignite.Core _mvccVacuumFreq = r.ReadLongNullable(); _mvccVacuumThreadCnt = r.ReadIntNullable(); _sysWorkerBlockedTimeout = r.ReadTimeSpanNullable(); - _initBaselineAutoAdjustEnabled = r.ReadBooleanNullable(); - _initBaselineAutoAdjustTimeout = r.ReadLongNullable(); - _initBaselineAutoAdjustMaxTimeout = r.ReadLongNullable(); _sqlQueryHistorySize = r.ReadIntNullable(); int sqlSchemasCnt = r.ReadInt(); @@ -1708,35 +1678,5 @@ namespace Apache.Ignite.Core /// </summary> [SuppressMessage("Microsoft.Usage", "CA2227:CollectionPropertiesShouldBeReadOnly")] public ICollection<string> SqlSchemas { get; set; } - - /// <summary> - /// Initial value of manual baseline control or auto adjusting baseline. - /// </summary> - [DefaultValue(DefaultInitBaselineAutoAdjustEnabled)] - public bool InitBaselineAutoAdjustEnabled - { - get { return _initBaselineAutoAdjustEnabled ?? DefaultInitBaselineAutoAdjustEnabled; } - set { _initBaselineAutoAdjustEnabled = value; } - } - - /// <summary> - /// Initial value of time which we would wait before the actual topology change since last discovery event. - /// </summary> - [DefaultValue(DefaultInitBaselineAutoAdjustTimeout)] - public long InitBaselineAutoAdjustTimeout - { - get { return _initBaselineAutoAdjustTimeout ?? DefaultInitBaselineAutoAdjustTimeout; } - set { _initBaselineAutoAdjustTimeout = value; } - } - - /// <summary> - /// Initial value of time which we would wait from the first discovery event in the chain(node join/exit). - /// </summary> - [DefaultValue(DefaultInitBaselineAutoAdjustMaxTimeout)] - public long InitBaselineAutoAdjustMaxTimeout - { - get { return _initBaselineAutoAdjustMaxTimeout ?? DefaultInitBaselineAutoAdjustMaxTimeout; } - set { _initBaselineAutoAdjustMaxTimeout = value; } - } } } diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/IgniteConfigurationSection.xsd b/modules/platforms/dotnet/Apache.Ignite.Core/IgniteConfigurationSection.xsd index d1d7037..59c6b9e 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core/IgniteConfigurationSection.xsd +++ b/modules/platforms/dotnet/Apache.Ignite.Core/IgniteConfigurationSection.xsd @@ -2347,21 +2347,6 @@ <xs:documentation>Whether Java console output should be redirected to Console.Out and Console.Error.</xs:documentation> </xs:annotation> </xs:attribute> - <xs:attribute name="initBaselineAutoAdjustEnabled" type="xs:boolean"> - <xs:annotation> - <xs:documentation>Initial value of manual baseline control or auto adjusting baseline.</xs:documentation> - </xs:annotation> - </xs:attribute> - <xs:attribute name="initBaselineAutoAdjustTimeout" type="xs:long"> - <xs:annotation> - <xs:documentation>Initial value of time which we would wait before the actual topology change since last discovery event.</xs:documentation> - </xs:annotation> - </xs:attribute> - <xs:attribute name="initBaselineAutoAdjustMaxTimeout" type="xs:long"> - <xs:annotation> - <xs:documentation>Initial value of time which we would wait from the first discovery event in the chain(node join/exit).</xs:documentation> - </xs:annotation> - </xs:attribute> </xs:complexType> </xs:element> </xs:schema>