gyfora commented on code in PR #978:
URL: 
https://github.com/apache/flink-kubernetes-operator/pull/978#discussion_r2097105275


##########
flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/observer/JobStatusObserver.java:
##########
@@ -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);
+                }

Review Comment:
   Maybe a better logic would be to check that old job status is not in a 
terminal state. 
   With the current logic if the job fails with a fatal error (goes into 
terminal FAILED state) the exception(s) wouldn't be reported if I understand 
correctly.



##########
flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/observer/JobStatusObserver.java:
##########
@@ -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()));

Review Comment:
   We should simply use `ctx.getObserveConfig()` here that's what we use for 
accessing the currently running cluster (to observe it)



##########
flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/service/AbstractFlinkService.java:
##########
@@ -845,6 +849,37 @@ public RestClusterClient<String> 
getClusterClient(Configuration conf) throws Exc
                 (c, e) -> new StandaloneClientHAServices(restServerAddress));
     }
 
+    @Override
+    public JobExceptionsInfoWithHistory getJobExceptions(
+            AbstractFlinkResource resource, JobID jobId, Configuration 
deployConfig) {

Review Comment:
   Should rename the config to observeConfig as that what's should be passed I 
noted in another comment



##########
flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/observer/JobStatusObserver.java:
##########
@@ -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()));
+
+            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;
+                }
+
+                emitJobManagerExceptionEvent(ctx, exception, exceptionTime, 
maxStackTraceLines);
+                if (++count >= maxEvents) {
+                    break;
+                }
+            }
+            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()));
+
+        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(
+                exceptionTime.toEpochMilli(),
+                ctx.getResource(),
+                EventRecorder.Type.Warning,
+                EventRecorder.Reason.JobException,
+                eventMessage.toString().trim(),
+                EventRecorder.Component.JobManagerDeployment,
+                "jobmanager-exception-" + keyMessage.hashCode(),
+                ctx.getKubernetesClient(),
+                annotations);

Review Comment:
   It probably makes sense to trim the values of the annotations map here or in 
the `eventRecorder.triggerEventOnceWithAnnotations` as there is a 63 character 
limit and some character limitations on the annotations in Kubernetes. Maybe AI 
can help you create a filter/formatter :)



