Skip to content

Commit

Permalink
[PLAT-16399]: Add support for PG-15 upgrades on dedicated VM node uni…
Browse files Browse the repository at this point in the history
…verses

Summary:
 - Upgrading PG-11 YB_DB builds to PG-15 requires a catalog upgrade, which involves backing up the old catalog from PG-11 and restoring it to the new PG-15 postgres.

 - During the upgrade, the database spins up a new PG-15 postgres on a master node. If the old postgres exists on the same node, it connects to it to perform the backup and restore.

 - For universes with authentication enabled, the DB uses a local socket file to connect to the old postgres. However, if the PG-11 postgres is running on a different node, a password is required for authentication. Since YBA does not store the yb-db user password, a temporary superuser (yugabyte_upgrade) is created, and its password is stored in .pgPass on the master node. This allows the new postgres to connect to the old postgres on a different machine.

 - The temporary superuser is valid for only 4 hours and is used exclusively for the YSQL major catalog upgrade. It is created before master upgrades, and its password is deleted after the catalog upgrade completes. The user can only be removed after finalization or rollback, as DDLs are blocked during the upgrade.

 - In case of failure, the password file is deleted, except during platform restarts. However, if the user re-triggers the upgrade, the file will be deleted immediately.

 - Currently, upgrades on dedicated node universes are not retriable. This will be addressed in a follow-up task: PLAT-16502.

**Additional Changes:**

 - Modified the sequence of enabling pushdown (gflag upgrade) to occur as the final step during upgrade finalization as advised by the DB team.

Test Plan:
 - Manually tested upgrade, rollback, and finalize-upgrade processes on both auth-enabled and non-auth-enabled dedicated node universes.
 - Verified that passwords are not logged and are deleted after finalizing or rolling back the upgrade.
 - Confirmed that the password file is removed once the upgrade task is completed or fails.

Reviewers: #yba-api-review, sneelakantan, sanketh, anijhawan

Reviewed By: #yba-api-review, sanketh

Subscribers: yugaware

Differential Revision: https://phorge.dev.yugabyte.com/D41478
  • Loading branch information
vipul-yb committed Jan 30, 2025
1 parent 9a11dd2 commit 7acbd98
Show file tree
Hide file tree
Showing 11 changed files with 278 additions and 54 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -239,6 +239,13 @@ public void runUpgrade(Runnable upgradeLambda) {
}

public void runUpgrade(Runnable upgradeLambda, @Nullable Consumer<Universe> firstRunTxnCallback) {
runUpgrade(upgradeLambda, firstRunTxnCallback, null /* onFailureTask */);
}

