diff --git a/modules/core/src/main/java/org/apache/ignite/events/DiscoveryEvent.java b/modules/core/src/main/java/org/apache/ignite/events/DiscoveryEvent.java index 0f4b004e4f58d..f05873f0a9cd8 100644 --- a/modules/core/src/main/java/org/apache/ignite/events/DiscoveryEvent.java +++ b/modules/core/src/main/java/org/apache/ignite/events/DiscoveryEvent.java @@ -62,6 +62,7 @@ * @see EventType#EVT_NODE_JOINED * @see EventType#EVT_NODE_LEFT * @see EventType#EVT_NODE_SEGMENTED + * @see EventType#EVT_NODE_VALIDATION_FAILED * @see EventType#EVTS_DISCOVERY_ALL * @see EventType#EVTS_DISCOVERY */ diff --git a/modules/core/src/main/java/org/apache/ignite/events/EventType.java b/modules/core/src/main/java/org/apache/ignite/events/EventType.java index 669faab998bd8..0e4ed8ba252a8 100644 --- a/modules/core/src/main/java/org/apache/ignite/events/EventType.java +++ b/modules/core/src/main/java/org/apache/ignite/events/EventType.java @@ -1102,6 +1102,7 @@ public interface EventType { EVT_NODE_JOINED, EVT_NODE_LEFT, EVT_NODE_FAILED, + EVT_NODE_VALIDATION_FAILED, EVT_NODE_SEGMENTED, EVT_CLIENT_NODE_DISCONNECTED, EVT_CLIENT_NODE_RECONNECTED @@ -1119,6 +1120,7 @@ public interface EventType { EVT_NODE_LEFT, EVT_NODE_FAILED, EVT_NODE_SEGMENTED, + EVT_NODE_VALIDATION_FAILED, EVT_NODE_METRICS_UPDATED, EVT_CLIENT_NODE_DISCONNECTED, EVT_CLIENT_NODE_RECONNECTED diff --git a/modules/core/src/main/java/org/apache/ignite/events/NodeValidationFailedEvent.java b/modules/core/src/main/java/org/apache/ignite/events/NodeValidationFailedEvent.java index 74cf3a9f5a19f..312cfb4179e1a 100644 --- a/modules/core/src/main/java/org/apache/ignite/events/NodeValidationFailedEvent.java +++ b/modules/core/src/main/java/org/apache/ignite/events/NodeValidationFailedEvent.java @@ -31,13 +31,10 @@ * @see EventType#EVT_NODE_VALIDATION_FAILED * @see GridComponent#validateNode */ -public class NodeValidationFailedEvent extends EventAdapter { +public class NodeValidationFailedEvent extends DiscoveryEvent { /** */ private static final long serialVersionUID = 0L; - /** The node that attempted to join cluster. */ - private final ClusterNode evtNode; - /** Validation result. */ private final IgniteNodeValidationResult res; @@ -49,17 +46,11 @@ public class NodeValidationFailedEvent extends EventAdapter { * @param res Joining node validation result. */ public NodeValidationFailedEvent(ClusterNode node, ClusterNode evtNode, IgniteNodeValidationResult res) { - super(node, res.message(), EVT_NODE_VALIDATION_FAILED); + super(node, res.message(), EVT_NODE_VALIDATION_FAILED, evtNode); - this.evtNode = evtNode; this.res = res; } - /** @return Node that couldn't join the topology due to a validation failure. */ - public ClusterNode eventNode() { - return evtNode; - } - /** @return Joining node validation result. */ public IgniteNodeValidationResult validationResult() { return res; diff --git a/modules/core/src/main/java/org/apache/ignite/internal/CoreMessagesProvider.java b/modules/core/src/main/java/org/apache/ignite/internal/CoreMessagesProvider.java index 9da592635d229..61b173aaff0ed 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/CoreMessagesProvider.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/CoreMessagesProvider.java @@ -653,7 +653,7 @@ public CoreMessagesProvider(Marshaller dfltMarsh, Marshaller schemaAwareMarsh, C withNoSchema(GroupKeyEncrypted.class); withNoSchema(NodeEncryptionKeys.class); - // [13000 - 13300]: Control, configuration, diagnostincs and other messages. + // [13000 - 13300]: Control, configuration, diagnostics and other messages. msgIdx = 13000; withSchema(GridEventStorageMessage.class); withNoSchema(ChangeGlobalStateMessage.class); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/GridComponent.java b/modules/core/src/main/java/org/apache/ignite/internal/GridComponent.java index 87d769bf3d7de..95913aec95f1c 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/GridComponent.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/GridComponent.java @@ -82,7 +82,10 @@ enum DiscoveryDataExchangeType { PERFORMANCE_STAT_PROC, /** Event storage manager. */ - EVENT_MGR; + EVENT_MGR, + + /** Rolling upgrade processor. */ + ROLLING_UPGRADE_PROC; /** Cached values array. */ public static final DiscoveryDataExchangeType[] VALUES = values(); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManager.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManager.java index 4bfaa20f304a2..cbc0e147da17b 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManager.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManager.java @@ -782,7 +782,9 @@ else if (customMsg instanceof ChangeGlobalStateMessage) { discoEvtHnd.discoCache = discoCache; if (!ctx.clientDisconnected()) { - // The security processor must be notified first, since {@link IgniteSecurity#onLocalJoin} + ctx.rollingUpgrade().features().onLocalJoin(); + + // The security processor must be notified second, since {@link IgniteSecurity#onLocalJoin} // finishes local node security context initialization that can be demanded by other Ignite // components. ctx.security().onLocalJoin(); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/metastorage/persistence/DistributedMetaStorageHistoryItemSerializer.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/metastorage/persistence/DistributedMetaStorageHistoryItemSerializer.java index 1383f056167d5..8ac3bbb440370 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/metastorage/persistence/DistributedMetaStorageHistoryItemSerializer.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/metastorage/persistence/DistributedMetaStorageHistoryItemSerializer.java @@ -26,7 +26,7 @@ /** * This class is implements {@link IgniteDataTransferObjectSerializer}. * Most implementation of the interface autogenerated. - * Please, be aware - serializer works while Rolling Upgrade in progress. + * Please, be aware - serializer works while rolling upgrade being n progress. * All changes must be made with the respece of RU rules. * * Note, this class written in PDS! diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/rollingupgrade/AbstractProcess.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/rollingupgrade/AbstractProcess.java new file mode 100644 index 0000000000000..7b9d1ddaef858 --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/rollingupgrade/AbstractProcess.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.rollingupgrade; + +import java.util.UUID; +import org.apache.ignite.IgniteCheckedException; +import org.apache.ignite.internal.IgniteInternalFuture; +import org.apache.ignite.internal.util.future.GridFutureAdapter; +import org.jetbrains.annotations.Nullable; + +/** */ +abstract class AbstractProcess { + /** */ + @Nullable private UUID activeOpId; + + /** */ + @Nullable private GridFutureAdapter activeOpFut; + + /** */ + protected abstract UUID startInternal() throws IgniteCheckedException; + + /** */ + public synchronized IgniteInternalFuture start() throws IgniteCheckedException { + if (activeOpFut != null) + return activeOpFut; + + activeOpFut = new GridFutureAdapter<>(); + activeOpId = startInternal(); + + return activeOpFut; + } + + /** */ + protected synchronized void finishProcess(UUID reqId, @Nullable Throwable err) { + if (!isInitiator(reqId)) + return; + + activeOpFut.onDone(err); + + activeOpId = null; + activeOpFut = null; + } + + /** */ + private boolean isInitiator(UUID reqId) { + return activeOpFut != null && reqId.equals(activeOpId); + } +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/rollingupgrade/RollingUpgradeProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/rollingupgrade/RollingUpgradeProcessor.java index b14e6a8042347..7c210215d3d00 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/rollingupgrade/RollingUpgradeProcessor.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/rollingupgrade/RollingUpgradeProcessor.java @@ -17,334 +17,475 @@ package org.apache.ignite.internal.processors.rollingupgrade; -import java.util.Objects; +import java.io.Serializable; +import java.util.HashSet; +import java.util.Map; +import java.util.Set; +import java.util.SortedSet; +import java.util.TreeSet; import java.util.UUID; -import java.util.concurrent.CountDownLatch; +import java.util.function.Supplier; import org.apache.ignite.IgniteCheckedException; -import org.apache.ignite.IgniteException; import org.apache.ignite.cluster.ClusterNode; import org.apache.ignite.events.DiscoveryEvent; -import org.apache.ignite.events.Event; import org.apache.ignite.internal.GridKernalContext; -import org.apache.ignite.internal.managers.eventstorage.GridLocalEventListener; +import org.apache.ignite.internal.IgniteInternalFuture; +import org.apache.ignite.internal.IgniteVersionUtils; import org.apache.ignite.internal.processors.GridProcessorAdapter; -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.nodevalidation.DiscoveryNodeValidationProcessor; -import org.apache.ignite.internal.util.lang.IgnitePair; +import org.apache.ignite.internal.processors.rollingupgrade.feature.IgniteFeatureManager; +import org.apache.ignite.internal.processors.rollingupgrade.feature.IgniteFeatureSet; +import org.apache.ignite.internal.processors.rollingupgrade.feature.IgniteFeatureUtils; +import org.apache.ignite.internal.processors.rollingupgrade.feature.IgniteVersionFeatures; +import org.apache.ignite.internal.util.distributed.DistributedProcess; +import org.apache.ignite.internal.util.distributed.InitMessage; +import org.apache.ignite.internal.util.future.GridFinishedFuture; import org.apache.ignite.internal.util.typedef.F; -import org.apache.ignite.internal.util.typedef.internal.LT; import org.apache.ignite.internal.util.typedef.internal.U; import org.apache.ignite.lang.IgniteProductVersion; -import org.apache.ignite.plugin.security.SecurityPermission; +import org.apache.ignite.plugin.extensions.communication.Message; import org.apache.ignite.spi.IgniteNodeValidationResult; -import org.apache.ignite.spi.discovery.DiscoverySpi; -import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi; -import org.apache.ignite.spi.discovery.tcp.internal.TcpDiscoveryNodesRing; +import org.apache.ignite.spi.discovery.DiscoveryDataBag; import org.jetbrains.annotations.Nullable; 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.IgniteNodeAttributes.ATTR_BUILD_VER; -import static org.apache.ignite.internal.processors.metastorage.DistributedMetaStorage.IGNITE_INTERNAL_KEY_PREFIX; - -/** Rolling upgrade processor. Manages current and target versions of cluster. */ +import static org.apache.ignite.events.EventType.EVT_NODE_VALIDATION_FAILED; +import static org.apache.ignite.internal.GridComponent.DiscoveryDataExchangeType.ROLLING_UPGRADE_PROC; +import static org.apache.ignite.internal.IgniteNodeAttributes.ATTR_IGNITE_FEATURES; +import static org.apache.ignite.internal.processors.rollingupgrade.feature.IgniteFeatureUtils.extractFeatures; +import static org.apache.ignite.internal.util.distributed.DistributedProcess.DistributedProcessType.RU_COMPLETE_VERSION_FINALIZATION; +import static org.apache.ignite.internal.util.distributed.DistributedProcess.DistributedProcessType.RU_ENABLE; +import static org.apache.ignite.internal.util.distributed.DistributedProcess.DistributedProcessType.RU_PREPARE_VERSION_FINALIZATION; +import static org.apache.ignite.plugin.security.SecurityPermission.ADMIN_ROLLING_UPGRADE; + +/** */ public class RollingUpgradeProcessor extends GridProcessorAdapter implements DiscoveryNodeValidationProcessor { - /** Key for the distributed property that holds current and target versions. */ - private static final String ROLLING_UPGRADE_VERSIONS_KEY = IGNITE_INTERNAL_KEY_PREFIX + "rolling.upgrade.versions"; - - /** Metastorage with the write access. */ - @Nullable private volatile DistributedMetaStorage metastorage; + /** */ + private final IgniteFeatureManager featureMgr; - /** TCP discovery nodes ring. */ - private TcpDiscoveryNodesRing ring; + /** */ + private final ClusterVersionUpgradeEnableProcess enableProc; - /** Last joining node. */ - private ClusterNode lastJoiningNode; + /** */ + private final ClusterVersionFinalizationProcess finalizeProc; - /** Last joining node timestamp. */ - private long lastJoiningNodeTimestamp; + /** */ + private final Object topGuard = new Object(); - /** Lock for synchronization between tcp-disco-msg-worker thread and management operations. */ - private final Object lock = new Object(); + /** */ + private final Set joiningNodes = new HashSet<>(); /** */ - private final CountDownLatch startLatch = new CountDownLatch(1); + private volatile boolean isNodeFenceActive; - /** Pair with current and target versions. {@code null} when rolling upgrade is disabled. */ - @Nullable private volatile IgnitePair rollUpVers; + /** */ + private volatile boolean isVersionUpgradeEnabled; - /** - * @param ctx Context. - */ + /** */ public RollingUpgradeProcessor(GridKernalContext ctx) { + this(ctx, () -> IgniteVersionUtils.VER, IgniteFeatureUtils::readProductFeatures); + } + + /** */ + protected RollingUpgradeProcessor( + GridKernalContext ctx, + Supplier productVersionProv, + Supplier productFeaturesProv + ) { super(ctx); + + enableProc = new ClusterVersionUpgradeEnableProcess(); + finalizeProc = new ClusterVersionFinalizationProcess(); + featureMgr = new IgniteFeatureManager(productVersionProv, productFeaturesProv); } - /** {@inheritDoc} */ - @Override public void onKernalStart(boolean active) throws IgniteCheckedException { - DiscoverySpi spi = ctx.config().getDiscoverySpi(); + /** */ + public boolean isVersionUpgradeEnabled() { + return isVersionUpgradeEnabled; + } + + /** */ + public void enableVersionUpgrade() throws IgniteCheckedException { + ctx.security().authorize(ADMIN_ROLLING_UPGRADE); + + if (isVersionUpgradeEnabled) + return; + + enableProc.start().get(); + + if (log.isInfoEnabled()) + log.info("Cluster version Rolling Upgrade was enabled"); + } - if (spi instanceof TcpDiscoverySpi) - ring = ((TcpDiscoverySpi)spi).discoveryRing(); + /** */ + public void finalizeClusterVersion() throws IgniteCheckedException { + ctx.security().authorize(ADMIN_ROLLING_UPGRADE); - startLatch.countDown(); + if (!isVersionUpgradeEnabled) + return; + + finalizeProc.start().get(); + + if (log.isInfoEnabled()) + log.info("Cluster version was successfully finalized [activeLogicalVer=" + clusterLogicalVersion() + ']'); + } + + /** */ + public IgniteFeatureManager features() { + return featureMgr; } /** {@inheritDoc} */ @Override public void start() throws IgniteCheckedException { - ctx.event().addLocalEventListener(new GridLocalEventListener() { - @Override public void onEvent(Event evt) { - UUID nodeId = ((DiscoveryEvent)evt).eventNode().id(); + ctx.addNodeAttribute(ATTR_IGNITE_FEATURES, featureMgr.productFeatures().features()); - synchronized (lock) { - if (lastJoiningNode != null && lastJoiningNode.id().equals(nodeId)) - lastJoiningNode = null; + ctx.event().addLocalEventListener( + evt -> { + synchronized (topGuard) { + joiningNodes.remove(((DiscoveryEvent)evt).eventNode()); } - } - }, EVT_NODE_JOINED, EVT_NODE_FAILED, EVT_NODE_LEFT); + }, + EVT_NODE_JOINED, + EVT_NODE_FAILED, + EVT_NODE_LEFT, + EVT_NODE_VALIDATION_FAILED + ); + } + + /** {@inheritDoc} */ + @Override public DiscoveryDataExchangeType discoveryDataType() { + return ROLLING_UPGRADE_PROC; + } + + /** {@inheritDoc} */ + @Override public void collectGridNodeData(DiscoveryDataBag dataBag) { + if (ctx.clientNode()) + return; + + int cmpId = discoveryDataType().ordinal(); + + if (!dataBag.commonDataCollectedFor(cmpId)) + dataBag.addGridCommonData(cmpId, collectRollingUpgradeNodeData()); + } - ctx.internalSubscriptionProcessor().registerDistributedMetastorageListener(new DistributedMetastorageLifecycleListener() { - @Override public void onReadyForWrite(DistributedMetaStorage metastorage) { - RollingUpgradeProcessor.this.metastorage = metastorage; + /** {@inheritDoc} */ + @Override public void onGridDataReceived(DiscoveryDataBag.GridDiscoveryData data) { + RollingUpgradeNodeData gridData = (RollingUpgradeNodeData)data.commonData(); + + isVersionUpgradeEnabled = gridData.isVersionUpgradeEnabled(); + isNodeFenceActive = gridData.isNodeFenceActive(); + + featureMgr.onGridDataReceived(gridData.activeFeatures()); + } + + /** {@inheritDoc} */ + @Override public @Nullable IgniteNodeValidationResult validateNode(ClusterNode joiningNode) { + synchronized (topGuard) { + if (isNodeFenceActive) { + return new IgniteNodeValidationResult( + joiningNode.id(), + "Node joins are not allowed during cluster version finalization [joiningNode=" + joiningNode + ']'); } - @Override public void onReadyForRead(ReadableDistributedMetaStorage metastorage) { - try { - rollUpVers = metastorage.read(ROLLING_UPGRADE_VERSIONS_KEY); - } - catch (IgniteCheckedException e) { - throw new IgniteException(e); + if (isVersionUpgradeEnabled) { + RollingUpgradeState state = detectRollingUpgradeStatus(); + + if (!state.isCompatible(joiningNode)) { + return new IgniteNodeValidationResult( + joiningNode.id(), + "The joining node is incompatible with the current state of the cluster version rolling upgrade being in progress" + + " [rollingUpgradeState=" + state + + ", joiningNodeVer=" + joiningNode.version() + + ", joiningNode=" + joiningNode + ']'); } - // Keep the current and target version pair in sync with metastorage updates, e.g., to handle coordinator changes. - metastorage.listen(ROLLING_UPGRADE_VERSIONS_KEY::equals, (key, oldVal, newVal) -> { - rollUpVers = (IgnitePair)newVal; - }); + IgniteVersionFeatures locActiveFeatures = featureMgr.activeFeatures(); + + if (!locActiveFeatures.isUpgradableTo(extractFeatures(joiningNode))) { + return new IgniteNodeValidationResult( + joiningNode.id(), + "Rolling Upgrade is not available between the current cluster logical version and the joining node" + + " product version [clusterLogicalVer=" + locActiveFeatures.version() + + ", joiningNodeVer=" + joiningNode.version() + + ", joiningNode=" + joiningNode + ']'); + } + } + else if (!joiningNode.version().equals(localProductVersion())) { + return new IgniteNodeValidationResult( + joiningNode.id(), + "The joining node version differs from the version of the cluster" + + " [clusterVer=" + localProductVersion() + + ", joiningNodeVer=" + joiningNode.version() + + ", joiningNode=" + joiningNode + ']'); } - }); - } - /** {@inheritDoc} The joining node is stored to verify later whether it successfully connected to the ring or failed to join. */ - @Override public @Nullable IgniteNodeValidationResult validateNode(ClusterNode node) { - synchronized (lock) { - lastJoiningNode = node; + joiningNodes.add(joiningNode); - lastJoiningNodeTimestamp = U.currentTimeMillis(); + return null; } + } - ClusterNode locNode = ctx.discovery().localNode(); - - String locBuildVer = locNode.attribute(ATTR_BUILD_VER); - String rmtBuildVer = node.attribute(ATTR_BUILD_VER); + /** */ + private IgniteProductVersion localProductVersion() { + return featureMgr.productFeatures().version(); + } - IgniteProductVersion rmtVer = IgniteProductVersion.fromString(rmtBuildVer); + /** */ + private IgniteProductVersion clusterLogicalVersion() { + return featureMgr.activeFeatures().version(); + } - IgnitePair pair = rollUpVers; + /** */ + private RollingUpgradeState detectRollingUpgradeStatus() { + SortedSet clusterVers = distinctClusterProductVersions(); - IgniteProductVersion curVer = pair == null ? IgniteProductVersion.fromString(locBuildVer) : pair.get1(); - IgniteProductVersion targetVer = pair == null ? null : pair.get2(); + assert !clusterVers.isEmpty() && clusterVers.size() <= 2; - if (Objects.equals(rmtVer, curVer) || Objects.equals(rmtVer, targetVer)) - return null; + IgniteProductVersion minClusterVer = clusterVers.first(); + IgniteProductVersion maxClusterVer = clusterVers.last(); - String errMsg = "Remote node rejected due to incompatible version for cluster join.\n" - + "Remote node info:\n" - + " - Version : " + rmtBuildVer + "\n" - + " - Addresses : " + U.addressesAsString(node) + "\n" - + " - Node ID : " + node.id() + "\n" - + "Local node info:\n" - + " - Version : " + locBuildVer + "\n" - + " - Addresses : " + U.addressesAsString(locNode) + "\n" - + " - Node ID : " + locNode.id() + "\n" - + "Allowed versions for joining: " + curVer + (targetVer == null ? "" : ", " + targetVer); + boolean isClusterVerHeterogeneous = !maxClusterVer.equals(minClusterVer); - LT.warn(log, errMsg); + if (isClusterVerHeterogeneous) + return new RollingUpgradeState(minClusterVer, maxClusterVer, true); - if (log.isDebugEnabled()) - log.debug(errMsg); + IgniteProductVersion logicalVer = clusterLogicalVersion(); - return new IgniteNodeValidationResult(node.id(), errMsg); + return logicalVer.equals(maxClusterVer) + ? new RollingUpgradeState(logicalVer, null, false) + : new RollingUpgradeState(logicalVer, maxClusterVer, false); } - /** - * Enables rolling upgrade with specified target version. - * This method can only be called on coordinator node with {@link TcpDiscoverySpi}. - * - * @param target Target version. - * @param force If {@code true}, skips target version compatibility checks and forcibly enables rolling upgrade. - * This flag does not override an already active upgrade configuration. - * @throws IgniteCheckedException If: - *
    - *
  • The current and target versions are incompatible;
  • - *
  • The local node is not a coordinator;
  • - *
  • The discovery SPI is not {@link TcpDiscoverySpi};
  • - *
  • The distributed metastorage is not ready;
  • - *
