-
Notifications
You must be signed in to change notification settings - Fork 501
[FLINK-37730][Job Manager] Expose JM exception as K8s exceptions #978
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
Changes from 7 commits
2ee8c55
22b84b9
7d011c3
c434722
3c3a1b5
6fdbf84
61ad2cd
f1e9320
4722465
5104e13
8f1cab7
1b5e654
7414fc4
8c45e14
0f64ad4
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -19,6 +19,7 @@ | |
|
|
||
| import org.apache.flink.api.common.JobID; | ||
| import org.apache.flink.api.common.JobStatus; | ||
| import org.apache.flink.autoscaler.utils.DateTimeUtils; | ||
| import org.apache.flink.kubernetes.operator.api.AbstractFlinkResource; | ||
| import org.apache.flink.kubernetes.operator.api.FlinkSessionJob; | ||
| import org.apache.flink.kubernetes.operator.api.spec.JobState; | ||
|
|
@@ -29,10 +30,15 @@ | |
| import org.apache.flink.kubernetes.operator.utils.EventRecorder; | ||
| import org.apache.flink.kubernetes.operator.utils.ExceptionUtils; | ||
| import org.apache.flink.runtime.client.JobStatusMessage; | ||
| import org.apache.flink.runtime.rest.messages.JobExceptionsInfoWithHistory; | ||
|
|
||
| import org.slf4j.Logger; | ||
| import org.slf4j.LoggerFactory; | ||
|
|
||
| import java.time.Instant; | ||
| import java.time.ZoneId; | ||
| import java.util.HashMap; | ||
| import java.util.Map; | ||
| import java.util.concurrent.TimeoutException; | ||
|
|
||
| import static org.apache.flink.kubernetes.operator.utils.FlinkResourceExceptionUtils.updateFlinkResourceException; | ||
|
|
@@ -69,17 +75,23 @@ public boolean observe(FlinkResourceContext<R> ctx) { | |
| var jobStatus = resource.getStatus().getJobStatus(); | ||
| LOG.debug("Observing job status"); | ||
| var previousJobStatus = jobStatus.getState(); | ||
|
|
||
| var jobId = jobStatus.getJobId(); | ||
| try { | ||
| var newJobStatusOpt = | ||
| ctx.getFlinkService() | ||
| .getJobStatus( | ||
| ctx.getObserveConfig(), | ||
| JobID.fromHexString(jobStatus.getJobId())); | ||
| .getJobStatus(ctx.getObserveConfig(), JobID.fromHexString(jobId)); | ||
|
|
||
| if (newJobStatusOpt.isPresent()) { | ||
| updateJobStatus(ctx, newJobStatusOpt.get()); | ||
| var newJobStatus = newJobStatusOpt.get(); | ||
| updateJobStatus(ctx, newJobStatus); | ||
| ReconciliationUtils.checkAndUpdateStableSpec(resource.getStatus()); | ||
| // now check if the job is in a terminal state. This might not be need as we have | ||
| // already | ||
| // verified that the REST API server is available | ||
| // now check if the job is NOT in a terminal state | ||
| if (!newJobStatus.getJobState().isGloballyTerminalState()) { | ||
| observeJobManagerExceptions(ctx); | ||
| } | ||
| return true; | ||
| } else { | ||
| onTargetJobNotFound(ctx); | ||
|
|
@@ -95,6 +107,143 @@ public boolean observe(FlinkResourceContext<R> ctx) { | |
| return false; | ||
| } | ||
|
|
||
| /** | ||
| * Observe the exceptions raised in the job manager and take appropriate action. | ||
| * | ||
| * @param ctx the context with which the operation is executed | ||
| */ | ||
| protected void observeJobManagerExceptions(FlinkResourceContext<R> ctx) { | ||
| var resource = ctx.getResource(); | ||
| var operatorConfig = ctx.getOperatorConfig(); | ||
| var jobStatus = resource.getStatus().getJobStatus(); | ||
|
|
||
| try { | ||
| var jobId = JobID.fromHexString(jobStatus.getJobId()); | ||
| // TODO: Ideally the best way to restrict the number of events is to use the query param | ||
| // `maxExceptions` | ||
| // but the JobExceptionsMessageParameters does not expose the parameters and nor does | ||
| // it have setters. | ||
| var history = | ||
| ctx.getFlinkService() | ||
| .getJobExceptions( | ||
| resource, jobId, ctx.getDeployConfig(resource.getSpec())); | ||
vsantwana marked this conversation as resolved.
Outdated
Show resolved
Hide resolved
|
||
|
|
||
| if (history == null || history.getExceptionHistory() == null) { | ||
| return; | ||
| } | ||
|
|
||
| var exceptionHistory = history.getExceptionHistory(); | ||
| var exceptions = exceptionHistory.getEntries(); | ||
| if (exceptions.isEmpty()) { | ||
| return; | ||
| } | ||
|
|
||
| if (exceptionHistory.isTruncated()) { | ||
| LOG.warn( | ||
| "Job exception history is truncated for jobId '{}'. Some exceptions may be missing.", | ||
| jobId); | ||
| } | ||
|
|
||
| String currentJobId = jobStatus.getJobId(); | ||
| Instant lastRecorded = null; // first reconciliation | ||
|
|
||
| var cacheEntry = ctx.getExceptionCacheEntry(); | ||
| // a cache entry is created should always be present. The timestamp for the first | ||
| // reconciliation would be | ||
| // when the job was created. This check is still necessary because even though there | ||
| // might be an entry, | ||
| // the jobId could have changed since the job was first created. | ||
| if (cacheEntry.getJobId().equals(currentJobId)) { | ||
| lastRecorded = Instant.ofEpochMilli(cacheEntry.getLastTimestamp()); | ||
| } | ||
|
|
||
| Instant now = Instant.now(); | ||
| int maxEvents = operatorConfig.getReportedExceptionEventsMaxCount(); | ||
| int maxStackTraceLines = operatorConfig.getReportedExceptionEventsMaxStackTraceLength(); | ||
|
|
||
| int count = 0; | ||
| for (var exception : exceptions) { | ||
| Instant exceptionTime = Instant.ofEpochMilli(exception.getTimestamp()); | ||
| if (lastRecorded != null && exceptionTime.isBefore(lastRecorded)) { | ||
| continue; | ||
vsantwana marked this conversation as resolved.
Outdated
Show resolved
Hide resolved
|
||
| } | ||
|
|
||
| emitJobManagerExceptionEvent(ctx, exception, exceptionTime, maxStackTraceLines); | ||
| if (++count >= maxEvents) { | ||
| break; | ||
| } | ||
| } | ||
vsantwana marked this conversation as resolved.
Outdated
Show resolved
Hide resolved
|
||
| ctx.getExceptionCacheEntry().setJobId(currentJobId); | ||
| ctx.getExceptionCacheEntry().setLastTimestamp(now.toEpochMilli()); | ||
|
||
| } catch (Exception e) { | ||
| LOG.warn("Failed to fetch JobManager exception info.", e); | ||
| } | ||
| } | ||
|
|
||
| private void emitJobManagerExceptionEvent( | ||
| FlinkResourceContext<R> ctx, | ||
| JobExceptionsInfoWithHistory.RootExceptionInfo exception, | ||
| Instant exceptionTime, | ||
| int maxStackTraceLines) { | ||
|
|
||
| String exceptionName = exception.getExceptionName(); | ||
| if (exceptionName == null || exceptionName.isBlank()) { | ||
| return; | ||
| } | ||
|
|
||
| Map<String, String> annotations = new HashMap<>(); | ||
| annotations.put( | ||
| "event-time-readable", | ||
| DateTimeUtils.readable(exceptionTime, ZoneId.systemDefault())); | ||
| annotations.put("event-timestamp-millis", String.valueOf(exceptionTime.toEpochMilli())); | ||
vsantwana marked this conversation as resolved.
Outdated
Show resolved
Hide resolved
|
||
|
|
||
| if (exception.getTaskName() != null) { | ||
| annotations.put("task-name", exception.getTaskName()); | ||
| } | ||
| if (exception.getEndpoint() != null) { | ||
| annotations.put("endpoint", exception.getEndpoint()); | ||
| } | ||
| if (exception.getTaskManagerId() != null) { | ||
| annotations.put("tm-id", exception.getTaskManagerId()); | ||
| } | ||
|
|
||
| if (exception.getFailureLabels() != null) { | ||
| exception | ||
| .getFailureLabels() | ||
| .forEach((k, v) -> annotations.put("failure-label-" + k, v)); | ||
| } | ||
|
|
||
| StringBuilder eventMessage = new StringBuilder(exceptionName); | ||
| String stacktrace = exception.getStacktrace(); | ||
| if (stacktrace != null && !stacktrace.isBlank()) { | ||
| String[] lines = stacktrace.split("\n"); | ||
| eventMessage.append("\n\nStacktrace (truncated):\n"); | ||
|
||
| for (int i = 0; i < Math.min(maxStackTraceLines, lines.length); i++) { | ||
| eventMessage.append(lines[i]).append("\n"); | ||
| } | ||
| if (lines.length > maxStackTraceLines) { | ||
| eventMessage | ||
| .append("... (") | ||
| .append(lines.length - maxStackTraceLines) | ||
| .append(" more lines)"); | ||
| } | ||
| } | ||
|
|
||
| String keyMessage = | ||
| exceptionName.length() > 128 ? exceptionName.substring(0, 128) : exceptionName; | ||
|
|
||
| eventRecorder.triggerEventOnceWithAnnotations( | ||
vsantwana marked this conversation as resolved.
Outdated
Show resolved
Hide resolved
|
||
| exceptionTime.toEpochMilli(), | ||
| ctx.getResource(), | ||
| EventRecorder.Type.Warning, | ||
| EventRecorder.Reason.JobException, | ||
| eventMessage.toString().trim(), | ||
| EventRecorder.Component.JobManagerDeployment, | ||
| "jobmanager-exception-" + keyMessage.hashCode(), | ||
| ctx.getKubernetesClient(), | ||
| annotations); | ||
gyfora marked this conversation as resolved.
Outdated
Show resolved
Hide resolved
|
||
| } | ||
|
|
||
| /** | ||
| * Callback when no matching target job was found on a cluster where jobs were found. | ||
| * | ||
|
|
||

Uh oh!
There was an error while loading. Please reload this page.