Skip to content

Commit

Permalink
[PLAT-16167] Add validation API to upgrades
Browse files Browse the repository at this point in the history
Summary:
Added new endpoint for validation before running upgrade.
Inside this api we run upgrade task with runOnlyPrechecks flag, but skip creating customer task (to avoid showing this in UI)
Waiting for task to finish with timeout of 1.5 minutes and caching result for 1 minute after task is completed (or failed)

Example call:
```
http://localhost:9000/api/v1/customers/f33e3c9b-75ab-4c30-80ad-cba85646ea39/universes/8466355b-db5c-4394-b361-d5f0c4a92eee/upgrade/validate_before_upgrade?upgradeType=ResizeNode
```
Possible values for upgradeType:
1. Software - Upgrade database version
2. Systemd - Upgrade to systemd
3. VMImage - Upgade linux version
4. Restart - Restart universe
5. Certs - Rotate certificates
6. ToggleTls - Edit security
7. ResizeNode - Resize node
8. Reboot - Reboot universe
9. ThirdPartyPackages - Third party packages upgrade
10. GFlags - Edit flags

Payload:
```
{
  "taskType": "ResizeNode",
  "upgradeOption": "Rolling",
  "universeUUID": "8466355b-db5c-4394-b361-d5f0c4a92eee",
  "sleepAfterMasterRestartMillis": 10000,
  "sleepAfterTServerRestartMillis": 60000,
  "ybSoftwareVersion": "2.23.1.0-b220",
  "clusters": [
    {
        "clusterType": "PRIMARY",
        "userIntent": {
            "instanceType": "c5d.large"
        }
    }
  ]
}
```

Test Plan:
1) Create RF3 universe
2) Run validation for resizeNode - see OK
3) Set incorrect payload -> validation failed
4) Stop one node
5) Mark it as Live
6) Run validation -> see AreNodesToSafeDown fails
7) Disable AreNodesToSafeDown -> see CheckClusterConsistency fails
8) Set timeout to 5 secs -> see that first request fails, wait for task to complete, re-run and get result immediately

Reviewers: nsingh, #yba-api-review, sneelakantan

Reviewed By: nsingh, #yba-api-review, sneelakantan

Subscribers: yugaware

Differential Revision: https://phorge.dev.yugabyte.com/D40219
  • Loading branch information