public void runUpgrade(
Runnable upgradeLambda,
@Nullable Consumer<Universe> firstRunTxnCallback,
Runnable onFailureTask) {
if (maybeRunOnlyPrechecks()) {
return;
}
Expand All @@ -264,6 +271,11 @@ public void runUpgrade(Runnable upgradeLambda, @Nullable Consumer<Universe> firs
getRunnableTask().runSubTasks();
} catch (Throwable t) {
log.error("Error executing task {} with error: ", getName(), t);
if (onFailureTask != null) {
log.info("Running on failure upgrade task");
onFailureTask.run();
log.info("Finished on failure upgrade task");
}

if (taskParams().getUniverseSoftwareUpgradeStateOnFailure() != null) {
Universe universe = getUniverse();
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -66,6 +66,7 @@
import com.yugabyte.yw.commissioner.tasks.subtasks.KubernetesCommandExecutor;
import com.yugabyte.yw.commissioner.tasks.subtasks.LoadBalancerStateChange;
import com.yugabyte.yw.commissioner.tasks.subtasks.ManageAlertDefinitions;
import com.yugabyte.yw.commissioner.tasks.subtasks.ManageCatalogUpgradeSuperUser;
import com.yugabyte.yw.commissioner.tasks.subtasks.ManageLoadBalancerGroup;
import com.yugabyte.yw.commissioner.tasks.subtasks.ManipulateDnsRecordTask;
import com.yugabyte.yw.commissioner.tasks.subtasks.MarkSourceMetric;
Expand Down Expand Up @@ -1718,6 +1719,20 @@ public SubTaskGroup createPGUpgradeTServerCheckTask(String ybSoftwareVersion) {
return subTaskGroup;
}

public SubTaskGroup createManageCatalogUpgradeSuperUserTask(
ManageCatalogUpgradeSuperUser.Action action) {
SubTaskGroup subTaskGroup =
createSubTaskGroup("ManageCatalogUpgradeSuperUser", SubTaskGroupType.ConfigureUniverse);
ManageCatalogUpgradeSuperUser task = createTask(ManageCatalogUpgradeSuperUser.class);
ManageCatalogUpgradeSuperUser.Params params = new ManageCatalogUpgradeSuperUser.Params();
params.setUniverseUUID(taskParams().getUniverseUUID());
params.action = action;
task.initialize(params);
subTaskGroup.addSubTask(task);
getRunnableTask().addSubTaskGroup(subTaskGroup);
return subTaskGroup;
}

public SubTaskGroup createRunYsqlMajorVersionCatalogUpgradeTask() {
SubTaskGroup subTaskGroup =
createSubTaskGroup("RunYsqlMajorVersionCatalogUpgrade", SubTaskGroupType.UpgradingSoftware);
Expand Down Expand Up @@ -6347,12 +6362,6 @@ protected void createFinalizeUpgradeTasks(
}
}

if (ysqlUpgradeFinalizeTask != null) {
// Run YSQL upgrade finalize task on the universe.
// This is a temp step as we need to remove flags set during upgrade.
ysqlUpgradeFinalizeTask.run();
}

// Promote all auto flags upto class External.
createPromoteAutoFlagTask(
universe.getUniverseUUID(),
Expand All @@ -6364,6 +6373,12 @@ protected void createFinalizeUpgradeTasks(
createRunYsqlUpgradeTask(version);
}

if (ysqlUpgradeFinalizeTask != null) {
// Run YSQL upgrade finalize task on the universe.
// This is a temp step as we need to remove flags set during upgrade.
ysqlUpgradeFinalizeTask.run();
}

createUpdateUniverseSoftwareUpgradeStateTask(
UniverseDefinitionTaskParams.SoftwareUpgradeState.Ready,
false /* isSoftwareRollbackAllowed */);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -148,20 +148,6 @@ && taskParams().isMasterInShellMode
}
}

if (universe.getUniverseDetails().isSoftwareRollbackAllowed
&& universe.getUniverseDetails().prevYBSoftwareConfig != null) {
String currentVersion =
universe.getUniverseDetails().getPrimaryCluster().userIntent.ybSoftwareVersion;
String oldVersion = universe.getUniverseDetails().prevYBSoftwareConfig.getSoftwareVersion();
if (gFlagsValidation.ysqlMajorVersionUpgrade(oldVersion, currentVersion)
&& taskParams().ysqlMajorVersionUpgradeState == null) {
log.debug(
"Setting ysqlMajorVersionUpgradeState to Pre_finalize for universe {}",
universe.getUniverseUUID());
taskParams().ysqlMajorVersionUpgradeState = YsqlMajorVersionUpgradeState.PRE_FINALIZE;
}
}

log.debug(
"Reset master state is now {} for universe {}. It was {}",
resetMasterState,
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,125 @@
// Copyright (c) YugaByte, Inc.

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

import com.google.common.collect.ImmutableList;
import com.google.common.collect.ImmutableMap;
import com.google.inject.Inject;
import com.yugabyte.yw.commissioner.BaseTaskDependencies;
import com.yugabyte.yw.commissioner.tasks.UniverseTaskBase;
import com.yugabyte.yw.common.NodeUniverseManager;
import com.yugabyte.yw.common.ShellProcessContext;
import com.yugabyte.yw.common.Util;
import com.yugabyte.yw.common.YsqlQueryExecutor;
import com.yugabyte.yw.forms.UniverseTaskParams;
import com.yugabyte.yw.models.Universe;
import com.yugabyte.yw.models.helpers.NodeDetails;
import lombok.extern.slf4j.Slf4j;
import org.apache.commons.lang3.RandomStringUtils;

@Slf4j
public class ManageCatalogUpgradeSuperUser extends UniverseTaskBase {

private final NodeUniverseManager nodeUniverseManager;
private final YsqlQueryExecutor ysqlQueryExecutor;

private static final String UPGRADE_SUPERUSER = "yugabyte_upgrade";

@Inject
protected ManageCatalogUpgradeSuperUser(
BaseTaskDependencies baseTaskDependencies,
NodeUniverseManager nodeUniverseManager,
YsqlQueryExecutor ysqlQueryExecutor) {
super(baseTaskDependencies);
this.nodeUniverseManager = nodeUniverseManager;
this.ysqlQueryExecutor = ysqlQueryExecutor;
}

public enum Action {
CREATE_USER,
DELETE_USER,
DELETE_PG_PASS_FILE
}

public static class Params extends UniverseTaskParams {
public Action action;
}

public Params taskParams() {
return (Params) taskParams;
}

@Override
public void run() {
Universe universe = getUniverse();
NodeDetails masterLeaderNode = universe.getMasterLeaderNode();
String pgPassFilePath =
Util.getNodeHomeDir(universe.getUniverseUUID(), universe.getMasterLeaderNode())
+ "/.pgpass";
if (taskParams().action == Action.CREATE_USER) {
dropUser(universe, masterLeaderNode, pgPassFilePath);
deletePGPassFile(universe, masterLeaderNode, pgPassFilePath);
createUser(universe, masterLeaderNode, pgPassFilePath);
} else if (taskParams().action == Action.DELETE_USER) {
dropUser(universe, masterLeaderNode, pgPassFilePath);
deletePGPassFile(universe, masterLeaderNode, pgPassFilePath);
} else if (taskParams().action == Action.DELETE_PG_PASS_FILE) {
deletePGPassFile(universe, masterLeaderNode, pgPassFilePath);
}
}

private void dropUser(Universe universe, NodeDetails node, String pgPassFilePath) {
ysqlQueryExecutor.runUserDbCommands(
"DROP USER IF EXISTS " + UPGRADE_SUPERUSER, "template1", universe);
}

private void deletePGPassFile(Universe universe, NodeDetails node, String pgPassFilePath) {
nodeUniverseManager.runCommand(node, universe, ImmutableList.of("rm", "-f", pgPassFilePath));
}

private void createUser(Universe universe, NodeDetails node, String pgPassFilePath) {
String allowedCharsInPassword =
"ABCDEFGHIJKLMNOPQRSTUVWXYZabcdefghijklmnopqrstuvwxyz0123456789";
for (String charString : Util.SPECIAL_CHARACTERS_STRING_LIST) {
// Avoid using % in the password.
if (!charString.equals("%")) {
allowedCharsInPassword += charString;
}
}
String password = RandomStringUtils.secureStrong().next(20, allowedCharsInPassword);
String query =
String.format(
"""
DO $$DECLARE time TIMESTAMP := now() + INTERVAL '4 HOURS';
BEGIN
EXECUTE format('CREATE ROLE %s WITH SUPERUSER LOGIN PASSWORD ''%s'' VALID UNTIL ''%s'';', time);
END$$;
""",
UPGRADE_SUPERUSER, password, "%s");
ysqlQueryExecutor.runUserDbCommands(query, "template1", universe);
String pgPassFileContent = "*:*:*:" + UPGRADE_SUPERUSER + ":" + password;
ShellProcessContext context =
ShellProcessContext.builder()
.logCmdOutput(false)
.redactedVals(
ImmutableMap.of(pgPassFileContent, "*:*:*:REDACTED_USERNAME:REDACTED_PASSWORD"))
.build();
nodeUniverseManager.runCommand(
node,
universe,
ImmutableList.of(
"rm",
"-rf",
pgPassFilePath,
";",
"echo",
pgPassFileContent,
">>",
pgPassFilePath,
";",
"chmod",
"600",
pgPassFilePath),
context);
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -7,6 +7,7 @@
import com.yugabyte.yw.commissioner.ITask.Abortable;
import com.yugabyte.yw.commissioner.ITask.Retryable;
import com.yugabyte.yw.commissioner.UserTaskDetails.SubTaskGroupType;
import com.yugabyte.yw.commissioner.tasks.subtasks.ManageCatalogUpgradeSuperUser.Action;
import com.yugabyte.yw.common.gflags.GFlagsUtil;
import com.yugabyte.yw.forms.RollbackUpgradeParams;
import com.yugabyte.yw.forms.UniverseDefinitionTaskParams;
Expand Down Expand Up @@ -102,11 +103,17 @@ public void run() {
createDownloadTasks(toOrderedSet(nodes.asPair()), newVersion);

boolean ysqlMajorVersionUpgrade = false;
boolean requireAdditionalSuperUserForCatalogUpgrade = false;
if (prevYBSoftwareConfig != null) {
ysqlMajorVersionUpgrade =
gFlagsValidation.ysqlMajorVersionUpgrade(
prevYBSoftwareConfig.getSoftwareVersion(),
universe.getUniverseDetails().getPrimaryCluster().userIntent.ybSoftwareVersion);
requireAdditionalSuperUserForCatalogUpgrade =
isSuperUserRequiredForCatalogUpgrade(
universe,
prevYBSoftwareConfig.getSoftwareVersion(),
universe.getUniverseDetails().getPrimaryCluster().userIntent.ybSoftwareVersion);
}

if (nodes.tserversList.size() > 0) {
Expand Down Expand Up @@ -150,6 +157,10 @@ public void run() {
universe, getNodesToEnableExpressionPushdown(universe), true /* flagValue */);
createServerConfUpdateTaskForYsqlMajorUpgrade(
universe, universe.getTServers(), YsqlMajorVersionUpgradeState.ROLLBACK_COMPLETE);

if (requireAdditionalSuperUserForCatalogUpgrade) {
createManageCatalogUpgradeSuperUserTask(Action.DELETE_USER);
}
}

// Check software version on each node.
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -4,12 +4,13 @@

import com.google.common.collect.ImmutableMap;
import com.yugabyte.yw.commissioner.BaseTaskDependencies;
import com.yugabyte.yw.commissioner.Common.CloudType;
import com.yugabyte.yw.commissioner.TaskExecutor.SubTaskGroup;
import com.yugabyte.yw.commissioner.UpgradeTaskBase;
import com.yugabyte.yw.commissioner.UserTaskDetails.SubTaskGroupType;
import com.yugabyte.yw.commissioner.tasks.XClusterConfigTaskBase;
import com.yugabyte.yw.commissioner.tasks.subtasks.AnsibleConfigureServers;
import com.yugabyte.yw.common.PlatformServiceException;
import com.yugabyte.yw.commissioner.tasks.subtasks.ManageCatalogUpgradeSuperUser.Action;
import com.yugabyte.yw.common.Util;
import com.yugabyte.yw.common.config.UniverseConfKeys;
import com.yugabyte.yw.common.gflags.GFlagsUtil;
Expand Down Expand Up @@ -37,7 +38,6 @@
import javax.inject.Inject;
import lombok.extern.slf4j.Slf4j;
import org.yb.client.YBClient;
import play.mvc.Http.Status;

@Slf4j
public abstract class SoftwareUpgradeTaskBase extends UpgradeTaskBase {
Expand Down Expand Up @@ -346,21 +346,6 @@ protected void createXClusterSourceRootCertDirPathGFlagTasks() {
protected void createPrecheckTasks(Universe universe, String newVersion) {
super.createPrecheckTasks(universe);

boolean ysqlMajorVersionUpgrade =
gFlagsValidation.ysqlMajorVersionUpgrade(
universe.getUniverseDetails().getPrimaryCluster().userIntent.ybSoftwareVersion,
newVersion)
&& universe.getUniverseDetails().getPrimaryCluster().userIntent.enableYSQL;

if (ysqlMajorVersionUpgrade) {

// This is a temp check and it will be removed once we have solution from DB team.
if (universe.getUniverseDetails().getPrimaryCluster().userIntent.dedicatedNodes) {
throw new PlatformServiceException(
Status.INTERNAL_SERVER_ERROR, "Dedicated nodes is not supported with PG15 upgrade");
}
}

MastersAndTservers nodes = fetchNodes(taskParams().upgradeOption);
Set<NodeDetails> allNodes = toOrderedSet(nodes.asPair());

Expand Down Expand Up @@ -548,10 +533,35 @@ protected Cluster findCluster(Universe universe, UUID placementUuid) {
.orElse(null);
}

protected boolean isYsqlMajorVersionUpgrade(
Universe universe, String currentVersion, String newVersion) {
return gFlagsValidation.ysqlMajorVersionUpgrade(currentVersion, newVersion)
&& universe.getUniverseDetails().getPrimaryCluster().userIntent.enableYSQL;
}

protected boolean isSuperUserRequiredForCatalogUpgrade(
Universe universe, String currentVersion, String newVersion) {
UniverseDefinitionTaskParams.Cluster primaryCluster =
universe.getUniverseDetails().getPrimaryCluster();
return isYsqlMajorVersionUpgrade(universe, currentVersion, newVersion)
&& primaryCluster.userIntent.enableYSQLAuth
&& (primaryCluster.userIntent.dedicatedNodes
|| primaryCluster.userIntent.providerType.equals(CloudType.kubernetes));
}

protected Runnable getFinalizeYSQLMajorUpgradeTask(Universe universe) {
return () -> {
createGFlagsUpgradeTaskForYSQLMajorUpgrade(
universe, YsqlMajorVersionUpgradeState.FINALIZE_IN_PROGRESS);

if (universe.getUniverseDetails().prevYBSoftwareConfig != null) {
String currentVersion =
universe.getUniverseDetails().getPrimaryCluster().userIntent.ybSoftwareVersion;
String oldVersion = universe.getUniverseDetails().prevYBSoftwareConfig.getSoftwareVersion();
if (isSuperUserRequiredForCatalogUpgrade(universe, oldVersion, currentVersion)) {
createManageCatalogUpgradeSuperUserTask(Action.DELETE_USER);
}
}
};
}
}
Loading

0 comments on commit 7acbd98

Please sign in to comment.