You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@flink.apache.org by GitBox <gi...@apache.org> on 2022/05/14 06:48:34 UTC

[GitHub] [flink-kubernetes-operator] Aitozi opened a new pull request, #213: [FLINK-27612] Generate waring events when deleting the session cluste…

Aitozi opened a new pull request, #213:
URL: https://github.com/apache/flink-kubernetes-operator/pull/213

   …r with running jobs
   
   Manually test results:
   
   ![image](https://user-images.githubusercontent.com/9486140/168414353-aa9a1e26-072c-44ae-88a8-41966ce51463.png)
   


-- 
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-unsubscribe@flink.apache.org

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


[GitHub] [flink-kubernetes-operator] Aitozi commented on pull request #213: [FLINK-27612] Generate warning events when deleting the session cluste…

Posted by GitBox <gi...@apache.org>.
Aitozi commented on PR #213:
URL: https://github.com/apache/flink-kubernetes-operator/pull/213#issuecomment-1127833148

   I refactor based on the example, PTAL again @gyfora 


-- 
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-unsubscribe@flink.apache.org

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


[GitHub] [flink-kubernetes-operator] Aitozi commented on pull request #213: [FLINK-27612] Generate warning events when deleting the session cluste…

Posted by GitBox <gi...@apache.org>.
Aitozi commented on PR #213:
URL: https://github.com/apache/flink-kubernetes-operator/pull/213#issuecomment-1127761597

   @morhidi I have addressed your comments, please take a look again. cc @gyfora 


-- 
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-unsubscribe@flink.apache.org

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


[GitHub] [flink-kubernetes-operator] gyfora commented on a diff in pull request #213: [FLINK-27612] Generate warning events when deleting the session cluste…

Posted by GitBox <gi...@apache.org>.
gyfora commented on code in PR #213:
URL: https://github.com/apache/flink-kubernetes-operator/pull/213#discussion_r874051109


##########
flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/utils/EventUtils.java:
##########
@@ -0,0 +1,109 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.kubernetes.operator.utils;
+
+import io.fabric8.kubernetes.api.model.EventBuilder;
+import io.fabric8.kubernetes.api.model.HasMetadata;
+import io.fabric8.kubernetes.api.model.ObjectReferenceBuilder;
+import io.fabric8.kubernetes.client.KubernetesClient;
+
+import java.time.Instant;
+
+/** The util to generate an event for the target resource. */

Review Comment:
   We should add a note here that the core logic is copied/reused from https://github.com/EnMasseProject/enmasse/blob/master/k8s-api/src/main/java/io/enmasse/k8s/api/KubeEventLogger.java
   



##########
flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/utils/EventUtils.java:
##########
@@ -0,0 +1,109 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.kubernetes.operator.utils;
+
+import io.fabric8.kubernetes.api.model.EventBuilder;
+import io.fabric8.kubernetes.api.model.HasMetadata;
+import io.fabric8.kubernetes.api.model.ObjectReferenceBuilder;
+import io.fabric8.kubernetes.client.KubernetesClient;
+
+import java.time.Instant;
+
+/** The util to generate an event for the target resource. */
+public class EventUtils {
+
+    /** The type of the events. */
+    public enum Type {
+        Normal,
+        Warning
+    }
+
+    /** The component of events. */
+    public enum Component {
+        Operator
+    }

Review Comment:
   I think if we define these as Enums, we should use the enum values directly in the methods. Otherwise they could be String constants



-- 
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-unsubscribe@flink.apache.org

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


[GitHub] [flink-kubernetes-operator] Aitozi commented on pull request #213: [FLINK-27612] Generate warning events when deleting the session cluste…

Posted by GitBox <gi...@apache.org>.
Aitozi commented on PR #213:
URL: https://github.com/apache/flink-kubernetes-operator/pull/213#issuecomment-1127834234

   The output example :
   
   ```
     Type     Reason   Age                   From      Message
     ----     ------   ----                  ----      -------
     Warning  Cleanup  14s (x19 over 4m45s)  Operator  The session jobs [basic-session-job-example2, basic-session-job-example] should be deleted first
   ```


-- 
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-unsubscribe@flink.apache.org

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


[GitHub] [flink-kubernetes-operator] Aitozi commented on a diff in pull request #213: [FLINK-27612] Generate warning events when deleting the session cluste…

Posted by GitBox <gi...@apache.org>.
Aitozi commented on code in PR #213:
URL: https://github.com/apache/flink-kubernetes-operator/pull/213#discussion_r874248265


##########
flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/utils/EventUtils.java:
##########
@@ -0,0 +1,109 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.kubernetes.operator.utils;
+
+import io.fabric8.kubernetes.api.model.EventBuilder;
+import io.fabric8.kubernetes.api.model.HasMetadata;
+import io.fabric8.kubernetes.api.model.ObjectReferenceBuilder;
+import io.fabric8.kubernetes.client.KubernetesClient;
+
+import java.time.Instant;
+
+/** The util to generate an event for the target resource. */
+public class EventUtils {
+
+    /** The type of the events. */
+    public enum Type {
+        Normal,
+        Warning
+    }
+
+    /** The component of events. */
+    public enum Component {
+        Operator
+    }

Review Comment:
   You are right, Fixed



##########
flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/utils/EventUtils.java:
##########
@@ -0,0 +1,109 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.kubernetes.operator.utils;
+
+import io.fabric8.kubernetes.api.model.EventBuilder;
+import io.fabric8.kubernetes.api.model.HasMetadata;
+import io.fabric8.kubernetes.api.model.ObjectReferenceBuilder;
+import io.fabric8.kubernetes.client.KubernetesClient;
+
+import java.time.Instant;
+
+/** The util to generate an event for the target resource. */

Review Comment:
   Fixed



-- 
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-unsubscribe@flink.apache.org

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


[GitHub] [flink-kubernetes-operator] morhidi commented on a diff in pull request #213: [FLINK-27612] Generate warning events when deleting the session cluste…

Posted by GitBox <gi...@apache.org>.
morhidi commented on code in PR #213:
URL: https://github.com/apache/flink-kubernetes-operator/pull/213#discussion_r873626664


##########
flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/reconciler/deployment/SessionReconciler.java:
##########
@@ -157,8 +159,23 @@ public DeleteControl cleanup(FlinkDeployment flinkApp, Context context) {
                             sessionJobs.stream()
                                     .map(job -> job.getMetadata().getName())
                                     .collect(Collectors.toList()));
-            // TODO generate error events for this
-            LOG.warn(error);
+            var events =
+                    EventUtils.getEvents(flinkApp, kubernetesClient).getItems().stream()
+                            .filter(event -> event.getMessage().equals(error))
+                            .collect(Collectors.toList());
+            Event lastEvent = null;
+            if (events.isEmpty()) {
+                LOG.warn(error);
+            } else {
+                lastEvent = events.get(0);
+            }

Review Comment:
   Can we move this logic into the `EventUtils`? 



-- 
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-unsubscribe@flink.apache.org

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


[GitHub] [flink-kubernetes-operator] gyfora merged pull request #213: [FLINK-27612] Generate warning events when deleting the session cluste…

Posted by GitBox <gi...@apache.org>.
gyfora merged PR #213:
URL: https://github.com/apache/flink-kubernetes-operator/pull/213


-- 
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-unsubscribe@flink.apache.org

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


[GitHub] [flink-kubernetes-operator] gyfora commented on a diff in pull request #213: [FLINK-27612] Generate warning events when deleting the session cluste…

Posted by GitBox <gi...@apache.org>.
gyfora commented on code in PR #213:
URL: https://github.com/apache/flink-kubernetes-operator/pull/213#discussion_r873196927


##########
flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/utils/EventUtils.java:
##########
@@ -0,0 +1,98 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.kubernetes.operator.utils;
+
+import io.fabric8.kubernetes.api.model.Event;
+import io.fabric8.kubernetes.api.model.EventBuilder;
+import io.fabric8.kubernetes.api.model.EventList;
+import io.fabric8.kubernetes.api.model.HasMetadata;
+import io.fabric8.kubernetes.api.model.ObjectReferenceBuilder;
+import io.fabric8.kubernetes.client.KubernetesClient;
+
+import javax.annotation.Nullable;
+
+import java.time.ZonedDateTime;
+import java.time.format.DateTimeFormatter;
+
+/** The util to generate an event for the target resource. */
+public class EventUtils {
+
+    private static final DateTimeFormatter formatter = DateTimeFormatter.ISO_INSTANT;
+    public static final String TYPE_NORMAL = "Normal";
+    public static final String TYPE_WARNING = "Warning";
+
+    public static EventList getEvents(HasMetadata target, KubernetesClient client) {
+        var object =
+                new ObjectReferenceBuilder()
+                        .withKind(target.getKind())
+                        .withUid(target.getMetadata().getUid())
+                        .withName(target.getMetadata().getName())
+                        .withNamespace(target.getMetadata().getNamespace())
+                        .build();
+        return client.v1()
+                .events()
+                .inNamespace(target.getMetadata().getNamespace())
+                .withInvolvedObject(object)
+                .list();
+    }
+
+    public static Event createOrUpdateEvent(
+            KubernetesClient client,
+            HasMetadata target,
+            String type,
+            String reason,
+            String message,
+            @Nullable Event lastEvent) {
+
+        if (lastEvent == null) {
+            var event =
+                    new EventBuilder()
+                            .withApiVersion("v1")
+                            .withInvolvedObject(
+                                    new ObjectReferenceBuilder()
+                                            .withKind(target.getKind())
+                                            .withUid(target.getMetadata().getUid())
+                                            .withName(target.getMetadata().getName())
+                                            .withNamespace(target.getMetadata().getNamespace())
+                                            .build())
+                            .withType(type)
+                            .withReason(reason)
+                            .withFirstTimestamp(ZonedDateTime.now().format(formatter))
+                            .withLastTimestamp(ZonedDateTime.now().format(formatter))

Review Comment:
   I think `Instant.now().toString()` should do the trick also



-- 
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-unsubscribe@flink.apache.org

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


[GitHub] [flink-kubernetes-operator] gyfora commented on pull request #213: [FLINK-27612] Generate warning events when deleting the session cluste…

Posted by GitBox <gi...@apache.org>.
gyfora commented on PR #213:
URL: https://github.com/apache/flink-kubernetes-operator/pull/213#issuecomment-1127763601

   @Aitozi I think we should also adopt the naming scheme of the other solution that the @morhidi linked.
   This avoids using generated event names and allows us to find them easily by ns/name directly instead of listing and filtering.


-- 
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-unsubscribe@flink.apache.org

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


[GitHub] [flink-kubernetes-operator] Aitozi commented on a diff in pull request #213: [FLINK-27612] Generate warning events when deleting the session cluste…

Posted by GitBox <gi...@apache.org>.
Aitozi commented on code in PR #213:
URL: https://github.com/apache/flink-kubernetes-operator/pull/213#discussion_r873282895


##########
flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/utils/EventUtils.java:
##########
@@ -0,0 +1,98 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.kubernetes.operator.utils;
+
+import io.fabric8.kubernetes.api.model.Event;
+import io.fabric8.kubernetes.api.model.EventBuilder;
+import io.fabric8.kubernetes.api.model.EventList;
+import io.fabric8.kubernetes.api.model.HasMetadata;
+import io.fabric8.kubernetes.api.model.ObjectReferenceBuilder;
+import io.fabric8.kubernetes.client.KubernetesClient;
+
+import javax.annotation.Nullable;
+
+import java.time.ZonedDateTime;
+import java.time.format.DateTimeFormatter;
+
+/** The util to generate an event for the target resource. */
+public class EventUtils {
+
+    private static final DateTimeFormatter formatter = DateTimeFormatter.ISO_INSTANT;
+    public static final String TYPE_NORMAL = "Normal";
+    public static final String TYPE_WARNING = "Warning";
+
+    public static EventList getEvents(HasMetadata target, KubernetesClient client) {
+        var object =
+                new ObjectReferenceBuilder()
+                        .withKind(target.getKind())
+                        .withUid(target.getMetadata().getUid())
+                        .withName(target.getMetadata().getName())
+                        .withNamespace(target.getMetadata().getNamespace())
+                        .build();
+        return client.v1()
+                .events()
+                .inNamespace(target.getMetadata().getNamespace())
+                .withInvolvedObject(object)
+                .list();
+    }
+
+    public static Event createOrUpdateEvent(
+            KubernetesClient client,
+            HasMetadata target,
+            String type,
+            String reason,
+            String message,
+            @Nullable Event lastEvent) {
+
+        if (lastEvent == null) {
+            var event =
+                    new EventBuilder()
+                            .withApiVersion("v1")
+                            .withInvolvedObject(
+                                    new ObjectReferenceBuilder()
+                                            .withKind(target.getKind())
+                                            .withUid(target.getMetadata().getUid())
+                                            .withName(target.getMetadata().getName())
+                                            .withNamespace(target.getMetadata().getNamespace())
+                                            .build())
+                            .withType(type)
+                            .withReason(reason)
+                            .withFirstTimestamp(ZonedDateTime.now().format(formatter))
+                            .withLastTimestamp(ZonedDateTime.now().format(formatter))

Review Comment:
   Fixed



-- 
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-unsubscribe@flink.apache.org

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


[GitHub] [flink-kubernetes-operator] morhidi commented on a diff in pull request #213: [FLINK-27612] Generate warning events when deleting the session cluste…

Posted by GitBox <gi...@apache.org>.
morhidi commented on code in PR #213:
URL: https://github.com/apache/flink-kubernetes-operator/pull/213#discussion_r873650965


##########
flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/reconciler/deployment/SessionReconciler.java:
##########
@@ -157,8 +159,23 @@ public DeleteControl cleanup(FlinkDeployment flinkApp, Context context) {
                             sessionJobs.stream()
                                     .map(job -> job.getMetadata().getName())
                                     .collect(Collectors.toList()));
-            // TODO generate error events for this
-            LOG.warn(error);
+            var events =
+                    EventUtils.getEvents(flinkApp, kubernetesClient).getItems().stream()
+                            .filter(event -> event.getMessage().equals(error))
+                            .collect(Collectors.toList());
+            Event lastEvent = null;
+            if (events.isEmpty()) {
+                LOG.warn(error);
+            } else {
+                lastEvent = events.get(0);
+            }

Review Comment:
   Here is an example for reference: https://github.com/EnMasseProject/enmasse/blob/master/k8s-api/src/main/java/io/enmasse/k8s/api/KubeEventLogger.java



-- 
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-unsubscribe@flink.apache.org

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


[GitHub] [flink-kubernetes-operator] Aitozi commented on a diff in pull request #213: [FLINK-27612] Generate warning events when deleting the session cluste…

Posted by GitBox <gi...@apache.org>.
Aitozi commented on code in PR #213:
URL: https://github.com/apache/flink-kubernetes-operator/pull/213#discussion_r873655644


##########
flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/reconciler/deployment/SessionReconciler.java:
##########
@@ -157,8 +159,23 @@ public DeleteControl cleanup(FlinkDeployment flinkApp, Context context) {
                             sessionJobs.stream()
                                     .map(job -> job.getMetadata().getName())
                                     .collect(Collectors.toList()));
-            // TODO generate error events for this
-            LOG.warn(error);
+            var events =
+                    EventUtils.getEvents(flinkApp, kubernetesClient).getItems().stream()
+                            .filter(event -> event.getMessage().equals(error))
+                            .collect(Collectors.toList());
+            Event lastEvent = null;
+            if (events.isEmpty()) {
+                LOG.warn(error);
+            } else {
+                lastEvent = events.get(0);
+            }

Review Comment:
   Cool, I will update accordingly 



-- 
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-unsubscribe@flink.apache.org

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


[GitHub] [flink-kubernetes-operator] gyfora commented on pull request #213: [FLINK-27612] Generate warning events when deleting the session cluste…

Posted by GitBox <gi...@apache.org>.
gyfora commented on PR #213:
URL: https://github.com/apache/flink-kubernetes-operator/pull/213#issuecomment-1126979042

   I think this is a very nice change, would be great to reuse the utility for the deployment related events that we send also to avoid duplication.


-- 
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-unsubscribe@flink.apache.org

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


[GitHub] [flink-kubernetes-operator] gyfora commented on a diff in pull request #213: [FLINK-27612] Generate warning events when deleting the session cluste…

Posted by GitBox <gi...@apache.org>.
gyfora commented on code in PR #213:
URL: https://github.com/apache/flink-kubernetes-operator/pull/213#discussion_r873651593


##########
flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/reconciler/deployment/SessionReconciler.java:
##########
@@ -157,8 +159,23 @@ public DeleteControl cleanup(FlinkDeployment flinkApp, Context context) {
                             sessionJobs.stream()
                                     .map(job -> job.getMetadata().getName())
                                     .collect(Collectors.toList()));
-            // TODO generate error events for this
-            LOG.warn(error);
+            var events =
+                    EventUtils.getEvents(flinkApp, kubernetesClient).getItems().stream()
+                            .filter(event -> event.getMessage().equals(error))
+                            .collect(Collectors.toList());
+            Event lastEvent = null;
+            if (events.isEmpty()) {
+                LOG.warn(error);
+            } else {
+                lastEvent = events.get(0);
+            }

Review Comment:
   +1 for this suggestion, it makes for a nice self-contained logic that will be very easy to use everywhere.



-- 
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-unsubscribe@flink.apache.org

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


[GitHub] [flink-kubernetes-operator] Aitozi commented on pull request #213: [FLINK-27612] Generate warning events when deleting the session cluste…

Posted by GitBox <gi...@apache.org>.
Aitozi commented on PR #213:
URL: https://github.com/apache/flink-kubernetes-operator/pull/213#issuecomment-1127152146

   > I think this is a very nice change, would be great to reuse the utility for the deployment related events that we send also to avoid duplication.
   
   Yes, I notice that `TODO`, we could improve that in another PR


-- 
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-unsubscribe@flink.apache.org

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