yorq committed Dec 30, 2024
1 parent 616d4a7 commit 4cb57c7
Show file tree
Hide file tree
Showing 17 changed files with 487 additions and 33 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -170,7 +170,7 @@ public Optional<TaskInfo> maybeSubmitFailoverMasterTask(
log.info(
"Waiting for master failover task to complete for universe {}",
universe.getUniverseUUID());
getCommissioner().waitForTask(customerTask.getTaskUUID());
getCommissioner().waitForTask(customerTask.getTaskUUID(), null);
return TaskInfo.maybeGet(customerTask.getTaskUUID());
} finally {
log.info("Master failover check completed for universe {}", universe.getUniverseUUID());
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -32,6 +32,7 @@
import com.yugabyte.yw.models.Universe;
import com.yugabyte.yw.models.helpers.TaskType;
import io.ebean.annotation.Transactional;
import java.time.Duration;
import java.util.Collections;
import java.util.HashMap;
import java.util.HashSet;
Expand Down Expand Up @@ -401,8 +402,8 @@ public boolean isTaskRunning(UUID taskUuid) {
return taskExecutor.isTaskRunning(taskUuid);
}

public void waitForTask(UUID taskUuid) {
taskExecutor.waitForTask(taskUuid);
public void waitForTask(UUID taskUuid, @Nullable Duration timeout) {
taskExecutor.waitForTask(taskUuid, timeout);
}

public Optional<ObjectNode> mayGetStatus(UUID taskUUID) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -130,7 +130,7 @@ public boolean migrate(UUID customerUuid, UUID universeUuid) {
CustomerTask.TargetType.Universe,
CustomerTask.TaskType.EnableNodeAgent,
universeOpt.get().getName());
commissioner.waitForTask(customerTask.getTaskUUID());
commissioner.waitForTask(customerTask.getTaskUUID(), null);
TaskInfo taskInfo = TaskInfo.getOrBadRequest(taskUuid);
return taskInfo.getTaskState() == TaskInfo.State.Success;
});
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -1112,7 +1112,7 @@ synchronized void updateTaskDetailsOnError(TaskInfo.State state, Throwable t) {
"Failed to execute task %s, hit error:\n\n %s.",
StringUtils.abbreviate(redactedTaskParams, 500),
StringUtils.abbreviateMiddle(cause.getMessage(), "...", 3000));
taskError = new YBAError(Code.INTERNAL_ERROR, errorString);
taskError = new YBAError(Code.INTERNAL_ERROR, errorString, cause.getMessage());
}
log.error(
"Failed to execute task type {} UUID {} details {}, hit error.",
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,37 @@
// Copyright (c) YugaByte, Inc.

package com.yugabyte.yw.commissioner.tasks.params;

import com.yugabyte.yw.models.common.YbaApi;
import io.swagger.annotations.ApiModel;
import io.swagger.annotations.ApiModelProperty;
import java.util.Collections;
import java.util.List;
import lombok.Data;

@Data
@ApiModel(description = "Upgrade Validation Response")
public class PreUpgradeValidationResponse {

@ApiModelProperty(
value =
"WARNING: This is a preview API that could change. "
+ "Indicates whether all the checks passed",
accessMode = ApiModelProperty.AccessMode.READ_ONLY)
@YbaApi(visibility = YbaApi.YbaApiVisibility.PREVIEW, sinceYBAVersion = "2024.2.1")
private boolean success;

@ApiModelProperty(
value =
"WARNING: This is a preview API that could change. "
+ "List of errors that occurred during validation",
accessMode = ApiModelProperty.AccessMode.READ_ONLY)
@YbaApi(visibility = YbaApi.YbaApiVisibility.PREVIEW, sinceYBAVersion = "2024.2.1")
private List<String> errors;

public static PreUpgradeValidationResponse fromError(String error) {
PreUpgradeValidationResponse response = new PreUpgradeValidationResponse();
response.setErrors(Collections.singletonList(error));
return response;
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -100,9 +100,10 @@ public void run() {

private Set<String> doCheckServers(YBClient ybClient, Universe universe, boolean cloudEnabled) {
Set<String> errors = new HashSet<>();
long maxWaitTime = taskParams().isRunOnlyPrechecks() ? 1 : MAX_WAIT_TIME_MS;
doWithConstTimeout(
DELAY_BETWEEN_RETRIES_MS,
MAX_WAIT_TIME_MS,
maxWaitTime,
() -> {
errors.clear();
try {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -2,6 +2,7 @@

package com.yugabyte.yw.commissioner.tasks.subtasks;

import com.google.common.base.Throwables;
import com.yugabyte.yw.commissioner.BaseTaskDependencies;
import com.yugabyte.yw.commissioner.UpgradeTaskBase;
import com.yugabyte.yw.commissioner.tasks.params.ServerSubTaskParams;
Expand All @@ -16,6 +17,7 @@
import com.yugabyte.yw.models.helpers.NodeDetails;
import java.util.ArrayList;
import java.util.Collection;
import java.util.HashSet;
import java.util.List;
import java.util.Map;
import java.util.Set;
Expand All @@ -39,6 +41,7 @@ public class CheckNodesAreSafeToTakeDown extends ServerSubTaskBase {
private static final int MAX_DELAY_MS = 16000;

private static final int MAX_ERRORS_TO_IGNORE = 5;
private static final long MAX_WAIT_FOR_VALIDATION = TimeUnit.SECONDS.toMillis(40);

@Inject
protected CheckNodesAreSafeToTakeDown(BaseTaskDependencies baseTaskDependencies) {
Expand Down Expand Up @@ -143,6 +146,7 @@ && taskParams().fallbackToSingleSplits
}
}
} catch (Exception e) {
Throwables.throwIfUnchecked(e);
throw new RuntimeException(e);
} finally {
executor.shutdownNow();
Expand All @@ -167,11 +171,15 @@ private boolean checkNodes(
long maxAcceptableFollowerLagMs =
confGetter.getConfForScope(universe, UniverseConfKeys.followerLagMaxThreshold).toMillis();

long maxTimeoutMs =
confGetter
.getConfForScope(universe, UniverseConfKeys.nodesAreSafeToTakeDownCheckTimeout)
.toMillis();

long maxTimeoutMs;
if (taskParams().isRunOnlyPrechecks()) {
maxTimeoutMs = MAX_WAIT_FOR_VALIDATION;
} else {
maxTimeoutMs =
confGetter
.getConfForScope(universe, UniverseConfKeys.nodesAreSafeToTakeDownCheckTimeout)
.toMillis();
}
List<CheckBatch> checkBatches =
nodesToCheck.stream()
.map(mnt -> new CheckBatch(universe, mnt, cloudEnabled))
Expand Down Expand Up @@ -217,6 +225,7 @@ private boolean checkNodes(
}
if (!reschedule) {
log.debug("countdown for {}, error {}", checkBatch.nodesStr, checkBatch.errorStr);
checkBatch.finished = true;
countDownLatch.countDown();
}
log.debug("{}/{} checks remaining", countDownLatch.getCount(), checkBatches.size());
Expand All @@ -225,12 +234,21 @@ private boolean checkNodes(
executor.schedule(
runnable, (long) (Math.random() * INITIAL_DELAY_MS), TimeUnit.MILLISECONDS);
}
countDownLatch.await(maxTimeoutMs + TimeUnit.MINUTES.toMillis(1), TimeUnit.MILLISECONDS);
countDownLatch.await(maxTimeoutMs, TimeUnit.MILLISECONDS);
} catch (InterruptedException e) {
log.error("Interrupted", e);
}
if (countDownLatch.getCount() > 0) {
throw new RuntimeException("Timeouted while waiting for checks to complete");
Set<String> pendingIps = new HashSet<>();
checkBatches.stream()
.filter(cb -> !cb.finished && cb.errorStr == null)
.forEach(
cb -> {
pendingIps.addAll(cb.masterIps);
pendingIps.addAll(cb.tserverIps);
});
if (countDownLatch.getCount() > 0 && pendingIps.size() > 0) {
throw new RuntimeException(
"Timed out while waiting for all checks to complete. Pending ips: " + pendingIps);
}
for (CheckBatch checkBatch : checkBatches) {
if (checkBatch.errorStr != null) {
Expand Down Expand Up @@ -267,8 +285,9 @@ private class CheckBatch {
final Set<String> masterIps;
final Set<String> tserverIps;
final String nodesStr;
AtomicInteger iterationNumber = new AtomicInteger();
final AtomicInteger iterationNumber = new AtomicInteger();
volatile String errorStr;
volatile boolean finished;

private CheckBatch(
Universe universe, UpgradeTaskBase.MastersAndTservers target, boolean cloudEnabled) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -15,15 +15,21 @@
import com.yugabyte.yw.commissioner.BaseTaskDependencies;
import com.yugabyte.yw.commissioner.tasks.UniverseTaskBase.ServerType;
import com.yugabyte.yw.commissioner.tasks.params.ServerSubTaskParams;
import com.yugabyte.yw.common.services.config.YbClientConfig;
import com.yugabyte.yw.models.Universe;
import com.yugabyte.yw.models.helpers.NodeDetails;
import java.util.concurrent.TimeUnit;
import javax.inject.Inject;
import lombok.extern.slf4j.Slf4j;
import org.yb.client.YBClient;

@Slf4j
public abstract class ServerSubTaskBase extends AbstractTaskBase {

private static final Long VALIDATE_PRECHECK_ADMIN_TIMEOUT_MS = TimeUnit.SECONDS.toMillis(30);
private static final Long VALIDATE_PRECHECK_SOCKET_READ_TIMEOUT_MS = TimeUnit.SECONDS.toMillis(8);
private static final Long VALIDATE_PRECHECK_OPERATION_TIMEOUT_MS = TimeUnit.SECONDS.toMillis(15);

@Inject
protected ServerSubTaskBase(BaseTaskDependencies baseTaskDependencies) {
super(baseTaskDependencies);
Expand Down Expand Up @@ -67,6 +73,16 @@ public YBClient getClient() {
Universe universe = Universe.getOrBadRequest(taskParams().getUniverseUUID());
String masterAddresses = universe.getMasterAddresses();
String certificate = universe.getCertificateNodetoNode();
if (taskParams().isRunOnlyPrechecks()) {
YbClientConfig ybClientConfig =
new YbClientConfig(
masterAddresses,
certificate,
TimeUnit.SECONDS.toMillis(VALIDATE_PRECHECK_ADMIN_TIMEOUT_MS),
TimeUnit.SECONDS.toMillis(VALIDATE_PRECHECK_SOCKET_READ_TIMEOUT_MS),
TimeUnit.SECONDS.toMillis(VALIDATE_PRECHECK_OPERATION_TIMEOUT_MS));
return ybService.getClientWithConfig(ybClientConfig);
}
return ybService.getClient(masterAddresses, certificate);
}

Expand Down
Loading

0 comments on commit 4cb57c7

Please sign in to comment.