##########
flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/service/FlinkResourceContextFactory.java:
##########
@@ -52,6 +54,23 @@ public class FlinkResourceContextFactory {
 
     private static final Logger LOG = 
LoggerFactory.getLogger(FlinkResourceContextFactory.class);
 
+    /** The cache entry for the last recorded exception timestamp for a JobID. 
*/
+    @Getter
+    @Setter

Review Comment:
   Could be simply `@Data`



##########
flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/observer/JobStatusObserverTest.java:
##########
@@ -147,6 +157,128 @@ void testFailed() throws Exception {
         assertTrue(flinkResourceEventCollector.events.isEmpty());
     }
 
+    @Test
+    public void testExceptionLimitConfig() throws Exception {
+        var observer = new JobStatusObserver<>(eventRecorder);
+        var deployment = initDeployment();
+        var status = deployment.getStatus();
+        var jobStatus = status.getJobStatus();
+        jobStatus.setState(JobStatus.RUNNING);
+        Map<String, String> configuration = new HashMap<>();
+        configuration.put(
+                
KubernetesOperatorConfigOptions.OPERATOR_EVENT_EXCEPTION_LIMIT.key(), "2");
+        Configuration operatorConfig = Configuration.fromMap(configuration);
+        FlinkResourceContext<AbstractFlinkResource<?, ?>> ctx =
+                getResourceContext(deployment, operatorConfig); // set a 
non-terminal state
+
+        var jobId = 
JobID.fromHexString(deployment.getStatus().getJobStatus().getJobId());
+        ctx.getExceptionCacheEntry().setJobId(jobId.toHexString());
+        ctx.getExceptionCacheEntry().setLastTimestamp(500L);
+
+        flinkService.submitApplicationCluster(
+                deployment.getSpec().getJob(), 
ctx.getDeployConfig(deployment.getSpec()), false);
+        flinkService.addExceptionHistory(jobId, "ExceptionOne", "trace1", 
1000L);
+        flinkService.addExceptionHistory(jobId, "ExceptionTwo", "trace2", 
2000L);
+        flinkService.addExceptionHistory(jobId, "ExceptionThree", "trace3", 
3000L);
+
+        // Ensure jobFailedErr is null before the observe call
+        flinkService.setJobFailedErr(null);
+
+        observer.observe(ctx);
+
+        var events =
+                kubernetesClient
+                        .v1()
+                        .events()
+                        .inNamespace(deployment.getMetadata().getNamespace())
+                        .list()
+                        .getItems();
+        assertEquals(2, events.size());
+    }
+
+    @Test
+    public void testStackTraceTruncationConfig() throws Exception {
+        var deployment = initDeployment();
+        var status = deployment.getStatus();
+        var jobStatus = status.getJobStatus();
+        jobStatus.setState(JobStatus.RUNNING);
+        Map<String, String> configuration = new HashMap<>();
+        configuration.put(
+                
KubernetesOperatorConfigOptions.OPERATOR_EVENT_EXCEPTION_STACKTRACE_LINES.key(),
+                "2");
+        Configuration operatorConfig = Configuration.fromMap(configuration);
+        FlinkResourceContext<AbstractFlinkResource<?, ?>> ctx =
+                getResourceContext(deployment, operatorConfig);
+
+        var jobId = 
JobID.fromHexString(deployment.getStatus().getJobStatus().getJobId());
+        flinkService.submitApplicationCluster(
+                deployment.getSpec().getJob(), 
ctx.getDeployConfig(deployment.getSpec()), false);
+        ReconciliationUtils.updateStatusForDeployedSpec(deployment, new 
Configuration());
+        ctx.getExceptionCacheEntry().setJobId(jobId.toHexString());
+        ctx.getExceptionCacheEntry().setLastTimestamp(3000L);
+
+        long exceptionTime = 4000L;
+        String longTrace = "line1\nline2\nline3\nline4";
+        flinkService.addExceptionHistory(jobId, "StackTraceCheck", longTrace, 
exceptionTime);
+
+        // Ensure jobFailedErr is null before the observe call
+        flinkService.setJobFailedErr(null);
+        observer.observe(ctx);
+
+        var events =
+                kubernetesClient
+                        .v1()
+                        .events()
+                        .inNamespace(deployment.getMetadata().getNamespace())
+                        .list()
+                        .getItems();
+        assertEquals(1, events.size());
+        String msg = events.get(0).getMessage();
+        assertTrue(msg.contains("line1"));
+        assertTrue(msg.contains("line2"));
+        assertFalse(msg.contains("line3"));
+        assertTrue(msg.contains("... (2 more lines)"));
+        // check that exception time becomes the event time
+        ZonedDateTime zonedDateTime =
+                ZonedDateTime.ofInstant(Instant.ofEpochMilli(exceptionTime), 
ZoneId.of("UTC"));
+        MicroTime expectedEventTime = new MicroTime(zonedDateTime.toString());
+        assertEquals(expectedEventTime, events.get(0).getEventTime());
+    }
+
+    @Test
+    public void testIgnoreOldExceptions() throws Exception {
+        var deployment = initDeployment();
+        var status = deployment.getStatus();
+        var jobStatus = status.getJobStatus();
+        jobStatus.setState(JobStatus.RUNNING); // set a non-terminal state
+
+        FlinkResourceContext<AbstractFlinkResource<?, ?>> ctx = 
getResourceContext(deployment);
+        
ctx.getExceptionCacheEntry().setJobId(deployment.getStatus().getJobStatus().getJobId());
+        ctx.getExceptionCacheEntry().setLastTimestamp(2500L);
+
+        var jobId = 
JobID.fromHexString(deployment.getStatus().getJobStatus().getJobId());
+        flinkService.submitApplicationCluster(
+                deployment.getSpec().getJob(), 
ctx.getDeployConfig(deployment.getSpec()), false);
+
+        flinkService.addExceptionHistory(jobId, "OldException", "old", 1000L);
+        flinkService.addExceptionHistory(jobId, "MidException", "mid", 2000L);
+        flinkService.addExceptionHistory(jobId, "NewException", "new", 3000L);
+
+        // Ensure jobFailedErr is null before the observe call
+        flinkService.setJobFailedErr(null);
+        observer.observe(ctx);
+
+        var events =
+                kubernetesClient
+                        .v1()
+                        .events()
+                        .inNamespace(deployment.getMetadata().getNamespace())
+                        .list()
+                        .getItems();
+        assertEquals(1, events.size());
+        assertTrue(events.get(0).getMessage().contains("NewException"));
+    }
+

Review Comment:
   Would probably good to add a test for a job that failed terminally, to make 
sure the events are recorded (see my other comment)



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscr...@flink.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org

Reply via email to