- */ - public void enable(IgniteProductVersion target, boolean force) throws IgniteCheckedException { - ctx.security().authorize(SecurityPermission.ADMIN_ROLLING_UPGRADE); - - if (startLatch.getCount() > 0) - throw new IgniteCheckedException("Cannot enable rolling upgrade: processor has not been started yet"); - - if (!U.isLocalNodeCoordinator(ctx.discovery())) - throw new IgniteCheckedException("Rolling upgrade can be enabled only on coordinator node"); - - if (metastorage == null) - throw new IgniteCheckedException("Metastorage is not ready yet. Try again later"); - - if (!(ctx.config().getDiscoverySpi() instanceof TcpDiscoverySpi)) - throw new IgniteCheckedException("Rolling upgrade is supported only with TCP discovery SPI"); - - String curBuildVer = ctx.discovery().localNode().attribute(ATTR_BUILD_VER); - IgniteProductVersion curVer = IgniteProductVersion.fromString(curBuildVer); - - if (!checkVersionsForEnabling(curVer, target, force)) - return; + /** */ + private SortedSet distinctClusterProductVersions() { + assert Thread.holdsLock(topGuard); - IgnitePair newPair = F.pair(curVer, target); + TreeSet res = new TreeSet<>(); - if (!metastorage.compareAndSet(ROLLING_UPGRADE_VERSIONS_KEY, null, newPair)) { - IgnitePair oldVerPair = metastorage.read(ROLLING_UPGRADE_VERSIONS_KEY); + for (ClusterNode node : ctx.discovery().allNodes()) + res.add(node.version()); - if (newPair.equals(oldVerPair)) - return; + for (ClusterNode node : joiningNodes) + res.add(node.version()); - if (oldVerPair == null) - throw new IgniteCheckedException("Could not enable rolling upgrade. Try again"); + return res; + } - throw new IgniteCheckedException("Rolling upgrade is already enabled with a different current and target version: " + - oldVerPair.get1() + " , " + oldVerPair.get2()); + /** */ + private RollingUpgradeNodeData collectRollingUpgradeNodeData() { + return new RollingUpgradeNodeData(isVersionUpgradeEnabled, isNodeFenceActive, featureMgr.activeFeatures()); + } + + /** */ + private class ClusterVersionUpgradeEnableProcess extends AbstractProcess { + /** */ + private final DistributedProcess distributedProc; + + /** */ + public ClusterVersionUpgradeEnableProcess() { + distributedProc = new DistributedProcess<>( + ctx, + RU_ENABLE, + this::execute, + this::finish, + (reqId, req) -> new InitMessage<>(reqId, RU_ENABLE, req, true)); } - rollUpVers = newPair; + /** {@inheritDoc} */ + @Override protected UUID startInternal() { + UUID reqId = UUID.randomUUID(); - if (log.isInfoEnabled()) - log.info("Rolling upgrade enabled [current=" + curVer + ", target=" + target + ']'); + distributedProc.start(reqId, null); + + return reqId; + } + + /** */ + private IgniteInternalFuture execute(Message req) { + isVersionUpgradeEnabled = true; + + return new GridFinishedFuture<>(); + } + + /** */ + private void finish(UUID reqId, Map responses, Map errors) { + finishProcess(reqId, null); + } } - /** - * Disables rolling upgrade. - * This method can only be called on coordinator node. - * - *

May be blocked while a node with a different version is still joining or during metastorage operations.

- * - * @throws IgniteCheckedException If cluster has two or more nodes with different versions or if node is not coordinator - * or metastorage is not ready. - */ - public void disable() throws IgniteCheckedException { - ctx.security().authorize(SecurityPermission.ADMIN_ROLLING_UPGRADE); - - if (!U.isLocalNodeCoordinator(ctx.discovery())) - throw new IgniteCheckedException("Rolling upgrade can be disabled only on coordinator node"); - - if (metastorage == null) - throw new IgniteCheckedException("Meta storage is not ready. Try again"); - - if (rollUpVers == null) - return; + /** */ + private class ClusterVersionFinalizationProcess extends AbstractProcess { + /** */ + private final DistributedProcess preparePhase; + + /** */ + private final DistributedProcess completionPhase; + + /** */ + public ClusterVersionFinalizationProcess() { + preparePhase = new DistributedProcess<>( + ctx, + RU_PREPARE_VERSION_FINALIZATION, + this::executePreparePhase, + this::finishPreparePhase, + (reqId, req) -> new InitMessage<>(reqId, RU_PREPARE_VERSION_FINALIZATION, req, true)); + + completionPhase = new DistributedProcess<>( + ctx, + RU_COMPLETE_VERSION_FINALIZATION, + this::executeCompletionPhase, + this::finishCompletionPhase, + (reqId, req) -> new InitMessage<>(reqId, RU_COMPLETE_VERSION_FINALIZATION, req, true)); + } - IgnitePair minMaxVerPair = ring.minMaxNodeVersions(); + /** {@inheritDoc} */ + @Override protected UUID startInternal() { + UUID reqId = UUID.randomUUID(); - if (!minMaxVerPair.get1().equals(minMaxVerPair.get2())) - throw new IgniteCheckedException("Can't disable rolling upgrade with different versions in cluster: " - + minMaxVerPair.get1() + ", " + minMaxVerPair.get2()); + preparePhase.start(reqId, null); - synchronized (lock) { - if (lastJoiningNode != null) { - // Use 3 * joinTimeout as an upper time bound for joining nodes that may drop during validation - // without sending NODE_LEFT / NODE_FAILED events. - long timeout = ((TcpDiscoverySpi)ctx.config().getDiscoverySpi()).getJoinTimeout() * 3; + return reqId; + } - if (ring.node(lastJoiningNode.id()) != null || (timeout > 0 && U.currentTimeMillis() - lastJoiningNodeTimestamp > timeout)) - lastJoiningNode = null; - } + /** */ + private IgniteInternalFuture executePreparePhase(Message req) { + synchronized (topGuard) { + if (isNodeFenceActive) { + return new GridFinishedFuture<>(new IgniteCheckedException( + "Cluster version finalization procedure is already in progress")); + } - if (lastJoiningNode != null) { - IgniteProductVersion lastJoiningNodeVer = IgniteProductVersion.fromString(lastJoiningNode.attribute(ATTR_BUILD_VER)); + Set distinctNodeVersions = distinctClusterProductVersions(); - if (!minMaxVerPair.get1().equals(lastJoiningNodeVer)) - throw new IgniteCheckedException("Can't disable rolling upgrade with different versions in cluster: " - + minMaxVerPair.get1() + ", " + lastJoiningNodeVer); + if (distinctNodeVersions.size() > 1) { + return new GridFinishedFuture<>(new IgniteCheckedException( + "Cluster version finalization failed. The topology contains nodes running multiple different" + + " versions [distinctNodeVersions=" + distinctNodeVersions + "]" + )); + } + + isNodeFenceActive = true; + + return new GridFinishedFuture<>(); } + } - rollUpVers = null; + /** */ + private void finishPreparePhase(UUID reqId, Map responses, Map errors) { + if (!F.isEmpty(errors)) { + finishProcess(reqId, F.firstValue(errors)); + } + else if (U.isLocalNodeCoordinator(ctx.discovery())) + completionPhase.start(reqId, null); } - metastorage.remove(ROLLING_UPGRADE_VERSIONS_KEY); + /** */ + private IgniteInternalFuture executeCompletionPhase(Message req) { + isVersionUpgradeEnabled = false; - if (log.isInfoEnabled()) - log.info("Rolling upgrade disabled. Current version of nodes in cluster: " + minMaxVerPair.get1()); - } + featureMgr.activateProductFeatures(); - /** - * Returns a pair containing the current and target versions of the cluster. - *

- * This method returns {@code null} if rolling upgrade has not been enabled yet - * or if version information has not been read from the distributed metastorage. - * - * @return A pair where: - *

    - *
  • First element — current version of the cluster.
  • - *
  • Second element — target version to which the cluster is being upgraded.
  • - *
- * or {@code null} if rolling upgrade is not active. - */ - @Nullable public IgnitePair versions() { - return rollUpVers; - } + isNodeFenceActive = false; - /** Checks whether the cluster is in the rolling upgrade mode. */ - public boolean enabled() { - return versions() != null; + return new GridFinishedFuture<>(); + } + + /** */ + private void finishCompletionPhase(UUID reqId, Map responses, Map errors) { + finishProcess(reqId, null); + } } - /** - * Checks cur and target versions. - * - * @param cur Current cluster version. - * @param target Target cluster version. - * @param force Force flag to skip version checks. - * @throws IgniteCheckedException If versions are incorrect. - */ - private boolean checkVersionsForEnabling( - IgniteProductVersion cur, - IgniteProductVersion target, - boolean force - ) throws IgniteCheckedException { - IgnitePair oldVerPair = rollUpVers; - if (oldVerPair != null) { - if (oldVerPair.get1().equals(cur) && oldVerPair.get2().equals(target)) - return false; - - throw new IgniteCheckedException("Rolling upgrade is already enabled with a different current and target version: " + - oldVerPair.get1() + " , " + oldVerPair.get2()); + /** */ + private static class RollingUpgradeNodeData implements Serializable { + /** */ + private static final long serialVersionUID = 0L; + + /** */ + private final boolean isVersionUpgradeEnabled; + + /** */ + private final IgniteVersionFeatures activeFeatures; + + /** */ + private final boolean isNodeFenceActive; + + /** */ + private RollingUpgradeNodeData(boolean isVersionUpgradeEnabled, boolean isNodeFenceActive, IgniteVersionFeatures activeFeatures) { + this.isVersionUpgradeEnabled = isVersionUpgradeEnabled; + this.isNodeFenceActive = isNodeFenceActive; + this.activeFeatures = activeFeatures; + } + + /** */ + private boolean isVersionUpgradeEnabled() { + return isVersionUpgradeEnabled; } - if (force) { - if (log.isInfoEnabled()) - log.info("Skipping version compatibility check for rolling upgrade due to force flag " - + "[currentVer=" + cur + ", targetVer=" + target + ']'); + /** */ + private boolean isNodeFenceActive() { + return isNodeFenceActive; + } + + /** */ + private IgniteVersionFeatures activeFeatures() { + return activeFeatures; + } + } - return true; + /** */ + private static class RollingUpgradeState { + /** */ + public final IgniteProductVersion sourceVer; + + /** */ + public final @Nullable IgniteProductVersion targetVer; + + /** */ + public final boolean isClusterHeterogeneous; + + /** */ + private RollingUpgradeState( + IgniteProductVersion sourceVer, + @Nullable IgniteProductVersion targetVer, + boolean isClusterVerHeterogeneous + ) { + this.sourceVer = sourceVer; + this.targetVer = targetVer; + this.isClusterHeterogeneous = isClusterVerHeterogeneous; } - if (cur.major() != target.major()) - throw new IgniteCheckedException("Major versions are different"); + /** */ + boolean isCompatible(ClusterNode joiningNode) { + IgniteProductVersion joiningNodeVer = joiningNode.version(); + + if (targetVer == null) + return sourceVer.compareTo(joiningNodeVer) <= 0; - if (cur.minor() != target.minor()) { - if (target.minor() == cur.minor() + 1 && target.maintenance() == 0) + if (joiningNodeVer.equals(sourceVer) || joiningNodeVer.equals(targetVer)) return true; - throw new IgniteCheckedException("Minor version can only be incremented by 1"); + return !isClusterHeterogeneous && targetVer.compareTo(joiningNodeVer) < 0; } - if (cur.maintenance() + 1 != target.maintenance()) - throw new IgniteCheckedException("Patch version can only be incremented by 1"); + /** {@inheritDoc} */ + @Override public String toString() { + StringBuilder sb = new StringBuilder("[compatibleProductVersions="); - return true; + if (targetVer == null) + sb.append("[").append(sourceVer).append(" or greater]"); + else { + sb.append("{").append(sourceVer).append(", ").append(targetVer).append("}"); + + if (!isClusterHeterogeneous) + sb.append(" or [").append(targetVer).append(", ").append(" or greater]"); + } + + sb.append("]"); + + return sb.toString(); + } } } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/rollingupgrade/feature/IgniteCoreFeature.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/rollingupgrade/feature/IgniteCoreFeature.java new file mode 100644 index 0000000000000..07476bf5501e8 --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/rollingupgrade/feature/IgniteCoreFeature.java @@ -0,0 +1,65 @@ +/* + * 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.rollingupgrade.feature; + +import java.util.Objects; +import org.apache.ignite.internal.util.tostring.GridToStringInclude; +import org.apache.ignite.internal.util.typedef.internal.A; +import org.apache.ignite.internal.util.typedef.internal.S; + +/** Represents an implementation of {@link IgniteFeature} used to define incompatible changes in Ignite core functionality. */ +public class IgniteCoreFeature implements IgniteFeature { + /** */ + @GridToStringInclude + private final int id; + + /** */ + public IgniteCoreFeature(int id) { + A.ensure(id >= 0, "Feature ID must be non-negative"); + + this.id = id; + } + + /** {@inheritDoc} */ + @Override public int id() { + return id; + } + + /** {@inheritDoc} */ + @Override public boolean equals(Object o) { + if (this == o) + return true; + + if (o == null || getClass() != o.getClass()) + return false; + + IgniteCoreFeature that = (IgniteCoreFeature)o; + + return id == that.id; + } + + /** {@inheritDoc} */ + @Override public int hashCode() { + return Objects.hash(id); + } + + /** {@inheritDoc} */ + @Override public String toString() { + return S.toString(IgniteCoreFeature.class, this); + } +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/rollingupgrade/feature/IgniteFeature.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/rollingupgrade/feature/IgniteFeature.java new file mode 100644 index 0000000000000..7e8cefd2ca30b --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/rollingupgrade/feature/IgniteFeature.java @@ -0,0 +1,42 @@ +/* + * 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.rollingupgrade.feature; + +/** + * Represents a change in Ignite node behavior that is not compatible with Ignite nodes from previous versions. + * {@link IgniteFeature} is one of the core mechanisms on which the Ignite Rolling Upgrade implementation depends. + * It serves two main purposes: + *
    + *
  1. determining Rolling Upgrade availability between different Ignite versions
  2. + *
  3. providing the ability to deactivate Ignite functionality introduced in newer versions for the duration of a Rolling Upgrade
  4. + *
+ * + * @see IgniteFeatureSet + */ +public interface IgniteFeature { + /** + *

{@link IgniteFeature} is identified by a unique integer ID. IDs of {@link IgniteFeature} instances must:

+ *
    + *
  • remain unchanged between Ignite versions
  • + *
  • start at {@code 0} and increase monotonically
  • + *
+ * + * @return The unique identifier of this feature. + */ + int id(); +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/rollingupgrade/feature/IgniteFeatureManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/rollingupgrade/feature/IgniteFeatureManager.java new file mode 100644 index 0000000000000..3dd0ed77824a3 --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/rollingupgrade/feature/IgniteFeatureManager.java @@ -0,0 +1,106 @@ +/* + * 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.rollingupgrade.feature; + +import java.util.function.Supplier; +import org.apache.ignite.internal.util.future.GridFutureAdapter; +import org.apache.ignite.lang.IgniteProductVersion; +import org.apache.ignite.lang.IgniteRunnable; + +/** */ +public class IgniteFeatureManager { + /** */ + private final IgniteVersionFeatures productFeatures; + + /** */ + private final GridFutureAdapter productFeaturesActivationFut; + + /** */ + private volatile IgniteVersionFeatures activeFeatures; + + /** */ + public IgniteFeatureManager( + Supplier productVersionProv, + Supplier productFeaturesProv + ) { + productFeatures = new IgniteVersionFeatures(productVersionProv.get(), productFeaturesProv.get()); + productFeaturesActivationFut = new GridFutureAdapter<>(); + } + + /** */ + public IgniteVersionFeatures productFeatures() { + return productFeatures; + } + + /** */ + public IgniteVersionFeatures activeFeatures() { + final IgniteVersionFeatures finalActiveFeatures = activeFeatures; + + if (finalActiveFeatures == null) + throw new IllegalStateException("Node features are not yet initialized"); + + return finalActiveFeatures; + } + + /** */ + public boolean isActive(IgniteFeature feature) { + final IgniteVersionFeatures finalActiveFeatures = activeFeatures; + + if (finalActiveFeatures == null) + throw new IllegalStateException("Node features are not yet initialized"); + + return finalActiveFeatures.contains(feature); + } + + /** */ + public void listenActivation(IgniteFeature feature, IgniteRunnable lsnr) { + assert productFeatures.contains(feature); + + if (activeFeatures.contains(feature)) + lsnr.run(); + else + productFeaturesActivationFut.listen(lsnr); + } + + /** */ + public void onGridDataReceived(IgniteVersionFeatures activeClusterFeatures) { + if (productFeatures.equals(activeClusterFeatures)) + activateProductFeatures(); + else + this.activeFeatures = activeClusterFeatures; + } + + /** */ + public void onLocalJoin() { + if (activeFeatures == null) + activateProductFeatures(); + } + + /** */ + public synchronized void activateProductFeatures() { + if (productFeaturesActivationFut.isDone()) + return; + + activeFeatures = productFeatures; + + productFeaturesActivationFut.onDone(); + } +} + + + diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/rollingupgrade/feature/IgniteFeatureSet.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/rollingupgrade/feature/IgniteFeatureSet.java new file mode 100644 index 0000000000000..adef0cf952f75 --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/rollingupgrade/feature/IgniteFeatureSet.java @@ -0,0 +1,267 @@ +/* + * 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.rollingupgrade.feature; + +import java.io.Externalizable; +import java.io.IOException; +import java.io.ObjectInput; +import java.io.ObjectOutput; +import java.util.Collection; +import java.util.Iterator; +import java.util.NoSuchElementException; +import java.util.Objects; +import org.apache.ignite.internal.util.GridIntIterator; +import org.apache.ignite.internal.util.GridIntList; +import org.apache.ignite.internal.util.typedef.internal.A; +import org.jetbrains.annotations.NotNull; +import org.jetbrains.annotations.Nullable; + +/** + * Represents a set of Ignite node features. A feature set determines the behavior of an Ignite node. Therefore, + * comparing the feature sets of Ignite nodes running different versions helps identify behavioral differences between + * versions and determine Rolling Upgrade compatibility. + * + *

The Ignite Feature Set divides all features into two categories: features that can be deactivated to simulate + * the behavior of nodes from previous versions, and features that cannot be deactivated.

+ * + *

Note that the Ignite Feature Set operates on {@link IgniteFeature} identifiers and relies on the following + * properties of {@link IgniteFeature} identifiers:

+ * + *
    + *
  • The identifiers of existing features do not change between Ignite versions
  • + *
  • Identifier values start at {@code 0} and increase monotonically
  • + *
+ * + *

The feature set divides features into three ranges:

+ * + *
    + *
  1. + * A continuous range of feature IDs that cannot be deactivated: + * {@code [0 -> }{@link IgniteFeatureSet#rangeStartInclusive}{@code )} + *
  2. + *
  3. + * A continuous range of feature IDs that can be deactivated: + * {@code [}{@link IgniteFeatureSet#rangeStartInclusive}{@code -> } + * {@link IgniteFeatureSet#rangeEndInclusive}{@code ]} + *
  4. + *
  5. + * A sparse suffix containing IDs of features that can be deactivated. The identifier values in this range + * are greater than {@link IgniteFeatureSet#rangeEndInclusive} + *
  6. + *
+ * + * @see IgniteFeature + */ +public class IgniteFeatureSet implements Iterable, Externalizable { + /** */ + private static final long serialVersionUID = 0L; + + /** */ + private int rangeStartInclusive; + + /** */ + private int rangeEndInclusive; + + /** */ + @Nullable private GridIntList sparseSuffix; + + /** */ + public IgniteFeatureSet() { + // No-op. + } + + /** */ + private IgniteFeatureSet(int rangeStartInclusive, int rangeEndInclusive, @Nullable GridIntList sparseSuffix) { + A.ensure(rangeStartInclusive >= 0, "rangeStart must be greater than or equal to 0"); + A.ensure(rangeEndInclusive >= rangeStartInclusive, "rangeEnd must be greater than or equal to rangeStart"); + + this.rangeStartInclusive = rangeStartInclusive; + this.rangeEndInclusive = rangeEndInclusive; + this.sparseSuffix = sparseSuffix; + } + + /** */ + public boolean isUpgradableTo(IgniteFeatureSet target) { + GridIntList diff = difference(target); + + for (GridIntIterator iter = diff.iterator(); iter.hasNext(); ) { + int featureId = iter.next(); + + if (!target.contains(featureId) || !target.canDeactivate(featureId)) + return false; + } + + return true; + } + + /** */ + public GridIntList difference(IgniteFeatureSet other) { + GridIntList res = new GridIntList(); + + for (int featureId : other) { + if (!contains(featureId)) + res.add(featureId); + } + + for (int featureId : this) { + if (!other.contains(featureId)) + res.add(featureId); + } + + return res; + } + + /** */ + public boolean contains(int featureId) { + if (featureId <= rangeEndInclusive) + return true; + + return sparseSuffix != null && sparseSuffix.contains(featureId); + } + + /** + * Creates an {@link IgniteFeatureSet} from the specified collection of {@link IgniteFeature}s that can be deactivated + * on an Ignite node. + * + *

{@link IgniteFeature} instances that are not present in the specified collection and whose IDs are lower than + * the minimum ID among the specified {@link IgniteFeature}s are considered non-deactivatable.

+ */ + public static IgniteFeatureSet buildFrom(Collection nodeFeatures) { + A.notEmpty(nodeFeatures, "node features"); + + GridIntList featureIds = new GridIntList(nodeFeatures.stream().mapToInt(IgniteFeature::id).toArray()); + + featureIds.sort(); + + int rangeStart = featureIds.get(0); + int rangeEnd = rangeStart; + + int idx = 1; + + for (; idx < featureIds.size(); idx++) { + int nextFeatureId = featureIds.get(idx); + + assert nextFeatureId != rangeEnd : "Duplication of Ignite Feature ID"; + + if (nextFeatureId == rangeEnd + 1) + rangeEnd = nextFeatureId; + else + break; + } + + GridIntList sparseSuffix = idx < featureIds.size() ? featureIds.copyOfRange(idx, featureIds.size()) : null; + + return new IgniteFeatureSet(rangeStart, rangeEnd, sparseSuffix); + } + + /** */ + private int calculateSize() { + int size = rangeEndInclusive + 1; + + if (sparseSuffix != null) + size += sparseSuffix.size(); + + return size; + } + + /** */ + private boolean canDeactivate(int featureId) { + return rangeStartInclusive <= featureId; + } + + /** {@inheritDoc} */ + @Override public boolean equals(Object o) { + if (this == o) + return true; + + if (o == null || getClass() != o.getClass()) + return false; + + IgniteFeatureSet that = (IgniteFeatureSet)o; + + return rangeStartInclusive == that.rangeStartInclusive + && rangeEndInclusive == that.rangeEndInclusive + && Objects.equals(sparseSuffix, that.sparseSuffix); + } + + /** {@inheritDoc} */ + @Override public @NotNull Iterator iterator() { + return new Iterator<>() { + /** */ + private int idx = 0; + + /** */ + private final int size = calculateSize(); + + /** {@inheritDoc} */ + @Override public boolean hasNext() { + return idx < size; + } + + /** {@inheritDoc} */ + @Override public Integer next() { + if (!hasNext()) + throw new NoSuchElementException(); + + Integer res = idx <= rangeEndInclusive ? idx : sparseSuffix.get(idx - rangeEndInclusive - 1); + + ++idx; + + return res; + } + }; + } + + /** {@inheritDoc} */ + @Override public void writeExternal(ObjectOutput out) throws IOException { + out.writeInt(rangeStartInclusive); + out.writeInt(rangeEndInclusive); + out.writeObject(sparseSuffix); + } + + /** {@inheritDoc} */ + @Override public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException { + rangeStartInclusive = in.readInt(); + rangeEndInclusive = in.readInt(); + sparseSuffix = (GridIntList)in.readObject(); + } + + /** {@inheritDoc} */ + @Override public int hashCode() { + return Objects.hash(rangeStartInclusive, rangeEndInclusive, sparseSuffix); + } + + /** {@inheritDoc} */ + @Override public String toString() { + StringBuilder sb = new StringBuilder(); + + sb.append("IgniteFeatureSet [").append(rangeStartInclusive); + + if (rangeStartInclusive != rangeEndInclusive) + sb.append(rangeEndInclusive - rangeStartInclusive > 1 ? " -> " : ", ").append(rangeEndInclusive); + + if (sparseSuffix != null) { + for (GridIntIterator iter = sparseSuffix.iterator(); iter.hasNext(); ) + sb.append(", ").append(iter.next()); + } + + sb.append(']'); + + return sb.toString(); + } +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/rollingupgrade/feature/IgniteFeatureUtils.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/rollingupgrade/feature/IgniteFeatureUtils.java new file mode 100644 index 0000000000000..1da89dae2601c --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/rollingupgrade/feature/IgniteFeatureUtils.java @@ -0,0 +1,66 @@ +/* + * 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.rollingupgrade.feature; + +import java.lang.reflect.Field; +import java.lang.reflect.Modifier; +import java.util.ArrayList; +import java.util.Collection; +import java.util.List; +import org.apache.ignite.IgniteException; +import org.apache.ignite.cluster.ClusterNode; + +import static org.apache.ignite.internal.IgniteNodeAttributes.ATTR_IGNITE_FEATURES; + +/** */ +public class IgniteFeatureUtils { + /** */ + public static IgniteFeatureSet readProductFeatures() { + return IgniteFeatureSet.buildFrom(readDeclaredFeatures(IgniteProductFeatures.class)); + } + + /** */ + public static IgniteFeatureSet extractFeatures(ClusterNode node) { + return node.attribute(ATTR_IGNITE_FEATURES); + } + + /** */ + public static Collection readDeclaredFeatures(Class cls) { + List features = new ArrayList<>(); + + for (Field field : cls.getFields()) { + if (Modifier.isStatic(field.getModifiers()) && field.getType().equals(IgniteFeature.class)) { + if (field.getName().endsWith("_FEATURE")) { + try { + IgniteFeature feature = (IgniteFeature)field.get(null); + + features.add(feature); + } + catch (IllegalAccessException e) { + throw new IgniteException("Failed to parse Ignite Product Features", e); + } + } + } + } + + if (features.isEmpty()) + throw new IgniteException("No Ignite feature definitions were found in the specified class [cls=" + cls.getName() + ']'); + + return features; + } +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/rollingupgrade/feature/IgniteProductFeatures.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/rollingupgrade/feature/IgniteProductFeatures.java new file mode 100644 index 0000000000000..6592c35b239fd --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/rollingupgrade/feature/IgniteProductFeatures.java @@ -0,0 +1,24 @@ +/* + * 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.rollingupgrade.feature; + +/** */ +public class IgniteProductFeatures { + /** */ + public static final IgniteFeature ROLLING_UPGRADE_FEATURE = new IgniteCoreFeature(0); +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/rollingupgrade/feature/IgniteVersionFeatures.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/rollingupgrade/feature/IgniteVersionFeatures.java new file mode 100644 index 0000000000000..bf92928b5215a --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/rollingupgrade/feature/IgniteVersionFeatures.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.rollingupgrade.feature; + +import java.io.Serializable; +import java.util.Objects; +import org.apache.ignite.lang.IgniteProductVersion; + +/** */ +public class IgniteVersionFeatures implements Serializable { + /** */ + private static final long serialVersionUID = 0L; + + /** */ + private final IgniteProductVersion ver; + + /** */ + private final IgniteFeatureSet features; + + /** */ + public IgniteVersionFeatures(IgniteProductVersion ver, IgniteFeatureSet features) { + this.ver = ver; + this.features = features; + } + + /** */ + public IgniteProductVersion version() { + return ver; + } + + /** */ + public IgniteFeatureSet features() { + return features; + } + + /** */ + public boolean contains(IgniteFeature feature) { + return features.contains(feature.id()); + } + + /** */ + public boolean isUpgradableTo(IgniteFeatureSet target) { + return features.isUpgradableTo(target); + } + + /** {@inheritDoc} */ + @Override public boolean equals(Object o) { + if (o == null || getClass() != o.getClass()) + return false; + + IgniteVersionFeatures other = (IgniteVersionFeatures)o; + + return Objects.equals(features, other.features) && Objects.equals(ver, other.ver); + } + + /** {@inheritDoc} */ + @Override public int hashCode() { + return Objects.hash(features, ver); + } +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/GridIntList.java b/modules/core/src/main/java/org/apache/ignite/internal/util/GridIntList.java index 0457b592dfce9..b4590ef640230 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/util/GridIntList.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/util/GridIntList.java @@ -223,6 +223,17 @@ public int[] arrayCopy() { return res; } + /** + * @param from the initial index of the range to be copied, inclusive. + * @param to the final index of the range to be copied, exclusive. + * @return a new {@link GridIntList} containing the specified range from the current {@link GridIntList}. + */ + public GridIntList copyOfRange(int from, int to) { + assert 0 <= from && from <= to && to <= idx; + + return new GridIntList(Arrays.copyOfRange(arr, from, to)); + } + /** {@inheritDoc} */ @Override public void writeExternal(ObjectOutput out) throws IOException { out.writeInt(idx); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/distributed/DistributedProcess.java b/modules/core/src/main/java/org/apache/ignite/internal/util/distributed/DistributedProcess.java index f4aeb18fd2882..d6365de61f2c8 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/util/distributed/DistributedProcess.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/util/distributed/DistributedProcess.java @@ -499,6 +499,21 @@ public enum DistributedProcessType { /** * Snapshot partitions validation. */ - CHECK_SNAPSHOT_PARTS + CHECK_SNAPSHOT_PARTS, + + /** + * Rolling Upgrade activation. + */ + RU_ENABLE, + + /** + * Rolling Upgrade node fence activation. + */ + RU_PREPARE_VERSION_FINALIZATION, + + /** + * Rolling Upgrade cluster version finalization. + */ + RU_COMPLETE_VERSION_FINALIZATION } } diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySpi.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySpi.java index 6e14dfa8f272f..31747bb33a673 100644 --- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySpi.java +++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySpi.java @@ -97,7 +97,6 @@ import org.apache.ignite.spi.discovery.DiscoverySpiOrderSupport; import org.apache.ignite.spi.discovery.tcp.internal.DiscoveryDataPacket; import org.apache.ignite.spi.discovery.tcp.internal.TcpDiscoveryNode; -import org.apache.ignite.spi.discovery.tcp.internal.TcpDiscoveryNodesRing; import org.apache.ignite.spi.discovery.tcp.internal.TcpDiscoveryStatistics; import org.apache.ignite.spi.discovery.tcp.ipfinder.TcpDiscoveryIpFinder; import org.apache.ignite.spi.discovery.tcp.ipfinder.jdbc.TcpDiscoveryJdbcIpFinder; @@ -515,16 +514,6 @@ public ClusterNode getNode0(UUID id) { return getNode(id); } - /** - * @return TCP discovery nodes ring. - */ - @Nullable public TcpDiscoveryNodesRing discoveryRing() { - if (impl instanceof ServerImpl) - return ((ServerImpl)impl).ring(); - - return null; - } - /** {@inheritDoc} */ @Override public boolean pingNode(UUID nodeId) { return impl.pingNode(nodeId); diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/internal/TcpDiscoveryNodesRing.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/internal/TcpDiscoveryNodesRing.java index 7752994cf00e9..5cafd17073233 100644 --- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/internal/TcpDiscoveryNodesRing.java +++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/internal/TcpDiscoveryNodesRing.java @@ -31,7 +31,6 @@ import org.apache.ignite.IgniteSystemProperties; import org.apache.ignite.cluster.ClusterNode; import org.apache.ignite.internal.util.lang.ClusterNodeFunc; -import org.apache.ignite.internal.util.lang.IgnitePair; import org.apache.ignite.internal.util.tostring.GridToStringExclude; import org.apache.ignite.internal.util.tostring.GridToStringInclude; import org.apache.ignite.internal.util.typedef.F; @@ -40,7 +39,6 @@ import org.apache.ignite.internal.util.typedef.internal.S; import org.apache.ignite.internal.util.typedef.internal.U; import org.apache.ignite.lang.IgnitePredicate; -import org.apache.ignite.lang.IgniteProductVersion; import org.jetbrains.annotations.Nullable; /** @@ -96,24 +94,6 @@ public class TcpDiscoveryNodesRing { @GridToStringExclude private final ReadWriteLock rwLock = new ReentrantReadWriteLock(); - /** Minimum node version in the cluster. */ - private IgniteProductVersion minNodeVer; - - /** Maximum node version in the cluster. */ - private IgniteProductVersion maxNodeVer; - - /** Returns min and max node versions. */ - public IgnitePair minMaxNodeVersions() { - rwLock.readLock().lock(); - - try { - return F.pair(minNodeVer, maxNodeVer); - } - finally { - rwLock.readLock().unlock(); - } - } - /** * Sets local node. * @@ -262,8 +242,6 @@ public boolean add(TcpDiscoveryNode node) { nodeOrder = node.internalOrder(); maxInternalOrder = node.internalOrder(); - - initializeMinMaxVersions(); } finally { rwLock.writeLock().unlock(); @@ -334,8 +312,6 @@ public void restoreTopology(Iterable nodes, long topVer) { } nodeOrder = topVer; - - initializeMinMaxVersions(); } finally { rwLock.writeLock().unlock(); @@ -382,8 +358,6 @@ public void restoreTopology(Iterable nodes, long topVer) { nodes.remove(rmv); } - initializeMinMaxVersions(); - return rmv; } finally { @@ -415,11 +389,6 @@ public void clear() { maxInternalOrder = 0; topVer = 0; - - if (locNode != null) { - minNodeVer = locNode.version(); - maxNodeVer = locNode.version(); - } } finally { rwLock.writeLock().unlock(); @@ -727,22 +696,6 @@ private Collection serverNodes(@Nullable final Collection 0) - maxNodeVer = node.version(); - } - } - /** {@inheritDoc} */ @Override public String toString() { rwLock.readLock().lock(); diff --git a/modules/core/src/test/java/org/apache/ignite/internal/GridReleaseTypeSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/GridReleaseTypeSelfTest.java deleted file mode 100644 index 853f330c107f0..0000000000000 --- a/modules/core/src/test/java/org/apache/ignite/internal/GridReleaseTypeSelfTest.java +++ /dev/null @@ -1,529 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.ignite.internal; - -import java.util.Collection; -import java.util.List; -import java.util.Map; -import java.util.function.UnaryOperator; -import org.apache.ignite.IgniteCheckedException; -import org.apache.ignite.IgniteException; -import org.apache.ignite.Ignition; -import org.apache.ignite.cluster.ClusterState; -import org.apache.ignite.configuration.DataStorageConfiguration; -import org.apache.ignite.configuration.IgniteConfiguration; -import org.apache.ignite.internal.util.function.ThrowableSupplier; -import org.apache.ignite.internal.util.lang.IgnitePair; -import org.apache.ignite.internal.util.lang.RunnableX; -import org.apache.ignite.internal.util.typedef.F; -import org.apache.ignite.internal.util.typedef.X; -import org.apache.ignite.lang.IgniteProductVersion; -import org.apache.ignite.spi.IgniteSpiException; -import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi; -import org.apache.ignite.testframework.GridTestUtils; -import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; -import org.junit.Test; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; - -import static org.apache.ignite.testframework.GridTestUtils.assertThrows; -import static org.apache.ignite.testframework.GridTestUtils.waitForCondition; -import static org.junit.Assume.assumeTrue; - -/** - * Test Rolling Upgrade release types. - */ -@RunWith(Parameterized.class) -public class GridReleaseTypeSelfTest extends GridCommonAbstractTest { - /** */ - private String nodeVer; - - /** - * Indicates whether the tested node is started as a client. - * This flag is used to run all test cases for both client and server node configurations. - */ - @Parameterized.Parameter - public boolean client; - - /** Persistence. */ - @Parameterized.Parameter(1) - public boolean persistence; - - /** @return Test parameters. */ - @Parameterized.Parameters(name = "client={0}, persistence={1}") - public static Collection parameters() { - return GridTestUtils.cartesianProduct(List.of(false, true), List.of(false, true)); - } - - /** {@inheritDoc} */ - @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception { - IgniteConfiguration cfg = super.getConfiguration(igniteInstanceName); - - TcpDiscoverySpi discoSpi = new TcpDiscoverySpi() { - @Override public void setNodeAttributes(Map attrs, - IgniteProductVersion ver) { - super.setNodeAttributes(attrs, ver); - - attrs.put(IgniteNodeAttributes.ATTR_BUILD_VER, nodeVer); - } - }; - - discoSpi.setIpFinder(sharedStaticIpFinder); - - cfg.setDiscoverySpi(discoSpi); - - DataStorageConfiguration storageCfg = new DataStorageConfiguration(); - - storageCfg.getDefaultDataRegionConfiguration().setPersistenceEnabled(persistence); - - cfg.setDataStorageConfiguration(storageCfg); - - return cfg; - } - - /** {@inheritDoc} */ - @Override protected void afterTest() throws Exception { - stopAllGrids(); - cleanPersistenceDir(); - } - - /** */ - @Test - public void testTwoConflictVersions() { - testConflictVersions("2.18.0", "2.16.0", client); - testConflictVersions("2.21.0", "2.23.1", client); - testConflictVersions("2.20.1", "2.20.2", client); - } - - /** */ - @Test - public void testThreeConflictVersions() throws Exception { - testConflictVersionsWithRollingUpgrade("2.18.0", "2.18.1", "2.18.2", client, "2.18.1"); - - testConflictVersionsWithRollingUpgrade("2.18.0", "2.18.1", "2.17.2", client, "2.18.1"); - - testConflictVersionsWithRollingUpgrade("2.18.1", "2.19.0", "2.19.1", client, "2.19.0"); - - testConflictVersionsWithRollingUpgrade("2.18.1", "2.18.2", "2.18.0", client, "2.18.2"); - } - - /** */ - @Test - public void testTwoCompatibleVersions() throws Exception { - testCompatibleVersions("2.18.0", "2.18.0", client, null); - testCompatibleVersions("2.19.2", "2.19.2", client, null); - - testCompatibleVersions("2.18.0", "2.18.1", client, "2.18.1"); - testCompatibleVersions("2.18.2", "2.19.0", client, "2.19.0"); - } - - /** */ - @Test - public void testThreeCompatibleVersions() throws Exception { - testCompatibleVersions("2.18.0", "2.18.0", "2.18.0", client, null); - testCompatibleVersions("2.18.2", "2.18.2", "2.18.2", client, null); - - testCompatibleVersions("2.18.0", "2.18.1", "2.18.1", client, "2.18.1"); - testCompatibleVersions("2.18.1", "2.19.0", "2.18.1", client, "2.19.0"); - } - - /** */ - @Test - public void testForwardRollingUpgrade() throws Exception { - doTestRollingUpgrade("2.18.0", "2.18.1", false); - } - - /** */ - @Test - public void testForceRollingUpgrade() throws Exception { - doTestRollingUpgrade("2.18.0", "2.19.1", true); - } - - /** Performs full rolling upgrade scenario. */ - private void doTestRollingUpgrade(String curVer, String targetVer, boolean force) throws Exception { - IgniteEx ign0 = startGrid(0, curVer, false); - IgniteEx ign1 = startGrid(1, curVer, client); - IgniteEx ign2 = startGrid(2, curVer, client); - - assertClusterSize(3); - - assertRemoteRejected(() -> startGrid(3, targetVer, client)); - - configureRollingUpgradeVersion(ign0, targetVer, force); - - for (int i = 0; i < 3; i++) { - int finalI = i; - assertTrue(waitForCondition(() -> grid(finalI).context().rollingUpgrade().enabled(), getTestTimeout())); - } - - ign2.close(); - - assertClusterSize(2); - - startGrid(2, targetVer, client); - - assertClusterSize(3); - - ign1.close(); - - assertClusterSize(2); - - startGrid(1, targetVer, client); - - assertClusterSize(3); - - ign0.close(); - - assertClusterSize(2); - - startGrid(0, targetVer, false); - - assertClusterSize(3); - - if (client) - grid(0).context().rollingUpgrade().disable(); - else - grid(2).context().rollingUpgrade().disable(); - - for (int i = 0; i < 3; i++) { - if (!grid(i).localNode().isClient()) - assertFalse(grid(i).context().rollingUpgrade().enabled()); - } - - assertRemoteRejected(() -> startGrid(3, curVer, client)); - } - - /** */ - @Test - public void testJoiningNodeFailed() throws Exception { - int joinTimeout = 5_000; - - IgniteEx ign0 = startGrid(0, "2.18.0", false, - cfg -> { - ((TcpDiscoverySpi)cfg.getDiscoverySpi()).setJoinTimeout(joinTimeout); - return cfg; - }); - - configureRollingUpgradeVersion(ign0, "2.18.1"); - - RunnableX runnableX = () -> startGrid(1, "2.18.1", false, - cfg -> { - TcpDiscoverySpi oldSpi = (TcpDiscoverySpi)cfg.getDiscoverySpi(); - - TcpDiscoverySpi newSpi = new TcpDiscoverySpi() { - @Override public void setNodeAttributes(Map attrs, IgniteProductVersion ver) { - super.setNodeAttributes(attrs, ver); - - attrs.put(IgniteNodeAttributes.ATTR_BUILD_VER, nodeVer); - attrs.put(IgniteNodeAttributes.ATTR_MARSHALLER, "null"); - } - }; - - newSpi.setIpFinder(oldSpi.getIpFinder()); - - return cfg.setDiscoverySpi(newSpi); - }); - - Throwable e = assertThrows(log, runnableX, IgniteException.class, null); - - assertTrue(X.hasCause(e, "Local node's marshaller differs from remote node's marshaller", IgniteSpiException.class)); - - assertDisablingFails(ign0, "Can't disable rolling upgrade with different versions in cluster"); - - doSleep(joinTimeout * 3); - - ign0.context().rollingUpgrade().disable(); - - assertFalse(ign0.context().rollingUpgrade().enabled()); - } - - /** */ - @Test - public void testJoiningNodeLeft() throws Exception { - IgniteEx ign0 = startGrid(0, "2.18.0", false, cfg -> { - ((TcpDiscoverySpi)cfg.getDiscoverySpi()).setJoinTimeout(0); - return cfg; - }); - - configureRollingUpgradeVersion(ign0, "2.18.1"); - - try (IgniteEx ignore = startGrid(1, "2.18.1", false)) { - assertClusterSize(2); - } - - assertClusterSize(1); - - ign0.context().rollingUpgrade().disable(); - } - - /** */ - @Test - public void testCoordinatorChange() throws Exception { - IgniteEx ign0 = startGrid(0, "2.18.0", false); - IgniteEx ign1 = startGrid(1, "2.18.0", false); - - configureRollingUpgradeVersion(ign0, "2.19.0"); - - startGrid(2, "2.19.0", false); - - assertClusterSize(3); - - ign0.close(); - ign1.close(); - - assertClusterSize(1); - - startGrid(0, "2.18.0", client); - startGrid(1, "2.19.0", client); - - assertClusterSize(3); - - assertRemoteRejected(() -> startGrid(4, "2.20.0", client)); - - assertClusterSize(3); - } - - /** */ - @Test - public void testNodeRestart() throws Exception { - assumeTrue("Distributed metastorage is only preserved across restarts when persistence is enabled", persistence); - - for (int i = 0; i < 3; i++) - startGrid(i, "2.18.0", false); - - assertClusterSize(3); - - configureRollingUpgradeVersion(grid(0), "2.18.1"); - - for (int i = 0; i < 3; i++) - grid(i).close(); - - assertClusterSize(0); - - for (int i = 0; i < 3; i++) - startGrid(i, "2.18.0", false); - - assertClusterSize(3); - - for (int i = 0; i < 3; i++) { - assertTrue(grid(i).context().rollingUpgrade().enabled()); - - IgnitePair stored = grid(i).context().rollingUpgrade().versions(); - - assertEquals(F.pair(IgniteProductVersion.fromString("2.18.0"), IgniteProductVersion.fromString("2.18.1")), stored); - } - } - - /** */ - @Test - public void testRollingUpgradeProcessorVersionCheck() throws Exception { - IgniteEx grid0 = startGrid(0, "2.18.0", false); - startGrid(1, "2.18.0", client); - - assertClusterSize(2); - - assertEnablingFails(grid0, "3.0.0", "Major versions are different"); - assertEnablingFails(grid0, "2.19.2", "Minor version can only be incremented by 1"); - assertEnablingFails(grid0, "2.18.2", "Patch version can only be incremented by 1"); - - IgnitePair newPair = F.pair(IgniteProductVersion.fromString("2.18.0"), - IgniteProductVersion.fromString("2.19.0")); - - grid0.context().rollingUpgrade().enable(newPair.get2(), false); - - assertEnablingFails(grid0, "2.18.1", "Rolling upgrade is already enabled with a different current and target version"); - - for (int i = 0; i < 2; i++) { - assertTrue(waitForCondition(grid(i).context().rollingUpgrade()::enabled, getTestTimeout())); - - assertEquals(newPair, grid(i).context().rollingUpgrade().versions()); - } - } - - /** - * Checks that enabling rolling upgrade fails with expected error message. - * - * @param ex Ex. - * @param ver New version. - * @param errMsg Expected error message. - */ - private void assertEnablingFails(IgniteEx ex, String ver, String errMsg) { - Throwable e = assertThrows(log, - () -> ex.context().rollingUpgrade().enable(IgniteProductVersion.fromString(ver), false), - IgniteException.class, - null); - - assertTrue(e.getMessage().contains(errMsg)); - } - - /** - * Checks that disabling rolling upgrade fails with expected error message. - * - * @param ex Ex. - * @param errMsg Expected error message. - */ - private void assertDisablingFails(IgniteEx ex, String errMsg) { - Throwable e = assertThrows(log, - () -> ex.context().rollingUpgrade().disable(), - IgniteException.class, - null); - - assertTrue(e.getMessage().contains(errMsg)); - } - - /** Tests that starting a node with rejected version fails with remote rejection. */ - private void testConflictVersions(String acceptedVer, String rejVer, boolean client) { - ThrowableSupplier sup = () -> { - IgniteEx ign = startGrid(0, acceptedVer, false); - - startGrid(1, rejVer, client, cfg -> { - TcpDiscoverySpi spi = (TcpDiscoverySpi)cfg.getDiscoverySpi(); - - // Decrease network timeout to reduce waiting time for node failure - // after it has been rejected by the coordinator due to version conflict. - spi.setNetworkTimeout(1_000); - - return cfg; - }); - - return ign; - }; - - assertRemoteRejected(sup); - - stopAllGrids(); - } - - /** Checks that the third grid is not compatible when rolling upgrade version is set. */ - private void testConflictVersionsWithRollingUpgrade(String acceptedVer1, String acceptedVer2, String rejVer, - boolean client, String rollUpVer) throws Exception { - ThrowableSupplier sup = () -> { - IgniteEx ign = startGrid(0, acceptedVer1, false); - - configureRollingUpgradeVersion(ign, rollUpVer); - - startGrid(1, acceptedVer2, client); - - startGrid(2, rejVer, client); - - return ign; - }; - - assertRemoteRejected(sup); - - stopAllGrids(); - - cleanPersistenceDir(); - } - - /** Checks that remote node rejected due to incompatible version. */ - private void assertRemoteRejected(ThrowableSupplier gridStart) { - Throwable e = assertThrows(log, gridStart::get, IgniteCheckedException.class, null); - - assertTrue(X.hasCause(e, "Remote node rejected due to incompatible version for cluster join", IgniteSpiException.class)); - } - - /** Tests two compatible grids. */ - private void testCompatibleVersions(String acceptedVer1, - String acceptedVer2, - boolean client, - String rollUpVerCheck) throws Exception { - IgniteEx grid = startGrid(0, acceptedVer1, false); - - if (rollUpVerCheck != null) - configureRollingUpgradeVersion(grid, rollUpVerCheck); - - startGrid(1, acceptedVer2, client); - - assertClusterSize(2); - - stopAllGrids(); - - if (persistence) - cleanPersistenceDir(); - } - - /** Tests three compatible grids. */ - private void testCompatibleVersions( - String acceptedVer1, - String acceptedVer2, - String acceptedVer3, - boolean client, - String rollUpVerCheck - ) throws Exception { - IgniteEx grid = startGrid(0, acceptedVer1, false); - - if (rollUpVerCheck != null) - configureRollingUpgradeVersion(grid, rollUpVerCheck); - - startGrid(1, acceptedVer2, client); - startGrid(2, acceptedVer3, client); - - assertClusterSize(3); - - stopAllGrids(); - - if (persistence) - cleanPersistenceDir(); - } - - /** Starts grid with required version. */ - private IgniteEx startGrid(int idx, String ver, boolean isClient) throws Exception { - return startGrid(idx, ver, isClient, null); - } - - /** Starts grid with required version and custom configuration. */ - private IgniteEx startGrid(int idx, String ver, boolean isClient, UnaryOperator cfgOp) throws Exception { - nodeVer = ver; - - IgniteEx ign = isClient ? startClientGrid(idx, cfgOp) : startGrid(idx, cfgOp); - - if (persistence) - ign.cluster().state(ClusterState.ACTIVE); - - return ign; - } - - /** */ - private void configureRollingUpgradeVersion(IgniteEx grid, String ver) throws IgniteCheckedException { - configureRollingUpgradeVersion(grid, ver, false); - } - - /** - * @param ver Version for rolling upgrade support. - * @param force Force rolling upgrade. - */ - private void configureRollingUpgradeVersion(IgniteEx grid, String ver, boolean force) throws IgniteCheckedException { - if (ver == null) { - grid.context().rollingUpgrade().disable(); - return; - } - - IgniteProductVersion target = IgniteProductVersion.fromString(ver); - - grid.context().rollingUpgrade().enable(target, force); - } - - /** - * @param size Expected cluster size. - */ - private void assertClusterSize(int size) throws IgniteInterruptedCheckedException { - assertTrue("Expected cluster size: " + size + ", but was: " + Ignition.allGrids().size(), - waitForCondition(() -> Ignition.allGrids().size() == size, getTestTimeout())); - } -} diff --git a/modules/core/src/test/java/org/apache/ignite/internal/IgniteNodeValidationFailedEventTest.java b/modules/core/src/test/java/org/apache/ignite/internal/IgniteNodeValidationFailedEventTest.java index 9bd2dc149a847..97ad9ee70df3c 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/IgniteNodeValidationFailedEventTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/IgniteNodeValidationFailedEventTest.java @@ -38,7 +38,6 @@ public class IgniteNodeValidationFailedEventTest extends GridCommonAbstractTest /** {@inheritDoc} */ @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception { return super.getConfiguration(igniteInstanceName) - .setIncludeEventTypes(EVT_NODE_VALIDATION_FAILED) .setConsistentId(igniteInstanceName); } @@ -98,9 +97,9 @@ public void testNodeValidationFailedEvent() throws Exception { /** */ @Test - public void testEventDisabledByDefault() throws Exception { + public void testEventEnabledByDefault() throws Exception { IgniteEx ignite = startGrid(super.getConfiguration(getTestIgniteInstanceName(0))); - assertFalse(ignite.context().event().isRecordable(EVT_NODE_VALIDATION_FAILED)); + assertTrue(ignite.context().event().isRecordable(EVT_NODE_VALIDATION_FAILED)); } } diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/rollingupgrade/AbstractRollingUpgradeTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/rollingupgrade/AbstractRollingUpgradeTest.java new file mode 100644 index 0000000000000..48d2ce1e1cfce --- /dev/null +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/rollingupgrade/AbstractRollingUpgradeTest.java @@ -0,0 +1,137 @@ +/* + * 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.rollingupgrade; + +import java.util.Collection; +import java.util.Map; +import org.apache.ignite.Ignite; +import org.apache.ignite.configuration.IgniteConfiguration; +import org.apache.ignite.internal.GridKernalContext; +import org.apache.ignite.internal.IgniteEx; +import org.apache.ignite.internal.processors.nodevalidation.DiscoveryNodeValidationProcessor; +import org.apache.ignite.internal.processors.rollingupgrade.feature.IgniteFeature; +import org.apache.ignite.internal.processors.rollingupgrade.feature.IgniteFeatureSet; +import org.apache.ignite.internal.processors.rollingupgrade.feature.IgniteFeatureUtils; +import org.apache.ignite.internal.processors.rollingupgrade.feature.TestIgniteProductFeatures_2_18_0; +import org.apache.ignite.lang.IgniteProductVersion; +import org.apache.ignite.lang.IgniteRunnable; +import org.apache.ignite.plugin.AbstractTestPluginProvider; +import org.apache.ignite.plugin.PluginContext; +import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi; +import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; +import org.jspecify.annotations.Nullable; + +/** */ +public abstract class AbstractRollingUpgradeTest extends GridCommonAbstractTest { + /** */ + protected static final String TEST_DEFAULT_VER = "2.19.0"; + + /** {@inheritDoc} */ + @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception { + return getConfiguration(igniteInstanceName, TEST_DEFAULT_VER); + } + + /** {@inheritDoc} */ + @Override protected void afterTest() throws Exception { + super.afterTest(); + + stopAllGrids(); + } + + /** */ + protected IgniteConfiguration getConfiguration(int idx, String ver) throws Exception { + return getConfiguration(getTestIgniteInstanceName(idx), ver); + } + + /** */ + protected IgniteConfiguration getConfiguration(String igniteInstanceName, String ver) throws Exception { + IgniteConfiguration cfg = super.getConfiguration(igniteInstanceName); + + IgniteProductVersion testProductVer = IgniteProductVersion.fromString(ver); + IgniteFeatureSet testProductFeatures = IgniteFeatureSet.buildFrom(declaredFeatures(ver)); + + assertNotNull(testProductFeatures); + + cfg.setPluginProviders(new AbstractTestPluginProvider() { + @Override public String name() { + return "test-rolling-upgrade-processor-provider"; + } + + /** {@inheritDoc} */ + @Override public @Nullable T createComponent(PluginContext ctx, Class cls) { + if (cls.isAssignableFrom(DiscoveryNodeValidationProcessor.class)) + return (T)new TestRollingUpgradeProcessor(((IgniteEx)ctx.grid()).context(), testProductVer, testProductFeatures); + + return null; + } + }); + + TcpDiscoverySpi discoSpi = new TcpDiscoverySpi() { + @Override public void setNodeAttributes(Map attrs, IgniteProductVersion ignored) { + super.setNodeAttributes(attrs, IgniteProductVersion.fromString(ver)); + } + }; + + discoSpi.setIpFinder(((TcpDiscoverySpi)cfg.getDiscoverySpi()).getIpFinder()); + + cfg.setDiscoverySpi(discoSpi); + + return cfg; + } + + /** */ + public IgniteEx startGrid(int idx, String ver) throws Exception { + return startGrid(getConfiguration(idx, ver)); + } + + /** */ + public IgniteEx startClientGrid(int idx, String ver) throws Exception { + return startClientGrid(getConfiguration(idx, ver)); + } + + /** */ + protected Collection declaredFeatures(String ver) throws Exception { + Class cls = Class.forName( + TestIgniteProductFeatures_2_18_0.class.getPackageName() + ".TestIgniteProductFeatures_" + ver.replace(".", "_")); + + return IgniteFeatureUtils.readDeclaredFeatures(cls); + } + + /** */ + private static class TestRollingUpgradeProcessor extends RollingUpgradeProcessor { + /** */ + public TestRollingUpgradeProcessor(GridKernalContext ctx, IgniteProductVersion productVersion, IgniteFeatureSet productFeatures) { + super(ctx, () -> productVersion, () -> productFeatures); + } + } + + /** */ + protected boolean isFeatureActive(Ignite ignite, IgniteFeature feature) { + return ((IgniteEx)ignite).context().rollingUpgrade().features().isActive(feature); + } + + /** */ + protected void listenFeatureActivation(Ignite ignite, IgniteFeature feature, IgniteRunnable lsnr) { + ((IgniteEx)ignite).context().rollingUpgrade().features().listenActivation(feature, lsnr); + } + + /** */ + public static String toString(IgniteProductVersion ver) { + return ver.major() + "." + ver.minor() + "." + ver.maintenance(); + } +} diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/rollingupgrade/ClusterVersionsRollingUpgradeTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/rollingupgrade/ClusterVersionsRollingUpgradeTest.java new file mode 100644 index 0000000000000..8e138a572eeb7 --- /dev/null +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/rollingupgrade/ClusterVersionsRollingUpgradeTest.java @@ -0,0 +1,383 @@ +/* + * 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.rollingupgrade; + +import java.util.ArrayList; +import java.util.Collection; +import java.util.List; +import java.util.concurrent.Callable; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.atomic.AtomicInteger; +import org.apache.ignite.Ignite; +import org.apache.ignite.IgniteCheckedException; +import org.apache.ignite.Ignition; +import org.apache.ignite.internal.IgniteEx; +import org.apache.ignite.internal.processors.rollingupgrade.feature.IgniteFeature; +import org.apache.ignite.internal.util.typedef.F; +import org.apache.ignite.spi.IgniteSpiException; +import org.apache.ignite.testframework.GridTestUtils; +import org.junit.Test; + +import static java.util.concurrent.TimeUnit.MILLISECONDS; +import static org.apache.ignite.internal.processors.rollingupgrade.feature.TestIgniteProductFeatures_2_19_2.VER_2_19_2_ID_1_FEATURE; + +/** */ +public class ClusterVersionsRollingUpgradeTest extends AbstractRollingUpgradeTest { + /** */ + @Test + public void testVersionUpgradeDisabledSingleServer() throws Exception { + startGrid(0); + + checkClusterVersionUpgradeInactive("2.19.0"); + } + + /** */ + @Test + public void testVersionUpgradeDisabledNodeJoin() throws Exception { + startCluster(); + + checkProductFeaturesActive("2.19.0"); + + String msg = "The joining node version differs from the version of the cluster"; + + assertJoinFailed(() -> startGrid(3, "2.18.0"), msg); + assertJoinFailed(() -> startClientGrid(3, "2.18.0"), msg); + assertJoinFailed(() -> startGrid(3, "2.19.1"), msg); + assertJoinFailed(() -> startClientGrid(3, "2.19.1"), msg); + + checkClusterVersionUpgradeInactive("2.19.0"); + } + + /** */ + @Test + public void testVersionUpgradeEnabledNodeJoin() throws Exception { + startCluster(); + + grid(1).context().rollingUpgrade().enableVersionUpgrade(); + + checkVersionUpgradeEnabledStatus(true); + + stopGrid(2); + checkJoinSuccess(() -> startClientGrid(2)); + + stopGrid(0); + checkJoinSuccess(() -> startGrid(0)); + + checkJoinSuccess(() -> startGrid(3)); + + checkVersionUpgradeEnabledStatus(true); + + assertJoinFailed( + () -> startGrid(5, "2.18.0"), + "The joining node is incompatible with the current state of the cluster version rolling upgrade being in progress"); + assertJoinFailed( + () -> startGrid(5, "2.21.0"), + "Rolling Upgrade is not available between the current cluster logical version and the joining node product version"); + + checkJoinSuccess(() -> startGrid(4, "2.19.1")); + checkJoinSuccess(() -> startClientGrid(5, "2.19.1")); + + assertJoinFailed( + () -> startGrid(6, "2.19.2"), + "The joining node is incompatible with the current state of the cluster version rolling upgrade being in progress"); + + stopGrid(0); + checkJoinSuccess(() -> startGrid(0)); + + stopGrid(2); + checkJoinSuccess(() -> startClientGrid(2)); + } + + /** */ + @Test + public void testClusterVersionUpgrade() throws Exception { + startCluster(); + + grid(1).context().rollingUpgrade().enableVersionUpgrade(); + + upgradeNodeVersion(1, "2.19.2"); + + checkVersionFinalizationFailed( + 1, + "Cluster version finalization failed. The topology contains nodes running multiple different versions" + ); + + upgradeNodeVersion(0, "2.19.2"); + + checkVersionFinalizationFailed( + 1, + "Cluster version finalization failed. The topology contains nodes running multiple different versions" + ); + + upgradeNodeVersion(2, "2.19.2"); + + finalizeClusterVersion(1, "2.19.2"); + } + + /** */ + @Test + public void testJoinAfterClusterVersionFinalization() throws Exception { + startCluster(); + + grid(1).context().rollingUpgrade().enableVersionUpgrade(); + + upgradeNodeVersion(0, "2.19.2"); + upgradeNodeVersion(2, "2.19.2"); + stopGrid(1); + + finalizeClusterVersion(0, "2.19.2"); + + checkJoinSuccess(() -> startGrid(1, "2.19.2")); + + checkClusterVersionUpgradeInactive("2.19.2"); + } + + /** */ + @Test + public void testFeatureActivationListener() throws Exception { + startCluster(); + + grid(1).context().rollingUpgrade().enableVersionUpgrade(); + + CountDownLatch featureActivationLatch = new CountDownLatch(3); + + for (Ignite node : Ignition.allGrids()) { + int nodeIdx = getTestIgniteInstanceIndex(node.name()); + + upgradeNodeVersion(nodeIdx, "2.19.2"); + checkFeatureActivationSubscription(nodeIdx, VER_2_19_2_ID_1_FEATURE, featureActivationLatch); + } + + finalizeClusterVersion(1, "2.19.2"); + + assertTrue(featureActivationLatch.await(getTestTimeout(), MILLISECONDS)); + } + + /** */ + @Test + public void testIterativeClusterVersionUpgrade() throws Exception { + startCluster(); + + grid(1).context().rollingUpgrade().enableVersionUpgrade(); + + upgradeNodeVersion(0, "2.19.2"); + upgradeNodeVersion(1, "2.19.2"); + upgradeNodeVersion(2, "2.19.2"); + upgradeNodeVersion(2, "2.19.3"); + + stopGrid(1); + assertJoinFailed( + () -> startGrid(1, TEST_DEFAULT_VER), + "The joining node is incompatible with the current state of the cluster version rolling upgrade being in progress"); + checkJoinSuccess(() -> startGrid(1, "2.19.3")); + checkVersionUpgradeInProgress(TEST_DEFAULT_VER, "2.19.3"); + + upgradeNodeVersion(0, "2.19.3"); + + finalizeClusterVersion(1, "2.19.3"); + } + + /** */ + @Test + public void testProductVersionChangeDuringClusterVersionUpgrade() throws Exception { + startCluster(); + + grid(1).context().rollingUpgrade().enableVersionUpgrade(); + + upgradeNodeVersion(0, "2.19.2"); + upgradeNodeVersion(0, "2.19.3"); + + stopGrid(1); + assertJoinFailed( + () -> startGrid(1, "2.19.2"), + "The joining node is incompatible with the current state of the cluster version rolling upgrade being in progress"); + checkJoinSuccess(() -> startGrid(1, "2.19.3")); + checkVersionUpgradeInProgress(TEST_DEFAULT_VER, "2.19.3"); + + upgradeNodeVersion(2, "2.19.3"); + + finalizeClusterVersion(1, "2.19.3"); + } + + /** */ + @Test + public void testUpgradeBetweenVersionsWithCherryPicks() throws Exception { + startCluster("2.19.3"); + + grid(1).context().rollingUpgrade().enableVersionUpgrade(); + + assertJoinFailed( + () -> startGrid(3, "2.20.0"), + "Rolling Upgrade is not available between the current cluster logical version and the joining node product version"); + + for (Ignite node : Ignition.allGrids()) + upgradeNodeVersion(getTestIgniteInstanceIndex(node.name()), "2.20.1"); + + finalizeClusterVersion(1, "2.20.1"); + } + + /** */ + @Test + public void testUpgradeBetweenVersionsWithCherryPicksAndContinuousRange() throws Exception { + startCluster("2.20.1"); + + grid(1).context().rollingUpgrade().enableVersionUpgrade(); + + for (Ignite node : Ignition.allGrids()) + upgradeNodeVersion(getTestIgniteInstanceIndex(node.name()), "2.21.1"); + + finalizeClusterVersion(1, "2.21.1"); + } + + /** */ + private void startCluster() throws Exception { + startCluster(TEST_DEFAULT_VER); + } + + /** */ + private void startCluster(String ver) throws Exception { + startGrid(0, ver); + startGrid(1, ver); + startClientGrid(2, ver); + } + + /** */ + private void checkVersionFinalizationFailed(int initiatorNodeIdx, String msg) { + GridTestUtils.assertThrowsAnyCause( + log, + () -> { + grid(initiatorNodeIdx).context().rollingUpgrade().finalizeClusterVersion(); + + return null; + }, + IgniteCheckedException.class, + msg + ); + } + + /** */ + private void checkJoinSuccess(Callable noseStarter) throws Exception { + int clusterSize = clusterNode().cluster().nodes().size(); + + noseStarter.call(); + + assertEquals(clusterSize + 1, clusterNode().cluster().nodes().size()); + } + + /** */ + private void assertJoinFailed(Callable call, String msg) { + GridTestUtils.assertThrowsAnyCause(log, call, IgniteSpiException.class, msg); + } + + /** */ + private IgniteEx clusterNode() { + return (IgniteEx)Ignition.allGrids().stream().filter(i -> !i.cluster().localNode().isClient()).findFirst().orElseThrow(); + } + + /** */ + private void checkVersionUpgradeInProgress(String sourceVer, String targetVer) throws Exception { + Collection newFeatures = newFeatures(sourceVer, targetVer); + + checkVersionUpgradeEnabledStatus(true); + checkProductFeaturesActive(sourceVer); + checkFeaturesNotActive(newFeatures); + } + + /** */ + private void checkClusterVersionUpgradeInactive(String expVer) throws Exception { + checkVersionUpgradeEnabledStatus(false); + checkProductFeaturesActive(expVer); + } + + /** */ + private void checkVersionUpgradeEnabledStatus(boolean enabled) { + List cluster = Ignition.allGrids(); + + for (Ignite ignite : cluster) + assertEquals(enabled, ((IgniteEx)ignite).context().rollingUpgrade().isVersionUpgradeEnabled()); + } + + /** */ + private void checkProductFeaturesActive(String ver) throws Exception { + Collection features = declaredFeatures(ver); + + List cluster = Ignition.allGrids(); + + AtomicInteger validatedFeatures = new AtomicInteger(); + + for (Ignite ignite : cluster) { + for (IgniteFeature feature : features) { + assertTrue(isFeatureActive(ignite, feature)); + listenFeatureActivation(ignite, feature, validatedFeatures::incrementAndGet); + } + } + + assertEquals(cluster.size() * features.size(), validatedFeatures.get()); + } + + /** */ + private void checkFeaturesNotActive(Collection features) { + assertFalse(F.isEmpty(features)); + + for (Ignite ignite : Ignition.allGrids()) { + for (IgniteFeature feature : features) + assertFalse(isFeatureActive(ignite, feature)); + } + } + + /** */ + private Collection newFeatures(String srcVer, String targetVer) throws Exception { + Collection srcFeatures = declaredFeatures(srcVer); + Collection targetFeatures = declaredFeatures(targetVer); + + List res = new ArrayList<>(); + + for (IgniteFeature targetFeature : targetFeatures) { + if (!srcFeatures.contains(targetFeature)) + res.add(targetFeature); + } + + return res; + } + + /** */ + private void checkFeatureActivationSubscription(int nodeIdx, IgniteFeature feature, CountDownLatch featureActivationLatch) { + long expCnt = featureActivationLatch.getCount(); + + assertFalse(isFeatureActive(grid(nodeIdx), feature)); + listenFeatureActivation(grid(nodeIdx), feature, featureActivationLatch::countDown); + assertEquals(expCnt, featureActivationLatch.getCount()); + } + + /** */ + public void upgradeNodeVersion(int nodeIdx, String targetVer) throws Exception { + String srcVer = toString(grid(nodeIdx).context().rollingUpgrade().features().activeFeatures().version()); + boolean isClient = grid(nodeIdx).context().clientNode(); + + stopGrid(nodeIdx); + checkJoinSuccess(() -> isClient ? startClientGrid(nodeIdx, targetVer) : startGrid(nodeIdx, targetVer)); + checkVersionUpgradeInProgress(srcVer, targetVer); + } + + /** */ + public void finalizeClusterVersion(int nodeIdx, String expVer) throws Exception { + grid(nodeIdx).context().rollingUpgrade().finalizeClusterVersion(); + + checkClusterVersionUpgradeInactive(expVer); + } +} diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/rollingupgrade/feature/IgniteFeatureSetTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/rollingupgrade/feature/IgniteFeatureSetTest.java new file mode 100644 index 0000000000000..c6567279b8cb6 --- /dev/null +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/rollingupgrade/feature/IgniteFeatureSetTest.java @@ -0,0 +1,166 @@ +/* + * 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.rollingupgrade.feature; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.Iterator; +import java.util.List; +import java.util.NoSuchElementException; +import org.apache.ignite.internal.util.GridIntList; +import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; +import org.junit.Test; + +import static java.util.Arrays.asList; +import static org.apache.ignite.testframework.GridTestUtils.assertThrows; + +/** */ +public class IgniteFeatureSetTest extends GridCommonAbstractTest { + /** */ + @Test + public void testFeatureSetElements() { + checkFeatureSetElements(List.of(0), featureSetOf(0)); + checkFeatureSetElements(asList(0, 1, 2), featureSetOf(0, 1, 2)); + checkFeatureSetElements(asList(0, 2, 4, 6), featureSetOf(0, 2, 4, 6)); + checkFeatureSetElements(asList(0, 1, 2, 3), featureSetOf(3)); + checkFeatureSetElements(asList(0, 1, 2, 3, 4), featureSetOf(2, 3, 4)); + checkFeatureSetElements(asList(0, 10), featureSetOf(0, 10)); + checkFeatureSetElements(asList(0, 1, 2, 10), featureSetOf(0, 1, 2, 10)); + checkFeatureSetElements(asList(0, 1, 2, 3, 4, 10), featureSetOf(2, 3, 4, 10)); + checkFeatureSetElements(asList(0, 1, 2, 3, 4, 10, 15), featureSetOf(2, 3, 4, 10, 15)); + checkFeatureSetElements(asList(0, 1, 2, 3, 4, 5, 10, 15), featureSetOf(5, 2, 10, 3, 15, 4)); + } + + /** */ + @Test + public void testNotContains() { + IgniteFeatureSet featureSet = featureSetOf(2, 3, 4, 10, 15); + + assertFalse(featureSet.contains(5)); + assertFalse(featureSet.contains(11)); + assertFalse(featureSet.contains(16)); + } + + /** */ + @Test + public void testUpgradeAvailability() { + assertTrue(featureSetOf(0).isUpgradableTo(featureSetOf(0))); + assertTrue(featureSetOf(0).isUpgradableTo(featureSetOf(0, 1, 2))); + assertTrue(featureSetOf(0, 1, 2).isUpgradableTo(featureSetOf(0, 1, 2, 3))); + assertTrue(featureSetOf(0, 1, 2).isUpgradableTo(featureSetOf(2, 3, 4))); + assertTrue(featureSetOf(0, 1, 2).isUpgradableTo(featureSetOf(3, 4, 5))); + assertTrue(featureSetOf(1).isUpgradableTo(featureSetOf(1))); + assertTrue(featureSetOf(2, 3, 4).isUpgradableTo(featureSetOf(2, 3, 4))); + assertTrue(featureSetOf(2, 3, 4).isUpgradableTo(featureSetOf(2, 3, 4, 5, 6))); + assertTrue(featureSetOf(2, 3, 4).isUpgradableTo(featureSetOf(2, 3, 4, 5, 10))); + assertTrue(featureSetOf(0, 10).isUpgradableTo(featureSetOf(0, 1, 2, 10))); + assertTrue(featureSetOf(2, 3, 4, 8, 10).isUpgradableTo(featureSetOf(2, 3, 4, 5, 6, 8, 10))); + assertTrue(featureSetOf(2, 3, 4, 8, 10).isUpgradableTo(featureSetOf(0, 1, 2, 3, 4, 5, 6, 7, 8, 10))); + + assertFalse(featureSetOf(1).isUpgradableTo(featureSetOf(0))); + assertFalse(featureSetOf(0, 1, 2).isUpgradableTo(featureSetOf(4, 5, 6))); + assertFalse(featureSetOf(0, 1, 2, 3).isUpgradableTo(featureSetOf(1, 2))); + assertFalse(featureSetOf(2, 3, 4).isUpgradableTo(featureSetOf(0, 1))); + assertFalse(featureSetOf(2, 3, 4, 10).isUpgradableTo(featureSetOf(2, 3, 4, 5, 6, 11))); + assertFalse(featureSetOf(2, 3, 4, 8, 10).isUpgradableTo(featureSetOf(2, 3, 4, 5, 6, 10))); + } + + /** */ + @Test + public void testDifference() { + assertTrue(featureSetOf(0).difference(featureSetOf(0)).isEmpty()); + assertTrue(featureSetOf(0, 1, 2).difference(featureSetOf(0, 1, 2)).isEmpty()); + assertTrue(featureSetOf(0, 1, 2, 10).difference(featureSetOf(0, 1, 2, 10)).isEmpty()); + assertTrue(featureSetOf(2).difference(featureSetOf(2)).isEmpty()); + assertTrue(featureSetOf(2, 3, 4).difference(featureSetOf(2, 3, 4)).isEmpty()); + assertTrue(featureSetOf(2, 3, 4, 10).difference(featureSetOf(2, 3, 4, 10)).isEmpty()); + + assertEquals(intList(3), featureSetOf(0, 1, 2).difference(featureSetOf(0, 1, 2, 3))); + + assertEquals(intList(3, 4), featureSetOf(0, 1, 2, 10).difference(featureSetOf(0, 1, 2, 3, 4, 10))); + + assertEquals(intList(6, 7, 8), featureSetOf(2, 3, 4, 5).difference(featureSetOf(2, 3, 4, 5, 6, 7, 8))); + + assertEquals(intList(6, 7, 8, 15), featureSetOf(2, 3, 4, 5, 10).difference(featureSetOf(2, 3, 4, 5, 6, 7, 8, 10, 15))); + } + + /** */ + @Test + public void testEqualsIdenticalSets() { + assertTrue(isIdentical(featureSetOf(0, 1, 2), featureSetOf(0, 1, 2))); + assertTrue(isIdentical(featureSetOf(0, 1, 2, 10), featureSetOf(0, 1, 2, 10))); + assertTrue(isIdentical(featureSetOf(1, 2, 3), featureSetOf(1, 2, 3))); + assertTrue(isIdentical(featureSetOf(1, 2, 3, 10), featureSetOf(1, 2, 3, 10))); + + assertFalse(isIdentical(featureSetOf(0, 1, 2), null)); + assertFalse(isIdentical(featureSetOf(0, 1, 2), featureSetOf(0, 1, 2, 3))); + assertFalse(isIdentical(featureSetOf(1, 2, 3), featureSetOf(1, 2, 3, 10))); + assertFalse(isIdentical(featureSetOf(0, 1, 2, 10), featureSetOf(0, 1, 2, 8, 10))); + } + + /** */ + @Test + public void testToString() { + assertEquals("IgniteFeatureSet [2 -> 5]", featureSetOf(2, 3, 4, 5).toString()); + assertEquals("IgniteFeatureSet [2, 8, 10]", featureSetOf(2, 8, 10).toString()); + assertEquals("IgniteFeatureSet [2, 3, 8, 10]", featureSetOf(2, 3, 8, 10).toString()); + assertEquals("IgniteFeatureSet [2 -> 5, 8, 10]", featureSetOf(2, 3, 4, 5, 8, 10).toString()); + } + + /** */ + @Test + public void testBuildInput() { + assertThrows(log, () -> IgniteFeatureSet.buildFrom(Collections.emptyList()), IllegalArgumentException.class, null); + assertThrows(log, () -> IgniteFeatureSet.buildFrom(null), NullPointerException.class, null); + } + + /** */ + private IgniteFeatureSet featureSetOf(int... ids) { + List features = new ArrayList<>(); + + for (int id : ids) + features.add(new IgniteCoreFeature(id)); + + return IgniteFeatureSet.buildFrom(features); + } + + /** */ + private void checkFeatureSetElements(List expElements, IgniteFeatureSet featureSet) { + List curElements = new ArrayList<>(); + + Iterator iter = featureSet.iterator(); + + while (iter.hasNext()) + curElements.add(iter.next()); + + assertThrows(log, iter::next, NoSuchElementException.class, null); + + assertEquals(expElements, curElements); + assertTrue(expElements.stream().allMatch(featureSet::contains)); + } + + /** */ + private boolean isIdentical(IgniteFeatureSet lhs, IgniteFeatureSet rhs) { + return lhs.equals(rhs) && lhs.hashCode() == rhs.hashCode(); + } + + /** */ + private static GridIntList intList(int... vals) { + return new GridIntList(vals); + } +} diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/rollingupgrade/feature/TestIgniteProductFeatures_2_18_0.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/rollingupgrade/feature/TestIgniteProductFeatures_2_18_0.java new file mode 100644 index 0000000000000..a58eb84b51626 --- /dev/null +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/rollingupgrade/feature/TestIgniteProductFeatures_2_18_0.java @@ -0,0 +1,24 @@ +/* + * 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.rollingupgrade.feature; + +/** */ +public class TestIgniteProductFeatures_2_18_0 { + /** */ + public static final IgniteFeature STUB_FEATURE = new IgniteCoreFeature(Integer.MAX_VALUE); +} diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/rollingupgrade/feature/TestIgniteProductFeatures_2_19_0.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/rollingupgrade/feature/TestIgniteProductFeatures_2_19_0.java new file mode 100644 index 0000000000000..40379830df4ce --- /dev/null +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/rollingupgrade/feature/TestIgniteProductFeatures_2_19_0.java @@ -0,0 +1,24 @@ +/* + * 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.rollingupgrade.feature; + +/** */ +public class TestIgniteProductFeatures_2_19_0 { + /** */ + public static final IgniteFeature ROLLING_UPGRADE_FEATURE = new IgniteCoreFeature(0); +} diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/rollingupgrade/feature/TestIgniteProductFeatures_2_19_1.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/rollingupgrade/feature/TestIgniteProductFeatures_2_19_1.java new file mode 100644 index 0000000000000..2a0db42fbf420 --- /dev/null +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/rollingupgrade/feature/TestIgniteProductFeatures_2_19_1.java @@ -0,0 +1,24 @@ +/* + * 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.rollingupgrade.feature; + +/** */ +public class TestIgniteProductFeatures_2_19_1 { + /** */ + public static final IgniteFeature ROLLING_UPGRADE_FEATURE = TestIgniteProductFeatures_2_19_0.ROLLING_UPGRADE_FEATURE; +} diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/rollingupgrade/feature/TestIgniteProductFeatures_2_19_2.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/rollingupgrade/feature/TestIgniteProductFeatures_2_19_2.java new file mode 100644 index 0000000000000..0a6dfe38582d4 --- /dev/null +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/rollingupgrade/feature/TestIgniteProductFeatures_2_19_2.java @@ -0,0 +1,30 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.ignite.internal.processors.rollingupgrade.feature; + +/** */ +public class TestIgniteProductFeatures_2_19_2 { + /** */ + public static final IgniteFeature ROLLING_UPGRADE_FEATURE = TestIgniteProductFeatures_2_19_1.ROLLING_UPGRADE_FEATURE; + + /** */ + public static final IgniteFeature VER_2_19_2_ID_1_FEATURE = new IgniteCoreFeature(1); + + /** */ + public static final IgniteFeature VER_2_19_2_ID_2_FEATURE = new IgniteCoreFeature(2); +} diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/rollingupgrade/feature/TestIgniteProductFeatures_2_19_3.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/rollingupgrade/feature/TestIgniteProductFeatures_2_19_3.java new file mode 100644 index 0000000000000..a9895040ec7c0 --- /dev/null +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/rollingupgrade/feature/TestIgniteProductFeatures_2_19_3.java @@ -0,0 +1,33 @@ +/* + * 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.rollingupgrade.feature; + +/** */ +public class TestIgniteProductFeatures_2_19_3 { + /** */ + public static final IgniteFeature ROLLING_UPGRADE_FEATURE = TestIgniteProductFeatures_2_19_2.ROLLING_UPGRADE_FEATURE; + + /** */ + public static final IgniteFeature VER_2_19_2_ID_1_FEATURE = TestIgniteProductFeatures_2_19_2.VER_2_19_2_ID_1_FEATURE; + + /** */ + public static final IgniteFeature VER_2_19_2_ID_2_FEATURE = TestIgniteProductFeatures_2_19_2.VER_2_19_2_ID_2_FEATURE; + + /** */ + public static final IgniteFeature VER_2_19_3_ID_6_FEATURE = TestIgniteProductFeatures_2_21_0.VER_2_21_0_ID_6_FEATURE; +} diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/rollingupgrade/feature/TestIgniteProductFeatures_2_20_0.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/rollingupgrade/feature/TestIgniteProductFeatures_2_20_0.java new file mode 100644 index 0000000000000..9ed066abed6f8 --- /dev/null +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/rollingupgrade/feature/TestIgniteProductFeatures_2_20_0.java @@ -0,0 +1,36 @@ +/* + * 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.rollingupgrade.feature; + +/** */ +public class TestIgniteProductFeatures_2_20_0 { + /** */ + public static final IgniteFeature ROLLING_UPGRADE_FEATURE = TestIgniteProductFeatures_2_19_2.ROLLING_UPGRADE_FEATURE; + + /** */ + public static final IgniteFeature VER_2_19_2_ID_1_FEATURE = TestIgniteProductFeatures_2_19_2.VER_2_19_2_ID_1_FEATURE; + + /** */ + public static final IgniteFeature VER_2_19_2_ID_2_FEATURE = TestIgniteProductFeatures_2_19_2.VER_2_19_2_ID_2_FEATURE; + + /** */ + public static final IgniteFeature VER_2_20_0_ID_3_FEATURE = new IgniteCoreFeature(3); + + /** */ + public static final IgniteFeature VER_2_20_0_ID_4_FEATURE = new IgniteCoreFeature(4); +} diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/rollingupgrade/feature/TestIgniteProductFeatures_2_20_1.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/rollingupgrade/feature/TestIgniteProductFeatures_2_20_1.java new file mode 100644 index 0000000000000..1db0109455d8a --- /dev/null +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/rollingupgrade/feature/TestIgniteProductFeatures_2_20_1.java @@ -0,0 +1,39 @@ +/* + * 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.rollingupgrade.feature; + +/** */ +public class TestIgniteProductFeatures_2_20_1 { + /** */ + public static final IgniteFeature ROLLING_UPGRADE_FEATURE = TestIgniteProductFeatures_2_20_0.ROLLING_UPGRADE_FEATURE; + + /** */ + public static final IgniteFeature VER_2_19_2_ID_1_FEATURE = TestIgniteProductFeatures_2_20_0.VER_2_19_2_ID_1_FEATURE; + + /** */ + public static final IgniteFeature VER_2_19_2_ID_2_FEATURE = TestIgniteProductFeatures_2_20_0.VER_2_19_2_ID_2_FEATURE; + + /** */ + public static final IgniteFeature VER_2_20_0_ID_3_FEATURE = TestIgniteProductFeatures_2_20_0.VER_2_20_0_ID_3_FEATURE; + + /** */ + public static final IgniteFeature VER_2_20_0_ID_4_FEATURE = TestIgniteProductFeatures_2_20_0.VER_2_20_0_ID_4_FEATURE; + + /** */ + public static final IgniteFeature VER_2_20_1_ID_6_FEATURE = new IgniteCoreFeature(6); +} diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/rollingupgrade/feature/TestIgniteProductFeatures_2_21_0.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/rollingupgrade/feature/TestIgniteProductFeatures_2_21_0.java new file mode 100644 index 0000000000000..0c6375d7f58be --- /dev/null +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/rollingupgrade/feature/TestIgniteProductFeatures_2_21_0.java @@ -0,0 +1,33 @@ +/* + * 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.rollingupgrade.feature; + +/** */ +public class TestIgniteProductFeatures_2_21_0 { + /** */ + public static final IgniteFeature VER_2_20_0_ID_3_FEATURE = TestIgniteProductFeatures_2_20_1.VER_2_20_0_ID_3_FEATURE; + + /** */ + public static final IgniteFeature VER_2_20_0_ID_4_FEATURE = TestIgniteProductFeatures_2_20_1.VER_2_20_0_ID_4_FEATURE; + + /** */ + public static final IgniteFeature VER_2_21_0_ID_5_FEATURE = new IgniteCoreFeature(5); + + /** */ + public static final IgniteFeature VER_2_21_0_ID_6_FEATURE = new IgniteCoreFeature(6); +} diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/rollingupgrade/feature/TestIgniteProductFeatures_2_21_1.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/rollingupgrade/feature/TestIgniteProductFeatures_2_21_1.java new file mode 100644 index 0000000000000..d9a3e25845437 --- /dev/null +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/rollingupgrade/feature/TestIgniteProductFeatures_2_21_1.java @@ -0,0 +1,36 @@ +/* + * 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.rollingupgrade.feature; + +/** */ +public class TestIgniteProductFeatures_2_21_1 { + /** */ + public static final IgniteFeature VER_2_20_0_ID_3_FEATURE = TestIgniteProductFeatures_2_21_0.VER_2_20_0_ID_3_FEATURE; + + /** */ + public static final IgniteFeature VER_2_20_0_ID_4_FEATURE = TestIgniteProductFeatures_2_21_0.VER_2_20_0_ID_4_FEATURE; + + /** */ + public static final IgniteFeature VER_2_21_0_ID_5_FEATURE = TestIgniteProductFeatures_2_21_0.VER_2_21_0_ID_5_FEATURE; + + /** */ + public static final IgniteFeature VER_2_21_0_ID_6_FEATURE = TestIgniteProductFeatures_2_21_0.VER_2_21_0_ID_6_FEATURE; + + /** */ + public static final IgniteFeature VER_2_21_1_ID_7_FEATURE = new IgniteCoreFeature(7); +} diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteBasicTestSuite.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteBasicTestSuite.java index 40140614e7bf5..4b723cd1b07ba 100644 --- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteBasicTestSuite.java +++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteBasicTestSuite.java @@ -28,7 +28,6 @@ import org.apache.ignite.internal.GridNodeMetricsLogSelfTest; import org.apache.ignite.internal.GridProjectionForCachesSelfTest; import org.apache.ignite.internal.GridReduceSelfTest; -import org.apache.ignite.internal.GridReleaseTypeSelfTest; import org.apache.ignite.internal.GridSelfTest; import org.apache.ignite.internal.GridStartStopSelfTest; import org.apache.ignite.internal.GridStopWithCancelSelfTest; @@ -64,6 +63,7 @@ import org.apache.ignite.internal.processors.odbc.OdbcConfigurationValidationSelfTest; import org.apache.ignite.internal.processors.odbc.OdbcEscapeSequenceSelfTest; import org.apache.ignite.internal.processors.odbc.SqlListenerUtilsTest; +import org.apache.ignite.internal.processors.rollingupgrade.ClusterVersionsRollingUpgradeTest; import org.apache.ignite.internal.product.GridProductVersionSelfTest; import org.apache.ignite.internal.util.nio.IgniteExceptionInNioWorkerSelfTest; import org.apache.ignite.messaging.GridMessagingNoPeerClassLoadingSelfTest; @@ -104,7 +104,7 @@ GridMessagingSelfTest.class, GridMessagingNoPeerClassLoadingSelfTest.class, - GridReleaseTypeSelfTest.class, + ClusterVersionsRollingUpgradeTest.class, GridProductVersionSelfTest.class, GridAffinityAssignmentV2Test.class, GridAffinityAssignmentV2TestNoOptimizations.class, diff --git a/modules/core/src/test/java/org/apache/ignite/util/GridIntListSelfTest.java b/modules/core/src/test/java/org/apache/ignite/util/GridIntListSelfTest.java index 095eb32918898..a5164387e32b7 100644 --- a/modules/core/src/test/java/org/apache/ignite/util/GridIntListSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/util/GridIntListSelfTest.java @@ -97,6 +97,18 @@ public void testSort() { assertEquals(asList(0, 1, 3, 4, 5), list.sort()); } + /** */ + @Test + public void testCopyOfRange() { + assertEquals(asList(), asList(1).copyOfRange(0, 0)); + assertEquals(asList(1), asList(1).copyOfRange(0, 1)); + assertEquals(asList(), asList(1, 2).copyOfRange(1, 1)); + assertEquals(asList(1), asList(1, 2).copyOfRange(0, 1)); + assertEquals(asList(2), asList(1, 2).copyOfRange(1, 2)); + assertEquals(asList(2, 3), asList(1, 2, 3).copyOfRange(1, 3)); + assertEquals(asList(1, 2), asList(1, 2, 3).copyOfRange(0, 2)); + } + /** */ private GridIntList asList(int... vals) { return new GridIntList(vals);