[GitHub] [flink] dmvk commented on a change in pull request #18238: WIP: [FLINK-XXXXX] Task local recovery for the reactive mode.

2021-12-30 Thread GitBox


dmvk commented on a change in pull request #18238:
URL: https://github.com/apache/flink/pull/18238#discussion_r776615258



##
File path: 
flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskManagerRunner.java
##
@@ -382,6 +382,21 @@ public static int runTaskManager(Configuration 
configuration, PluginManager plug
 configuration,
 pluginManager,
 TaskManagerRunner::createTaskExecutorService);
+Runtime.getRuntime()
+.addShutdownHook(
+new Thread(
+new Runnable() {
+@Override
+public void run() {
+try {
+synchronized (this) {
+taskManagerRunner.close();
+}
+} catch (Exception e) {
+e.printStackTrace();
+}
+}
+}));

Review comment:
   this will be removed, it belongs to another story, but was useful for 
manual testing here




-- 
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




[GitHub] [flink] dmvk commented on a change in pull request #18238: WIP: [FLINK-XXXXX] Task local recovery for the reactive mode.

2021-12-30 Thread GitBox


dmvk commented on a change in pull request #18238:
URL: https://github.com/apache/flink/pull/18238#discussion_r776614745



##
File path: 
flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/adaptive/allocator/StateLocalitySlotAssigner.java
##
@@ -0,0 +1,301 @@
+/*
+ * 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.runtime.scheduler.adaptive.allocator;
+
+import org.apache.flink.runtime.clusterframework.types.AllocationID;
+import org.apache.flink.runtime.executiongraph.ArchivedExecutionGraph;
+import org.apache.flink.runtime.executiongraph.ArchivedExecutionJobVertex;
+import org.apache.flink.runtime.executiongraph.ArchivedExecutionVertex;
+import org.apache.flink.runtime.instance.SlotSharingGroupId;
+import org.apache.flink.runtime.jobgraph.JobVertexID;
+import org.apache.flink.runtime.jobmanager.scheduler.SlotSharingGroup;
+import org.apache.flink.runtime.jobmaster.SlotInfo;
+import org.apache.flink.runtime.scheduler.strategy.ExecutionVertexID;
+import org.apache.flink.runtime.state.KeyGroupRange;
+import org.apache.flink.runtime.state.KeyGroupRangeAssignment;
+import org.apache.flink.util.Preconditions;
+
+import org.apache.flink.shaded.guava30.com.google.common.base.MoreObjects;
+
+import javax.annotation.Nullable;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.PriorityQueue;
+import java.util.Set;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+import java.util.stream.StreamSupport;
+
+/** A {@link SlotAssigner} that assigns slots based on the number of local key 
groups. */
+public class StateLocalitySlotAssigner implements SlotAssigner {
+
+private static class AllocationScore>
+implements Comparable> {
+
+private final T id;
+private final AllocationID allocationId;
+
+public AllocationScore(T id, AllocationID allocationId, int score) {
+this.id = id;
+this.allocationId = allocationId;
+this.score = score;
+}
+
+private final int score;
+
+public T getId() {
+return id;
+}
+
+public AllocationID getAllocationId() {
+return allocationId;
+}
+
+public int getScore() {
+return score;
+}
+
+@Override
+public int compareTo(StateLocalitySlotAssigner.AllocationScore 
other) {
+int result = Integer.compare(score, other.score);
+if (result != 0) {
+return result;
+}
+result = other.allocationId.compareTo(allocationId);
+if (result != 0) {
+return result;
+}
+return other.id.compareTo(id);
+}
+
+@Override
+public String toString() {
+return MoreObjects.toStringHelper(this)
+.add("id", id)
+.add("allocationId", allocationId)
+.add("score", score)
+.toString();
+}
+}
+
+private final Map> locality;
+private final Map maxParallelism;
+
+public StateLocalitySlotAssigner(ArchivedExecutionGraph 
archivedExecutionGraph) {
+this.locality = calculateLocalKeyGroups(archivedExecutionGraph);
+this.maxParallelism =
+StreamSupport.stream(
+
archivedExecutionGraph.getVerticesTopologically().spliterator(),
+false)
+.collect(
+Collectors.toMap(
+
ArchivedExecutionJobVertex::getJobVertexId,
+
ArchivedExecutionJobVertex::getMaxParallelism));
+}
+
+@Override
+public List 
assignSlots(
+Collection slots,
+Collection 
groups) {
+final PriorityQueue> groupScore =
+new PriorityQueue<>(Comparator.reverseOrder());
+
+final 

[GitHub] [flink] dmvk commented on a change in pull request #18238: WIP: [FLINK-XXXXX] Task local recovery for the reactive mode.

2021-12-30 Thread GitBox


dmvk commented on a change in pull request #18238:
URL: https://github.com/apache/flink/pull/18238#discussion_r776614525



##
File path: 
flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/adaptive/allocator/SlotSharingSlotAllocator.java
##
@@ -103,60 +112,53 @@ public ResourceCounter calculateRequiredSlots(
 return Optional.empty();
 }
 
-final Iterator slotIterator = freeSlots.iterator();
-
 final Collection assignments = new 
ArrayList<>();
 final Map allVertexParallelism = new HashMap<>();
 
+final Map> slotsByGroupId =
+slotAssigner.splitSlotsBetweenSlotSharingGroups(
+freeSlots, jobInformation.getSlotSharingGroups());
+
 for (SlotSharingGroup slotSharingGroup : 
jobInformation.getSlotSharingGroups()) {
 final List containedJobVertices =
 slotSharingGroup.getJobVertexIds().stream()
 .map(jobInformation::getVertexInformation)
 .collect(Collectors.toList());
-
-final Map vertexParallelism =
-determineParallelism(containedJobVertices, 
slotsPerSlotSharingGroup);
-
-final Iterable 
sharedSlotToVertexAssignment =
-createExecutionSlotSharingGroups(vertexParallelism);
-
-for (ExecutionSlotSharingGroup executionSlotSharingGroup :
-sharedSlotToVertexAssignment) {
-final SlotInfo slotInfo = slotIterator.next();
-
-assignments.add(
-new 
ExecutionSlotSharingGroupAndSlot(executionSlotSharingGroup, slotInfo));
-}
-allVertexParallelism.putAll(vertexParallelism);
+final Map adjustedParallelism =
+adjustParallelism(containedJobVertices, 
slotsPerSlotSharingGroup);
+final List sharedSlotToVertexAssignment 
=
+createExecutionSlotSharingGroups(adjustedParallelism);
+final Set groupSlots =
+
slotsByGroupId.get(slotSharingGroup.getSlotSharingGroupId());
+assignments.addAll(slotAssigner.assignSlots(groupSlots, 
sharedSlotToVertexAssignment));
+allVertexParallelism.putAll(adjustedParallelism);
 }
 
+System.out.println("  ");
 return Optional.of(new 
VertexParallelismWithSlotSharing(allVertexParallelism, assignments));
 }
 
-private static Map determineParallelism(
+private static Map adjustParallelism(
 Collection containedJobVertices, 
int availableSlots) {
 final Map vertexParallelism = new HashMap<>();
 for (JobInformation.VertexInformation jobVertex : 
containedJobVertices) {
 final int parallelism = Math.min(jobVertex.getParallelism(), 
availableSlots);
-
 vertexParallelism.put(jobVertex.getJobVertexID(), parallelism);
 }
-
 return vertexParallelism;
 }
 
-private static Iterable 
createExecutionSlotSharingGroups(
+private static List 
createExecutionSlotSharingGroups(
 Map containedJobVertices) {
 final Map> 
sharedSlotToVertexAssignment = new HashMap<>();
-
-for (Map.Entry jobVertex : 
containedJobVertices.entrySet()) {
-for (int i = 0; i < jobVertex.getValue(); i++) {
-sharedSlotToVertexAssignment
-.computeIfAbsent(i, ignored -> new HashSet<>())
-.add(new ExecutionVertexID(jobVertex.getKey(), i));
-}
-}
-
+containedJobVertices.forEach(
+(jobVertexId, parallelism) -> {
+for (int subtaskIdx = 0; subtaskIdx < parallelism; 
subtaskIdx++) {
+sharedSlotToVertexAssignment
+.computeIfAbsent(subtaskIdx, ignored -> new 
HashSet<>())
+.add(new ExecutionVertexID(jobVertexId, 
subtaskIdx));
+}
+});

Review comment:
   no special reason, it just felt more readable having a named "key" and 
"value"




-- 
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




[GitHub] [flink] dmvk commented on a change in pull request #18238: WIP: [FLINK-XXXXX] Task local recovery for the reactive mode.

2021-12-30 Thread GitBox


dmvk commented on a change in pull request #18238:
URL: https://github.com/apache/flink/pull/18238#discussion_r776613842



##
File path: 
flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/adaptive/AdaptiveScheduler.java
##
@@ -203,6 +204,8 @@
 
 private final SchedulerExecutionMode executionMode;
 
+@Nullable private ArchivedExecutionGraph archivedExecutionGraph = null;

Review comment:
   
   
   ```
   getState()
   .as(StateWithExecutionGraph.class)
   .map(StateWithExecutionGraph::getJob)
   .orElse(null));
   ```
   at appropriate place seems to do the trick  






-- 
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




[GitHub] [flink] dmvk commented on a change in pull request #18238: WIP: [FLINK-XXXXX] Task local recovery for the reactive mode.

2021-12-30 Thread GitBox


dmvk commented on a change in pull request #18238:
URL: https://github.com/apache/flink/pull/18238#discussion_r776611441



##
File path: 
flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/adaptive/allocator/StateLocalitySlotAssigner.java
##
@@ -0,0 +1,301 @@
+/*
+ * 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.runtime.scheduler.adaptive.allocator;
+
+import org.apache.flink.runtime.clusterframework.types.AllocationID;
+import org.apache.flink.runtime.executiongraph.ArchivedExecutionGraph;
+import org.apache.flink.runtime.executiongraph.ArchivedExecutionJobVertex;
+import org.apache.flink.runtime.executiongraph.ArchivedExecutionVertex;
+import org.apache.flink.runtime.instance.SlotSharingGroupId;
+import org.apache.flink.runtime.jobgraph.JobVertexID;
+import org.apache.flink.runtime.jobmanager.scheduler.SlotSharingGroup;
+import org.apache.flink.runtime.jobmaster.SlotInfo;
+import org.apache.flink.runtime.scheduler.strategy.ExecutionVertexID;
+import org.apache.flink.runtime.state.KeyGroupRange;
+import org.apache.flink.runtime.state.KeyGroupRangeAssignment;
+import org.apache.flink.util.Preconditions;
+
+import org.apache.flink.shaded.guava30.com.google.common.base.MoreObjects;
+
+import javax.annotation.Nullable;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.PriorityQueue;
+import java.util.Set;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+import java.util.stream.StreamSupport;
+
+/** A {@link SlotAssigner} that assigns slots based on the number of local key 
groups. */
+public class StateLocalitySlotAssigner implements SlotAssigner {
+
+private static class AllocationScore>
+implements Comparable> {
+
+private final T id;
+private final AllocationID allocationId;
+
+public AllocationScore(T id, AllocationID allocationId, int score) {
+this.id = id;
+this.allocationId = allocationId;
+this.score = score;
+}
+
+private final int score;
+
+public T getId() {
+return id;
+}
+
+public AllocationID getAllocationId() {
+return allocationId;
+}
+
+public int getScore() {
+return score;
+}
+
+@Override
+public int compareTo(StateLocalitySlotAssigner.AllocationScore 
other) {
+int result = Integer.compare(score, other.score);
+if (result != 0) {
+return result;
+}
+result = other.allocationId.compareTo(allocationId);
+if (result != 0) {
+return result;
+}
+return other.id.compareTo(id);

Review comment:
   the id comparison is only important to get a deterministic result for 
testing




-- 
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