Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

[FLINK-27551] Update status manually instead of relying on updatecontrol #199

Merged
merged 4 commits into from
May 11, 2022
Merged
Show file tree
Hide file tree
Changes from 3 commits
Commits
File filter

Filter by extension

Filter by extension


Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -18,8 +18,10 @@
package org.apache.flink.kubernetes.operator.controller;

import org.apache.flink.api.common.JobStatus;
import org.apache.flink.api.java.tuple.Tuple2;
import org.apache.flink.kubernetes.operator.config.FlinkConfigManager;
import org.apache.flink.kubernetes.operator.crd.FlinkDeployment;
import org.apache.flink.kubernetes.operator.crd.status.FlinkDeploymentStatus;
import org.apache.flink.kubernetes.operator.crd.status.JobManagerDeploymentStatus;
import org.apache.flink.kubernetes.operator.exception.DeploymentFailedException;
import org.apache.flink.kubernetes.operator.exception.ReconciliationException;
Expand Down Expand Up @@ -49,6 +51,7 @@
import java.util.List;
import java.util.Optional;
import java.util.Set;
import java.util.concurrent.ConcurrentHashMap;
import java.util.stream.Collectors;

/** Controller that runs the main reconcile loop for Flink deployments. */
Expand All @@ -68,6 +71,9 @@ public class FlinkDeploymentController
private final MetricManager<FlinkDeployment> metricManager;
private FlinkControllerConfig<FlinkDeployment> controllerConfig;

private final ConcurrentHashMap<Tuple2<String, String>, FlinkDeploymentStatus> statusCache =
new ConcurrentHashMap<>();

