Skip to content

Commit

Permalink
[PLAT-14898] Record reason for failing to enable node agent
Browse files Browse the repository at this point in the history
Summary: This change saves error to the node agent table. The TaskError has been moved to YBAError to have only one type for YBA.

Test Plan:
1. Make node agent fail by commenting out the selinux policy fix.
2. Check the error on createU.
3. Uncomment the selinux policy fix.
4. retry the task. Verified it succeeded.
5. Verify old tasks have the same errors.

{F287709}

Reviewers: amalyshev, sanketh, cwang

Reviewed By: amalyshev

Subscribers: yugaware

Differential Revision: https://phorge.dev.yugabyte.com/D38222
  • Loading branch information
nkhogen committed Sep 23, 2024
1 parent b062c44 commit d3f0da5
Show file tree
Hide file tree
Showing 27 changed files with 366 additions and 144 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -48,6 +48,7 @@
import java.util.function.Predicate;
import java.util.function.Supplier;
import java.util.stream.Collectors;
import javax.annotation.Nullable;
import javax.inject.Inject;
import javax.inject.Singleton;
import lombok.Getter;
Expand Down Expand Up @@ -137,7 +138,7 @@ void markUniverses() {
return ips;
};
c.getUniverses().stream()
.filter(u -> !u.getUniverseDetails().disableNodeAgent)
.filter(u -> !u.getUniverseDetails().installNodeAgent)
.filter(
u -> {
Optional<Boolean> optional =
Expand Down Expand Up @@ -171,17 +172,77 @@ void enable() {
}

/**
* Checks if node agent is enabled for the universe during universe tasks like universe creation.
* For onprem provider, the provider flag is included in the check to detect old provider.
* Checks if the universe should be marked for pending node agent installation. It returns true
* for all the eligible universes even if the background installation may not happen because it is
* not supported. This is for audit and future changes.
*
* @param universe the given universe.
* @return empty for non-supported universes, true if node agent is enabled else false.
* @return true if it should be marked, else false.
*/
public Optional<Boolean> isNodeAgentEnabled(Universe universe) {
// Provider flag is always included for onprem irrespective of the enabler state to be lenient
// on new node addition. It is up to the caller to further check the universe field to verify if
// the node agent client is immediately available.
return isNodeAgentEnabled(universe, p -> p.getCloudCode() == CloudType.onprem || !isEnabled());
public boolean shouldMarkUniverse(Universe universe) {
return isEnabled() && isNodeAgentEnabled(universe, p -> true).orElse(false) == false;
}

/**
* Checks if node agent client is enabled for the provider and the universe if it is non-null.
*
* @param provider the given provider.
* @param universe the given universe.
* @return true if the client is enabled.
*/
public boolean isNodeAgentClientEnabled(Provider provider, @Nullable Universe universe) {
boolean clientEnabled =
confGetter.getConfForScope(provider, ProviderConfKeys.enableNodeAgentClient);
if (!clientEnabled) {
log.debug("Node agent client is disabled for provider {}", provider.getUuid());
return false;
}
if (!isEnabled()) {
log.debug("Node agent client is disabled for old provider {}", provider.getUuid());
return provider.getDetails().isEnableNodeAgent();
}
if (universe != null) {
// For client, the internal provider flag is not checked if enabler is enabled.
if (isNodeAgentEnabled(universe, p -> !isEnabled()).orElse(false) == false) {
return false;
}
if (universe.getUniverseDetails().installNodeAgent) {
// Mixed mode is allowed.
log.debug(
"Node agent is not available on all nodes for universe {}", universe.getUniverseUUID());
if (!confGetter.getConfForScope(universe, UniverseConfKeys.allowNodeAgentClientMixMode)) {
return false;
}
}
}
return true;
}

/*
* Checks if background installation for node agents is enabled for the given universe. It is
* disabled if node agent client is currently disabled. As node agent is enabled for all new
* providers by default, background installation is enabled unless it is explicitly disabled.
* For old providers, its support depends on the provider type.
*
* 1. Cloud service providers - supported if the client runtime config is not disabled.
* 2. Onprem fully manual providers - supported if the client runtime config is not disabled.
* 3. Onprem non-manual providers - not supported.
*
* For 1 and 2, provider flag must not be checked.
*/
private boolean isBackgroundInstallNodeAgentEnabled(Universe universe) {
return isNodeAgentEnabled(
universe,
p -> {
if (p.getCloudCode() != CloudType.onprem || p.getDetails().isSkipProvisioning()) {
// Do not include provider flag for cloud and fully manual onprem providers when the
// enabler is on.
return !isEnabled();
}
// Onprem non-manual provider for which the provider flag is also checked.
return false;
})
.orElse(false);
}

// This checks if node agent is enabled for the universe with the optional parameter to include or
Expand Down Expand Up @@ -222,7 +283,7 @@ private Optional<Boolean> isNodeAgentEnabled(
}

/**
* Mark universe to disable node agent only if the node agent enabler is enabled.
* Mark universe to install node agent only if the node agent enabler is enabled.
*
* @param universeUuid the given universe UUID.
*/
Expand All @@ -233,10 +294,10 @@ public void markUniverse(UUID universeUuid) {
null /* version increment CB */,
u -> {
UniverseDefinitionTaskParams d = u.getUniverseDetails();
d.disableNodeAgent = true;
d.installNodeAgent = true;
u.setUniverseDetails(d);
});
log.debug("Marked universe {} to disable node agent", universeUuid);
log.debug("Marked universe {} to install node agent", universeUuid);
}
}

Expand Down Expand Up @@ -404,7 +465,13 @@ void waitFor(Duration timeout) throws TimeoutException, InterruptedException {
*/
public boolean shouldInstallNodeAgents(Universe universe, boolean ignoreUniverseLock) {
UniverseDefinitionTaskParams details = universe.getUniverseDetails();
if (!details.disableNodeAgent) {
if (!isEnabled()) {
log.trace(
"Skipping installation for universe {} as enabler is disabled",
universe.getUniverseUUID());
return false;
}
if (!details.installNodeAgent) {
log.trace(
"Skipping installation for universe {} as marker is not set", universe.getUniverseUUID());
// No marker set to install node-agent.
Expand Down Expand Up @@ -435,8 +502,7 @@ public boolean shouldInstallNodeAgents(Universe universe, boolean ignoreUniverse
universe.getUniverseUUID());
return false;
}
Optional<Boolean> optional = isNodeAgentEnabled(universe, p -> !isEnabled());
return optional.isPresent() && optional.get();
return isBackgroundInstallNodeAgentEnabled(universe);
}

/**
Expand All @@ -455,7 +521,7 @@ boolean reinstall(
UUID customerUuid, UUID universeUuid, NodeDetails nodeDetails, NodeAgent nodeAgent)
throws Exception;

/** Set disableNodeAgent property in the universe details to false by locking the universe. */
/** Set installNodeAgent property in the universe details to false by locking the universe. */
boolean migrate(UUID customerUuid, UUID universeUuid) throws Exception;
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -18,6 +18,7 @@
import com.yugabyte.yw.common.config.GlobalConfKeys;
import com.yugabyte.yw.common.config.RuntimeConfGetter;
import com.yugabyte.yw.controllers.handlers.NodeAgentHandler;
import com.yugabyte.yw.models.Customer;
import com.yugabyte.yw.models.HighAvailabilityConfig;
import com.yugabyte.yw.models.NodeAgent;
import com.yugabyte.yw.models.NodeAgent.State;
Expand Down Expand Up @@ -234,16 +235,26 @@ private void poll(NodeAgent nodeAgent) {
Instant expiryDate =
Instant.now().minus(param.getLifetime().toMinutes(), ChronoUnit.MINUTES);
if (expiryDate.isAfter(nodeAgent.getUpdatedAt().toInstant())) {
// Purge the node agent record and its certs.
Customer customer = Customer.getOrBadRequest(nodeAgent.getCustomerUuid());
Set<String> nodeIps =
NodeInstance.getAll().stream()
NodeInstance.listByCustomer(customer.getUuid()).stream()
.map(node -> node.getDetails().ip)
.collect(Collectors.toSet());
customer.getUniverses().stream()
.flatMap(u -> u.getNodes().stream())
.filter(
n ->
n.cloudInfo != null
&& n.cloudInfo.private_ip != null
&& !nodeIps.contains(n.cloudInfo.private_ip))
.map(n -> n.cloudInfo.private_ip)
.forEach(nodeIps::add);
if (!nodeIps.contains(nodeAgent.getIp())) {
log.info(
"Purging node agent {} because connection failed. Error: {}",
nodeAgent.getUuid(),
e.getMessage());
// Purge the node agent record and its certs.
nodeAgentManager.purge(nodeAgent);
}
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -35,9 +35,9 @@
import com.yugabyte.yw.models.TaskInfo.State;
import com.yugabyte.yw.models.helpers.CommonUtils;
import com.yugabyte.yw.models.helpers.KnownAlertLabels;
import com.yugabyte.yw.models.helpers.TaskDetails.TaskError;
import com.yugabyte.yw.models.helpers.TaskDetails.TaskErrorCode;
import com.yugabyte.yw.models.helpers.TaskType;
import com.yugabyte.yw.models.helpers.YBAError;
import com.yugabyte.yw.models.helpers.YBAError.Code;
import io.prometheus.client.CollectorRegistry;
import io.prometheus.client.Summary;
import java.time.Duration;
Expand Down Expand Up @@ -1047,16 +1047,14 @@ synchronized void updateTaskDetailsOnError(TaskInfo.State state, Throwable t) {
TaskInfo.ERROR_STATES.contains(state),
"Task state must be one of " + TaskInfo.ERROR_STATES);
taskInfo.refresh();
TaskError taskError = new TaskError();
YBAError taskError = null;
// Method getRedactedParams does not modify the input as it makes a deep-copy.
String redactedTaskParams = taskInfo.getRedactedParams().toString();
if (state == TaskInfo.State.Aborted && isShutdown.get()) {
taskError.setCode(TaskErrorCode.PLATFORM_SHUTDOWN);
taskError.setMessage("Platform shutdown");
taskError = new YBAError(Code.PLATFORM_SHUTDOWN, "Platform shutdown");
} else if (t instanceof TaskExecutionException) {
TaskExecutionException e = (TaskExecutionException) t;
taskError.setCode(e.getCode());
taskError.setMessage(e.getMessage());
taskError = new YBAError(e.getCode(), e.getMessage());
} else {
Throwable cause = t;
// If an exception is eaten up by just wrapping the cause as RuntimeException(e),
Expand All @@ -1069,7 +1067,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.setMessage(errorString);
taskError = new YBAError(Code.INTERNAL_ERROR, errorString);
}
log.error(
"Failed to execute task type {} UUID {} details {}, hit error.",
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -59,7 +59,7 @@ public void run() {
createUpdateUniverseFieldsTask(
u -> {
UniverseDefinitionTaskParams universeDetails = u.getUniverseDetails();
universeDetails.disableNodeAgent = false;
universeDetails.installNodeAgent = false;
});
createMarkUniverseUpdateSuccessTasks(universe.getUniverseUUID());
getRunnableTask().runSubTasks();
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -2065,11 +2065,13 @@ public SubTaskGroup createInstallNodeAgentTasks(
}
int serverPort = confGetter.getGlobalConf(GlobalConfKeys.nodeAgentServerPort);
Universe universe = getUniverse();
if (!getInstanceOf(NodeAgentClient.class).isClientEnabled(universe)) {
NodeAgentEnabler nodeAgentEnabler = getInstanceOf(NodeAgentEnabler.class);
if (nodeAgentEnabler.shouldMarkUniverse(universe)) {
// Mark the universe.
log.info(
"Skipping node agent installation for universe {} as client is not enabled",
"Skipping node agent installation for universe {} as it is not enabled",
universe.getUniverseUUID());
getInstanceOf(NodeAgentEnabler.class).markUniverse(universe.getUniverseUUID());
nodeAgentEnabler.markUniverse(universe.getUniverseUUID());
return subTaskGroup;
}
Customer customer = Customer.get(universe.getCustomerId());
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -17,6 +17,8 @@
import com.yugabyte.yw.models.NodeAgent.State;
import com.yugabyte.yw.models.Universe;
import com.yugabyte.yw.models.helpers.NodeDetails;
import com.yugabyte.yw.models.helpers.YBAError;
import com.yugabyte.yw.models.helpers.YBAError.Code;
import java.nio.file.Path;
import java.nio.file.Paths;
import java.util.Arrays;
Expand Down Expand Up @@ -177,8 +179,27 @@ public NodeAgent install() {
}
command = ImmutableList.of("sudo", "-H", "/bin/bash", "-c", sb.toString());
log.debug("Running node agent installation command: {}", command);
nodeUniverseManager.runCommand(node, universe, command, shellContext).processErrors();
try {
nodeUniverseManager
.runCommand(node, universe, command, shellContext)
.processErrors("Installation failed");
} catch (RuntimeException e) {
nodeAgent.updateLastError(new YBAError(Code.INSTALLATION_ERROR, e.getMessage()));
throw e;
}
nodeAgent.saveState(State.REGISTERED);
log.debug("Waiting for node agent service to be running");
command =
ImmutableList.of(
"sudo", "-H", "/bin/bash", "-c", "systemctl is-active --quiet yb-node-agent");
try {
nodeUniverseManager
.runCommand(node, universe, command, shellContext)
.processErrors("Service startup failed");
} catch (RuntimeException e) {
nodeAgent.updateLastError(new YBAError(Code.SERVICE_START_ERROR, e.getMessage()));
throw e;
}
return nodeAgent;
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -16,7 +16,6 @@
import com.yugabyte.yw.commissioner.tasks.MultiTableBackup;
import com.yugabyte.yw.commissioner.tasks.ReadOnlyKubernetesClusterDelete;
import com.yugabyte.yw.commissioner.tasks.RebootNodeInUniverse;
import com.yugabyte.yw.commissioner.tasks.ReprovisionNode;
import com.yugabyte.yw.commissioner.tasks.params.IProviderTaskParams;
import com.yugabyte.yw.commissioner.tasks.params.NodeTaskParams;
import com.yugabyte.yw.common.YsqlQueryExecutor.ConsistencyInfoResp;
Expand Down Expand Up @@ -58,9 +57,9 @@
import com.yugabyte.yw.models.Universe;
import com.yugabyte.yw.models.Users;
import com.yugabyte.yw.models.helpers.CommonUtils;
import com.yugabyte.yw.models.helpers.TaskDetails.TaskError;
import com.yugabyte.yw.models.helpers.TaskDetails.TaskErrorCode;
import com.yugabyte.yw.models.helpers.TaskType;
import com.yugabyte.yw.models.helpers.YBAError;
import com.yugabyte.yw.models.helpers.YBAError.Code;
import io.ebean.DB;
import java.util.ArrayList;
import java.util.Arrays;
Expand Down Expand Up @@ -113,11 +112,9 @@ public CustomerTaskManager(
// Invoked if the task is in incomplete state.
private void setTaskError(TaskInfo taskInfo) {
taskInfo.setTaskState(TaskInfo.State.Failure);
TaskError taskError = taskInfo.getTaskError();
YBAError taskError = taskInfo.getTaskError();
if (taskError == null) {
taskError = new TaskError();
taskError.setCode(TaskErrorCode.PLATFORM_RESTARTED);
taskError.setMessage("Platform restarted.");
taskError = new YBAError(Code.PLATFORM_RESTARTED, "Platform restarted.");
taskInfo.setTaskError(taskError);
}
}
Expand Down
33 changes: 4 additions & 29 deletions managed/src/main/java/com/yugabyte/yw/common/NodeAgentClient.java
Original file line number Diff line number Diff line change
Expand Up @@ -22,6 +22,7 @@
import com.yugabyte.yw.models.NodeAgent.State;
import com.yugabyte.yw.models.Provider;
import com.yugabyte.yw.models.Universe;
import com.yugabyte.yw.models.helpers.YBAError;
import com.yugabyte.yw.nodeagent.NodeAgentGrpc;
import com.yugabyte.yw.nodeagent.NodeAgentGrpc.NodeAgentBlockingStub;
import com.yugabyte.yw.nodeagent.NodeAgentGrpc.NodeAgentStub;
Expand Down Expand Up @@ -511,34 +512,7 @@ public Optional<NodeAgent> maybeGetNodeAgent(

/* Passing universe allows more specific check for the universe. */
public boolean isClientEnabled(Provider provider, @Nullable Universe universe) {
boolean clientEnabled =
confGetter.getConfForScope(provider, ProviderConfKeys.enableNodeAgentClient);
if (!clientEnabled) {
log.debug("Node agent client is disabled for provider {}", provider.getUuid());
return false;
}
if (!nodeAgentEnablerProvider.get().isEnabled()) {
log.debug("Node agent client is disabled for old provider {}", provider.getUuid());
return provider.getDetails().isEnableNodeAgent();
}
return universe == null || isClientEnabled(universe);
}

public boolean isClientEnabled(Universe universe) {
// Check this first before the universe field to allow the change to be reflected immediately.
Optional<Boolean> optional = nodeAgentEnablerProvider.get().isNodeAgentEnabled(universe);
if (optional.isPresent() && !optional.get()) {
log.debug("Node agent client is disabled for universe {}", universe.getUniverseUUID());
return false;
}
if (nodeAgentEnablerProvider.get().isEnabled()
&& universe.getUniverseDetails().disableNodeAgent) {
log.debug(
"Node agent client is disabled for universe {} pending background installation",
universe.getUniverseUUID());
return false;
}
return true;
return nodeAgentEnablerProvider.get().isNodeAgentClientEnabled(provider, universe);
}

public boolean isAnsibleOffloadingEnabled(
Expand Down Expand Up @@ -599,9 +573,10 @@ public PingResponse waitForServerReady(NodeAgent nodeAgent, Duration timeout) {
while (true) {
try {
PingResponse response = ping(nodeAgent);
nodeAgent.updateOffloadable(response.getServerInfo().getOffloadable());
nodeAgent.updateServerInfo(response.getServerInfo());
return response;
} catch (StatusRuntimeException e) {
nodeAgent.updateLastError(new YBAError(YBAError.Code.CONNECTION_ERROR, e.getMessage()));
if (e.getStatus().getCode() != Code.UNAVAILABLE
&& e.getStatus().getCode() != Code.DEADLINE_EXCEEDED) {
log.error("Error in connecting to Node agent {} - {}", nodeAgent.getIp(), e.getStatus());
Expand Down
Loading

0 comments on commit d3f0da5

Please sign in to comment.