zentol commented on a change in pull request #17387:
URL: https://github.com/apache/flink/pull/17387#discussion_r747527918



##########
File path: 
flink-runtime/src/main/java/org/apache/flink/runtime/metrics/groups/TaskManagerMetricGroup.java
##########
@@ -81,61 +79,33 @@ public String taskManagerId() {
     //  job groups
     // ------------------------------------------------------------------------
 
-    public TaskMetricGroup addTaskForJob(
-            final JobID jobId,
-            final String jobName,
-            final JobVertexID jobVertexId,
-            final ExecutionAttemptID executionAttemptId,
-            final String taskName,
-            final int subtaskIndex,
-            final int attemptNumber) {
+    public TaskManagerJobMetricGroup getOrCreateTaskManagerJobMetricGroup(
+            JobID jobId, String jobName) {
         Preconditions.checkNotNull(jobId);
-
         String resolvedJobName = jobName == null || jobName.isEmpty() ? 
jobId.toString() : jobName;
-
-        // we cannot strictly lock both our map modification and the job group 
modification
-        // because it might lead to a deadlock
-        while (true) {
-            // get or create a jobs metric group
-            TaskManagerJobMetricGroup currentJobGroup;
-            synchronized (this) {
-                currentJobGroup = jobs.get(jobId);
-
-                if (currentJobGroup == null || currentJobGroup.isClosed()) {
-                    currentJobGroup =
-                            new TaskManagerJobMetricGroup(registry, this, 
jobId, resolvedJobName);
-                    jobs.put(jobId, currentJobGroup);
-                }
-            }
-
-            // try to add another task. this may fail if we found a 
pre-existing job metrics
-            // group and it is closed concurrently
-            TaskMetricGroup taskGroup =
-                    currentJobGroup.addTask(
-                            jobVertexId, executionAttemptId, taskName, 
subtaskIndex, attemptNumber);
-
-            if (taskGroup != null) {
-                // successfully added the next task
-                return taskGroup;
+        TaskManagerJobMetricGroup jobGroup;
+        synchronized (this) { // synchronization isn't strictly necessary as 
of FLINK-24864
+            jobGroup = jobs.get(jobId);
+            if (jobGroup == null) {
+                jobGroup = new TaskManagerJobMetricGroup(registry, this, 
jobId, resolvedJobName);
+                jobs.put(jobId, jobGroup);
             }
-
-            // else fall through the loop
         }
+        return jobGroup;
     }
 
-    public void removeJobMetricsGroup(JobID jobId, TaskManagerJobMetricGroup 
group) {
-        if (jobId == null || group == null || !group.isClosed()) {
-            return;
-        }
-
-        synchronized (this) {
-            // optimistically remove the currently contained group, and check 
later if it was
-            // correct
-            TaskManagerJobMetricGroup containedGroup = jobs.remove(jobId);
+    public TaskManagerJobMetricGroup getJobMetricsGroup(JobID jobId) {

Review comment:
       Can we mark this as `@VisibleForTesting`?

##########
File path: 
flink-runtime/src/main/java/org/apache/flink/runtime/metrics/groups/TaskManagerJobMetricGroup.java
##########
@@ -38,6 +39,7 @@
  * <p>Contains extra logic for adding Tasks ({@link TaskMetricGroup}).
  */
 @Internal
+@NotThreadSafe

Review comment:
       I thought we didn't want to change this?

##########
File path: 
flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorTest.java
##########
@@ -2536,6 +2484,61 @@ public void testDynamicSlotAllocation() throws Exception 
{
         }
     }
 
+    @Test
+    public void testReleasingJobResources() throws Exception {
+        AllocationID[] slots =
+                range(0, 5).mapToObj(i -> new 
AllocationID()).toArray(AllocationID[]::new);
+        try (TaskExecutorTestingContext ctx = 
createTaskExecutorTestingContext(slots.length)) {
+            ctx.start();
+            ResourceManagerId rmId;
+            {
+                CompletableFuture<Tuple3<ResourceID, InstanceID, SlotReport>>
+                        initialSlotReportFuture = new CompletableFuture<>();
+                rmId = 
createAndRegisterResourceManager(initialSlotReportFuture);
+                initialSlotReportFuture.get();
+            }
+
+            TaskExecutorGateway gateway =
+                    ctx.taskExecutor.getSelfGateway(TaskExecutorGateway.class);
+
+            for (int i = 0; i < slots.length; i++) {
+                requestSlot(
+                        gateway,
+                        jobId,
+                        slots[i],
+                        buildSlotID(i),
+                        ResourceProfile.UNKNOWN,
+                        ctx.jobMasterGateway.getAddress(),
+                        rmId);
+            }
+            ExecutionAttemptID attemptID =
+                    submitNoOpInvokableTask(
+                            slots[0], new 
TestingJobMasterGatewayBuilder().build(), gateway);
+            
assertNotNull(ctx.changelogStoragesManager.getChangelogStoragesByJobId(jobId));
+            assertNotNull(ctx.metricGroup.getJobMetricsGroup(jobId));
+
+            // cancel tasks before releasing the slots - so that TM will 
release job resources on
+            // the last slot release
+            gateway.cancelTask(attemptID, timeout).get();
+            while (ctx.taskSlotTable.getTasks(jobId).hasNext()) {

Review comment:
       this access isn't thread-safe; it's backed by a plain HashMap but the 
test and TM thread are accessing it.
   You need to run this within 
`ctx.taskExecutor.getMainThreadExecutableForTesting()`.

##########
File path: 
flink-runtime/src/test/java/org/apache/flink/runtime/metrics/groups/TaskManagerMetricGroupTest.java
##########
@@ -0,0 +1,92 @@
+package org.apache.flink.runtime.metrics.groups;
+
+/*
+ * 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.
+ */
+
+import org.apache.flink.api.common.JobID;
+
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+import static org.apache.flink.runtime.metrics.NoOpMetricRegistry.INSTANCE;
+import static org.junit.Assert.assertNotSame;
+import static org.junit.Assert.assertSame;
+import static org.junit.Assert.assertTrue;

Review comment:
       new tests should use junit5.

##########
File path: 
flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorTest.java
##########
@@ -2536,6 +2484,61 @@ public void testDynamicSlotAllocation() throws Exception 
{
         }
     }
 
+    @Test
+    public void testReleasingJobResources() throws Exception {
+        AllocationID[] slots =
+                range(0, 5).mapToObj(i -> new 
AllocationID()).toArray(AllocationID[]::new);
+        try (TaskExecutorTestingContext ctx = 
createTaskExecutorTestingContext(slots.length)) {
+            ctx.start();
+            ResourceManagerId rmId;
+            {
+                CompletableFuture<Tuple3<ResourceID, InstanceID, SlotReport>>
+                        initialSlotReportFuture = new CompletableFuture<>();
+                rmId = 
createAndRegisterResourceManager(initialSlotReportFuture);
+                initialSlotReportFuture.get();
+            }
+
+            TaskExecutorGateway gateway =
+                    ctx.taskExecutor.getSelfGateway(TaskExecutorGateway.class);
+
+            for (int i = 0; i < slots.length; i++) {
+                requestSlot(
+                        gateway,
+                        jobId,
+                        slots[i],
+                        buildSlotID(i),
+                        ResourceProfile.UNKNOWN,
+                        ctx.jobMasterGateway.getAddress(),
+                        rmId);
+            }
+            ExecutionAttemptID attemptID =
+                    submitNoOpInvokableTask(
+                            slots[0], new 
TestingJobMasterGatewayBuilder().build(), gateway);
+            
assertNotNull(ctx.changelogStoragesManager.getChangelogStoragesByJobId(jobId));
+            assertNotNull(ctx.metricGroup.getJobMetricsGroup(jobId));
+
+            // cancel tasks before releasing the slots - so that TM will 
release job resources on
+            // the last slot release
+            gateway.cancelTask(attemptID, timeout).get();
+            while (ctx.taskSlotTable.getTasks(jobId).hasNext()) {
+                // wait for task thread to notify TM about its final state
+                Thread.sleep(50);
+            }
+
+            for (int i = 0; i < slots.length; i++) {
+                gateway.freeSlot(slots[i], new RuntimeException("test 
exception"), timeout).get();
+                boolean isLast = i == slots.length - 1;
+                if (isLast) {
+                    
assertNull(ctx.changelogStoragesManager.getChangelogStoragesByJobId(jobId));

Review comment:
       this is also not thread-safe, same as above.

##########
File path: 
flink-runtime/src/main/java/org/apache/flink/runtime/metrics/groups/TaskManagerMetricGroup.java
##########
@@ -81,61 +79,33 @@ public String taskManagerId() {
     //  job groups
     // ------------------------------------------------------------------------
 
-    public TaskMetricGroup addTaskForJob(
-            final JobID jobId,
-            final String jobName,
-            final JobVertexID jobVertexId,
-            final ExecutionAttemptID executionAttemptId,
-            final String taskName,
-            final int subtaskIndex,
-            final int attemptNumber) {
+    public TaskManagerJobMetricGroup getOrCreateTaskManagerJobMetricGroup(

Review comment:
       this could be named `addJob` to be consistent with the 
`JobManagerMetricGroup`, and, well, pretty much all other methods? :D




-- 
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: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]


Reply via email to