public FlinkDeploymentController(
FlinkConfigManager configManager,
KubernetesClient kubernetesClient,
Expand All @@ -86,26 +92,29 @@ public FlinkDeploymentController(
@Override
public DeleteControl cleanup(FlinkDeployment flinkApp, Context context) {
LOG.info("Deleting FlinkDeployment");
OperatorUtils.updateStatusFromCache(flinkApp, statusCache);
gyfora marked this conversation as resolved.
Show resolved Hide resolved
try {
observerFactory.getOrCreate(flinkApp).observe(flinkApp, context);
} catch (DeploymentFailedException dfe) {
// ignore during cleanup
}
metricManager.onRemove(flinkApp);
statusCache.remove(
Tuple2.of(flinkApp.getMetadata().getNamespace(), flinkApp.getMetadata().getName()));
return reconcilerFactory.getOrCreate(flinkApp).cleanup(flinkApp, context);
}

@Override
public UpdateControl<FlinkDeployment> reconcile(FlinkDeployment flinkApp, Context context) {
LOG.info("Starting reconciliation");
FlinkDeployment originalCopy = ReconciliationUtils.clone(flinkApp);

OperatorUtils.updateStatusFromCache(flinkApp, statusCache);
try {
observerFactory.getOrCreate(flinkApp).observe(flinkApp, context);
if (!validateDeployment(flinkApp)) {
metricManager.onUpdate(flinkApp);
OperatorUtils.patchAndCacheStatus(kubernetesClient, flinkApp, statusCache);
return ReconciliationUtils.toUpdateControl(
configManager.getOperatorConfiguration(), originalCopy, flinkApp, false);
configManager.getOperatorConfiguration(), flinkApp, false);
}
reconcilerFactory.getOrCreate(flinkApp).reconcile(flinkApp, context);
} catch (DeploymentFailedException dfe) {
Expand All @@ -116,8 +125,9 @@ public UpdateControl<FlinkDeployment> reconcile(FlinkDeployment flinkApp, Contex

LOG.info("End of reconciliation");
metricManager.onUpdate(flinkApp);
OperatorUtils.patchAndCacheStatus(kubernetesClient, flinkApp, statusCache);
return ReconciliationUtils.toUpdateControl(
configManager.getOperatorConfiguration(), originalCopy, flinkApp, true);
configManager.getOperatorConfiguration(), flinkApp, true);
}

private void handleDeploymentFailed(FlinkDeployment flinkApp, DeploymentFailedException dfe) {
Expand Down Expand Up @@ -151,16 +161,8 @@ public List<EventSource> prepareEventSources(EventSourceContext<FlinkDeployment>
@Override
public Optional<FlinkDeployment> updateErrorStatus(
FlinkDeployment flinkApp, RetryInfo retryInfo, RuntimeException e) {
LOG.warn(
"Attempt count: {}, last attempt: {}",
retryInfo.getAttemptCount(),
retryInfo.isLastAttempt());

ReconciliationUtils.updateForReconciliationError(
flinkApp,
(e instanceof ReconciliationException) ? e.getCause().toString() : e.toString());
metricManager.onUpdate(flinkApp);
return Optional.of(flinkApp);
return ReconciliationUtils.updateErrorStatus(
kubernetesClient, flinkApp, retryInfo, e, metricManager, statusCache);
}

public void setControllerConfig(FlinkControllerConfig<FlinkDeployment> config) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -17,9 +17,11 @@

package org.apache.flink.kubernetes.operator.controller;

import org.apache.flink.api.java.tuple.Tuple2;
import org.apache.flink.kubernetes.operator.config.FlinkConfigManager;
import org.apache.flink.kubernetes.operator.crd.FlinkDeployment;
import org.apache.flink.kubernetes.operator.crd.FlinkSessionJob;
import org.apache.flink.kubernetes.operator.crd.status.FlinkSessionJobStatus;
import org.apache.flink.kubernetes.operator.exception.ReconciliationException;
import org.apache.flink.kubernetes.operator.metrics.MetricManager;
import org.apache.flink.kubernetes.operator.observer.Observer;
Expand Down Expand Up @@ -54,6 +56,7 @@
import java.util.Map;
import java.util.Optional;
import java.util.Set;
import java.util.concurrent.ConcurrentHashMap;
import java.util.function.Function;
import java.util.stream.Collectors;

Expand All @@ -78,6 +81,9 @@ public class FlinkSessionJobController
private Map<String, SharedIndexInformer<FlinkSessionJob>> informers;
private FlinkControllerConfig<FlinkSessionJob> controllerConfig;

private final ConcurrentHashMap<Tuple2<String, String>, FlinkSessionJobStatus> statusCache =
new ConcurrentHashMap<>();

public FlinkSessionJobController(
FlinkConfigManager configManager,
KubernetesClient kubernetesClient,
Expand All @@ -102,11 +108,13 @@ public void init(FlinkControllerConfig<FlinkSessionJob> config) {
public UpdateControl<FlinkSessionJob> reconcile(
FlinkSessionJob flinkSessionJob, Context context) {
LOG.info("Starting reconciliation");
FlinkSessionJob originalCopy = ReconciliationUtils.clone(flinkSessionJob);
OperatorUtils.updateStatusFromCache(flinkSessionJob, statusCache);
observer.observe(flinkSessionJob, context);
if (!validateSessionJob(flinkSessionJob, context)) {
metricManager.onUpdate(flinkSessionJob);
return ReconciliationUtils.toUpdateControl(flinkSessionJob, flinkSessionJob);
OperatorUtils.patchAndCacheStatus(kubernetesClient, flinkSessionJob, statusCache);
return ReconciliationUtils.toUpdateControl(
configManager.getOperatorConfiguration(), flinkSessionJob, false);
}

try {
Expand All @@ -116,31 +124,27 @@ public UpdateControl<FlinkSessionJob> reconcile(
throw new ReconciliationException(e);
}
metricManager.onUpdate(flinkSessionJob);
return ReconciliationUtils.toUpdateControl(originalCopy, flinkSessionJob)
.rescheduleAfter(
configManager.getOperatorConfiguration().getReconcileInterval().toMillis());
OperatorUtils.patchAndCacheStatus(kubernetesClient, flinkSessionJob, statusCache);
return ReconciliationUtils.toUpdateControl(
configManager.getOperatorConfiguration(), flinkSessionJob, true);
}

@Override
public DeleteControl cleanup(FlinkSessionJob sessionJob, Context context) {
LOG.info("Deleting FlinkSessionJob");
metricManager.onRemove(sessionJob);
statusCache.remove(
Tuple2.of(
sessionJob.getMetadata().getNamespace(),
sessionJob.getMetadata().getName()));
return reconciler.cleanup(sessionJob, context);
}

@Override
public Optional<FlinkSessionJob> updateErrorStatus(
FlinkSessionJob flinkSessionJob, RetryInfo retryInfo, RuntimeException e) {
LOG.warn(
"Attempt count: {}, last attempt: {}",
retryInfo.getAttemptCount(),
retryInfo.isLastAttempt());

ReconciliationUtils.updateForReconciliationError(
flinkSessionJob,
(e instanceof ReconciliationException) ? e.getCause().toString() : e.toString());
metricManager.onUpdate(flinkSessionJob);
return Optional.of(flinkSessionJob);
return ReconciliationUtils.updateErrorStatus(
kubernetesClient, flinkSessionJob, retryInfo, e, metricManager, statusCache);
}

@Override
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -17,6 +17,7 @@

package org.apache.flink.kubernetes.operator.reconciler;

import org.apache.flink.api.java.tuple.Tuple2;
import org.apache.flink.configuration.Configuration;
import org.apache.flink.kubernetes.operator.config.FlinkConfigBuilder;
import org.apache.flink.kubernetes.operator.config.FlinkConfigManager;
Expand All @@ -34,14 +35,19 @@
import org.apache.flink.kubernetes.operator.crd.status.JobManagerDeploymentStatus;
import org.apache.flink.kubernetes.operator.crd.status.ReconciliationState;
import org.apache.flink.kubernetes.operator.crd.status.ReconciliationStatus;
import org.apache.flink.kubernetes.operator.exception.ReconciliationException;
import org.apache.flink.kubernetes.operator.metrics.MetricManager;
import org.apache.flink.kubernetes.operator.utils.FlinkUtils;
import org.apache.flink.kubernetes.operator.utils.OperatorUtils;
import org.apache.flink.util.Preconditions;

import com.fasterxml.jackson.core.JsonProcessingException;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.fasterxml.jackson.databind.node.ObjectNode;
import com.fasterxml.jackson.databind.node.TextNode;
import io.fabric8.kubernetes.client.CustomResource;
import io.fabric8.kubernetes.client.KubernetesClient;
import io.javaoperatorsdk.operator.api.reconciler.RetryInfo;
import io.javaoperatorsdk.operator.api.reconciler.UpdateControl;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
Expand All @@ -50,7 +56,8 @@

import java.time.Duration;
import java.time.Instant;
import java.util.Objects;
import java.util.Optional;
import java.util.concurrent.ConcurrentHashMap;

/** Reconciliation utilities. */
public class ReconciliationUtils {
Expand Down Expand Up @@ -123,37 +130,37 @@ public static <T> T clone(T object) {
}
}

public static <CR extends CustomResource> UpdateControl<CR> toUpdateControl(
CR originalCopy, CR current) {

current.setSpec(originalCopy.getSpec());

var statusChanged = !Objects.equals(originalCopy.getStatus(), current.getStatus());
public static <
SPEC extends AbstractFlinkSpec,
STATUS extends CommonStatus<SPEC>,
R extends CustomResource<SPEC, STATUS>>
UpdateControl<R> toUpdateControl(
FlinkOperatorConfiguration operatorConfiguration,
R current,
boolean reschedule) {

return statusChanged ? UpdateControl.updateStatus(current) : UpdateControl.noUpdate();
}
STATUS status = current.getStatus();

public static UpdateControl<FlinkDeployment> toUpdateControl(
FlinkOperatorConfiguration operatorConfiguration,
FlinkDeployment originalCopy,
FlinkDeployment current,
boolean reschedule) {
UpdateControl<FlinkDeployment> updateControl = toUpdateControl(originalCopy, current);
UpdateControl<R> updateControl = UpdateControl.noUpdate();

if (!reschedule) {
return updateControl;
}

if (isJobUpgradeInProgress(current)) {
if (isJobUpgradeInProgress(current.getSpec(), status)) {
return updateControl.rescheduleAfter(0);
}

Duration rescheduleAfter =
current.getStatus()
.getJobManagerDeploymentStatus()
.rescheduleAfter(current, operatorConfiguration);

return updateControl.rescheduleAfter(rescheduleAfter.toMillis());
if (status instanceof FlinkDeploymentStatus) {
return updateControl.rescheduleAfter(
((FlinkDeploymentStatus) status)
.getJobManagerDeploymentStatus()
.rescheduleAfter((FlinkDeployment) current, operatorConfiguration)
.toMillis());
} else {
return updateControl.rescheduleAfter(
operatorConfiguration.getReconcileInterval().toMillis());
}
}

public static boolean isUpgradeModeChangedToLastStateAndHADisabledPreviously(
Expand Down Expand Up @@ -185,9 +192,8 @@ public static FlinkDeploymentSpec getDeployedSpec(FlinkDeployment deployment) {
}
}

private static boolean isJobUpgradeInProgress(FlinkDeployment current) {
ReconciliationStatus<FlinkDeploymentSpec> reconciliationStatus =
current.getStatus().getReconciliationStatus();
private static boolean isJobUpgradeInProgress(AbstractFlinkSpec spec, CommonStatus<?> status) {
var reconciliationStatus = status.getReconciliationStatus();

if (reconciliationStatus == null) {
return false;
Expand All @@ -197,13 +203,12 @@ private static boolean isJobUpgradeInProgress(FlinkDeployment current) {
return true;
}

if (current.getSpec().getJob() == null
|| reconciliationStatus.getLastReconciledSpec() == null) {
if (spec.getJob() == null || reconciliationStatus.getLastReconciledSpec() == null) {
return false;
}

return current.getSpec().getJob().getState() == JobState.RUNNING
&& current.getStatus().getError() == null
return spec.getJob().getState() == JobState.RUNNING
&& status.getError() == null
&& reconciliationStatus.deserializeLastReconciledSpec().getJob().getState()
== JobState.SUSPENDED;
}
Expand Down Expand Up @@ -349,4 +354,28 @@ boolean applyValidationErrorAndResetSpec(CR deployment, String validationError)
return true;
}
}

public static <
SPEC extends AbstractFlinkSpec,
STATUS extends CommonStatus<SPEC>,
R extends CustomResource<SPEC, STATUS>>
Optional<R> updateErrorStatus(
KubernetesClient client,
R resource,
RetryInfo retryInfo,
RuntimeException e,
MetricManager<R> metricManager,
ConcurrentHashMap<Tuple2<String, String>, STATUS> statusCache) {
LOG.warn(
"Attempt count: {}, last attempt: {}",
retryInfo.getAttemptCount(),
retryInfo.isLastAttempt());
OperatorUtils.updateStatusFromCache(resource, statusCache);
ReconciliationUtils.updateForReconciliationError(
resource,
(e instanceof ReconciliationException) ? e.getCause().toString() : e.toString());
metricManager.onUpdate(resource);
OperatorUtils.patchAndCacheStatus(client, resource, statusCache);
return Optional.empty();
}
}
Loading