zhuzhurk commented on a change in pull request #13730:
URL: https://github.com/apache/flink/pull/13730#discussion_r511720650



##########
File path: 
flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/SlotSharingExecutionSlotAllocator.java
##########
@@ -212,20 +214,24 @@ private ResourceProfile 
getPhysicalSlotResourceProfile(ExecutionSlotSharingGroup
                        .reduce(ResourceProfile.ZERO, (r, e) -> 
r.merge(resourceProfileRetriever.apply(e)), ResourceProfile::merge);
        }
 
-       private SharingPhysicalSlotRequestBulk 
createBulk(Map<ExecutionSlotSharingGroup, List<ExecutionVertexID>> executions) {
-               Map<ExecutionSlotSharingGroup, ResourceProfile> pendingRequests 
= executions
-                       .keySet()
-                       .stream()
-                       .collect(Collectors.toMap(
-                               group -> group,
-                               group -> 
sharedSlots.get(group).getPhysicalSlotResourceProfile()
-                       ));
+       private Optional<SharingPhysicalSlotRequestBulk> createBulk(
+                       Map<ExecutionSlotSharingGroup, List<ExecutionVertexID>> 
executions) {
+               Map<ExecutionSlotSharingGroup, ResourceProfile> pendingRequests 
= new HashMap<>();
+               for (ExecutionSlotSharingGroup group : executions.keySet()) {
+                       SharedSlot sharedSlot = sharedSlots.get(group);
+                       if (sharedSlot == null || 
sharedSlot.getSlotContextFuture().isCompletedExceptionally()) {

Review comment:
       I think this case is not possible to happen.
   If so it can happen only when there is a bug, can we just throw exception to 
expose the bug?
   Correct me if I'm wrong.

##########
File path: 
flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/DefaultSyncPreferredLocationsRetrieverTest.java
##########
@@ -0,0 +1,77 @@
+/*
+ * 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;
+
+import org.apache.flink.runtime.scheduler.strategy.ExecutionVertexID;
+import org.apache.flink.runtime.taskmanager.TaskManagerLocation;
+
+import org.junit.Test;
+
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Optional;
+
+import static 
org.apache.flink.runtime.executiongraph.ExecutionGraphTestUtils.createRandomExecutionVertexId;
+import static org.hamcrest.CoreMatchers.is;
+import static org.hamcrest.Matchers.hasSize;
+import static org.junit.Assert.assertThat;
+
+/**
+ * Tests for {@link DefaultSyncPreferredLocationsRetriever}.
+ */
+public class DefaultSyncPreferredLocationsRetrieverTest {

Review comment:
       should extend TestLogger

##########
File path: 
flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/DefaultSyncPreferredLocationsRetrieverTest.java
##########
@@ -0,0 +1,77 @@
+/*
+ * 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;
+
+import org.apache.flink.runtime.scheduler.strategy.ExecutionVertexID;
+import org.apache.flink.runtime.taskmanager.TaskManagerLocation;
+
+import org.junit.Test;
+
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Optional;
+
+import static 
org.apache.flink.runtime.executiongraph.ExecutionGraphTestUtils.createRandomExecutionVertexId;
+import static org.hamcrest.CoreMatchers.is;
+import static org.hamcrest.Matchers.hasSize;
+import static org.junit.Assert.assertThat;
+
+/**
+ * Tests for {@link DefaultSyncPreferredLocationsRetriever}.
+ */
+public class DefaultSyncPreferredLocationsRetrieverTest {
+       private static final ExecutionVertexID EV1 = 
createRandomExecutionVertexId();
+       private static final ExecutionVertexID EV2 = 
createRandomExecutionVertexId();
+       private static final ExecutionVertexID EV3 = 
createRandomExecutionVertexId();
+       private static final ExecutionVertexID EV4 = 
createRandomExecutionVertexId();
+       private static final ExecutionVertexID EV5 = 
createRandomExecutionVertexId();
+
+       @Test
+       public void testAvailableInputLocationRetrieval() {
+               TestingInputsLocationsRetriever originalLocationRetriever =
+                               new TestingInputsLocationsRetriever
+                                               .Builder()
+                                               .connectConsumerToProducer(EV5, 
EV1)
+                                               .connectConsumerToProducer(EV5, 
EV2)
+                                               .connectConsumerToProducer(EV5, 
EV3)
+                                               .connectConsumerToProducer(EV5, 
EV4)
+                                               .build();
+
+               originalLocationRetriever.assignTaskManagerLocation(EV1);
+               originalLocationRetriever.markScheduled(EV2);
+               originalLocationRetriever.failTaskManagerLocation(EV3, new 
Throwable());
+               originalLocationRetriever.cancelTaskManagerLocation(EV4);
+
+               SyncPreferredLocationsRetriever locationsRetriever =
+                               new DefaultSyncPreferredLocationsRetriever(
+                                               executionVertexId -> 
Optional.empty(),
+                                               originalLocationRetriever);
+
+               Collection<TaskManagerLocation> preferredLocations = 
locationsRetriever
+                               .getPreferredLocations(EV5, 
Collections.emptySet());
+               TaskManagerLocation expectedLocation = originalLocationRetriever
+                               .getTaskManagerLocation(EV1)
+                               .get()
+                               .join();
+
+               assertThat(preferredLocations, hasSize(1));
+               // only EV1 producer location is expected, all others are not 
available
+               assertThat(preferredLocations.iterator().next(), 
is(expectedLocation));

Review comment:
       can be `assertThat(preferredLocations, contains(expectedLocation))`
   and then we do not need `assertThat(preferredLocations, hasSize(1));`

##########
File path: 
flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/AvailableInputsLocationsRetrieverTest.java
##########
@@ -0,0 +1,90 @@
+/*
+ * 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;
+
+import org.apache.flink.runtime.scheduler.strategy.ExecutionVertexID;
+import org.apache.flink.util.TestLogger;
+
+import org.junit.Test;
+
+import java.util.Collection;
+
+import static 
org.apache.flink.runtime.executiongraph.ExecutionGraphTestUtils.createRandomExecutionVertexId;
+import static org.hamcrest.core.Is.is;
+import static org.junit.Assert.assertThat;
+
+/**
+ * Tests for {@link AvailableInputsLocationsRetriever}.
+ */
+public class AvailableInputsLocationsRetrieverTest extends TestLogger {
+       private static final ExecutionVertexID EV1 = 
createRandomExecutionVertexId();
+       private static final ExecutionVertexID EV2 = 
createRandomExecutionVertexId();
+
+       @Test
+       public void testNoInputLocation() {
+               TestingInputsLocationsRetriever originalLocationRetriever = 
getOriginalLocationRetriever();
+               InputsLocationsRetriever availableInputsLocationsRetriever =
+                       new 
AvailableInputsLocationsRetriever(originalLocationRetriever);
+               
assertThat(availableInputsLocationsRetriever.getTaskManagerLocation(EV1).isPresent(),
 is(false));
+       }
+
+       @Test
+       public void testNoInputLocationIfNotDone() {
+               TestingInputsLocationsRetriever originalLocationRetriever = 
getOriginalLocationRetriever();
+               originalLocationRetriever.markScheduled(EV1);
+               InputsLocationsRetriever availableInputsLocationsRetriever =
+                       new 
AvailableInputsLocationsRetriever(originalLocationRetriever);
+               
assertThat(availableInputsLocationsRetriever.getTaskManagerLocation(EV1).isPresent(),
 is(false));
+       }
+
+       @Test
+       public void testNoInputLocationIfFailed() {
+               TestingInputsLocationsRetriever originalLocationRetriever = 
getOriginalLocationRetriever();
+               originalLocationRetriever.failTaskManagerLocation(EV1, new 
Throwable());
+               InputsLocationsRetriever availableInputsLocationsRetriever =
+                       new 
AvailableInputsLocationsRetriever(originalLocationRetriever);
+               
assertThat(availableInputsLocationsRetriever.getTaskManagerLocation(EV1).isPresent(),
 is(false));
+       }
+
+       @Test
+       public void testInputLocationIfDone() {
+               TestingInputsLocationsRetriever originalLocationRetriever = 
getOriginalLocationRetriever();
+               originalLocationRetriever.assignTaskManagerLocation(EV1);
+               InputsLocationsRetriever availableInputsLocationsRetriever =
+                       new 
AvailableInputsLocationsRetriever(originalLocationRetriever);
+               
assertThat(availableInputsLocationsRetriever.getTaskManagerLocation(EV1).isPresent(),
 is(true));
+       }
+
+       @Test
+       public void testConsumedResultPartitionsProducers() {
+               TestingInputsLocationsRetriever originalLocationRetriever = 
getOriginalLocationRetriever();
+               InputsLocationsRetriever availableInputsLocationsRetriever =
+                       new 
AvailableInputsLocationsRetriever(originalLocationRetriever);
+               Collection<Collection<ExecutionVertexID>> producers =
+                       
availableInputsLocationsRetriever.getConsumedResultPartitionsProducers(EV2);
+               assertThat(producers.size(), is(1));
+               Collection<ExecutionVertexID> resultProducers = 
producers.iterator().next();
+               assertThat(resultProducers.size(), is(1));
+               assertThat(resultProducers.iterator().next(), is(EV1));

Review comment:
       These 2 asserts can be replaced with `assertThat(resultProducers, 
contains(EV1))`




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

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


Reply via email to