-
Notifications
You must be signed in to change notification settings - Fork 13k
Commit
This commit does not belong to any branch on this repository, and may belong to a fork outside of the repository.
[FLINK-19552][coordination] Consider only available input location pr…
…eferences for slot profile in pipelined region scheduling
- Loading branch information
Showing
12 changed files
with
374 additions
and
76 deletions.
There are no files selected for viewing
50 changes: 50 additions & 0 deletions
50
...e/src/main/java/org/apache/flink/runtime/scheduler/AvailableInputsLocationsRetriever.java
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,50 @@ | ||
/* | ||
* 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 java.util.Collection; | ||
import java.util.Optional; | ||
import java.util.concurrent.CompletableFuture; | ||
|
||
/** | ||
* {@link InputsLocationsRetriever} which filters out only already available input locations. | ||
*/ | ||
class AvailableInputsLocationsRetriever implements InputsLocationsRetriever { | ||
private final InputsLocationsRetriever inputsLocationsRetriever; | ||
|
||
AvailableInputsLocationsRetriever(InputsLocationsRetriever inputsLocationsRetriever) { | ||
this.inputsLocationsRetriever = inputsLocationsRetriever; | ||
} | ||
|
||
@Override | ||
public Collection<Collection<ExecutionVertexID>> getConsumedResultPartitionsProducers( | ||
ExecutionVertexID executionVertexId) { | ||
return inputsLocationsRetriever.getConsumedResultPartitionsProducers(executionVertexId); | ||
} | ||
|
||
@Override | ||
public Optional<CompletableFuture<TaskManagerLocation>> getTaskManagerLocation(ExecutionVertexID executionVertexId) { | ||
return inputsLocationsRetriever | ||
.getTaskManagerLocation(executionVertexId) | ||
.filter(future -> future.isDone() && !future.isCompletedExceptionally()); | ||
} | ||
} |
62 changes: 62 additions & 0 deletions
62
.../main/java/org/apache/flink/runtime/scheduler/DefaultSyncPreferredLocationsRetriever.java
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,62 @@ | ||
/* | ||
* 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.apache.flink.util.Preconditions; | ||
|
||
import java.util.Collection; | ||
import java.util.Set; | ||
import java.util.concurrent.CompletableFuture; | ||
|
||
/** | ||
* Synchronous version of {@link DefaultPreferredLocationsRetriever}. | ||
* | ||
* <p>This class turns {@link DefaultPreferredLocationsRetriever} into {@link SyncPreferredLocationsRetriever}. | ||
* The method {@link #getPreferredLocations(ExecutionVertexID, Set)} does not return {@link CompletableFuture} | ||
* of preferred locations, it returns only locations which are available immediately. | ||
* This behaviour is achieved by wrapping the original {@link InputsLocationsRetriever} with | ||
* {@link AvailableInputsLocationsRetriever} and hence making it synchronous without blocking. | ||
* As {@link StateLocationRetriever} is already synchronous, | ||
* the overall location retrieval becomes synchronous without blocking. | ||
*/ | ||
class DefaultSyncPreferredLocationsRetriever implements SyncPreferredLocationsRetriever { | ||
private final PreferredLocationsRetriever asyncPreferredLocationsRetriever; | ||
|
||
DefaultSyncPreferredLocationsRetriever( | ||
StateLocationRetriever stateLocationRetriever, | ||
InputsLocationsRetriever inputsLocationsRetriever) { | ||
this.asyncPreferredLocationsRetriever = new DefaultPreferredLocationsRetriever( | ||
stateLocationRetriever, | ||
new AvailableInputsLocationsRetriever(inputsLocationsRetriever)); | ||
} | ||
|
||
@Override | ||
public Collection<TaskManagerLocation> getPreferredLocations( | ||
ExecutionVertexID executionVertexId, | ||
Set<ExecutionVertexID> producersToIgnore) { | ||
CompletableFuture<Collection<TaskManagerLocation>> preferredLocationsFuture = | ||
asyncPreferredLocationsRetriever.getPreferredLocations(executionVertexId, producersToIgnore); | ||
Preconditions.checkState(preferredLocationsFuture.isDone()); | ||
// it is safe to do the blocking call here | ||
// as the underlying InputsLocationsRetriever returns only immediately available locations | ||
return preferredLocationsFuture.join(); | ||
} | ||
} |
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
43 changes: 43 additions & 0 deletions
43
...ime/src/main/java/org/apache/flink/runtime/scheduler/SyncPreferredLocationsRetriever.java
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,43 @@ | ||
/* | ||
* 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 java.util.Collection; | ||
import java.util.Set; | ||
|
||
/** | ||
* Component to retrieve the preferred locations of an execution vertex. | ||
*/ | ||
@FunctionalInterface | ||
public interface SyncPreferredLocationsRetriever { | ||
|
||
/** | ||
* Returns preferred locations of an execution vertex. | ||
* | ||
* @param executionVertexId id of the execution vertex | ||
* @param producersToIgnore producer vertices to ignore when calculating input locations | ||
* @return future of preferred locations | ||
*/ | ||
Collection<TaskManagerLocation> getPreferredLocations( | ||
ExecutionVertexID executionVertexId, | ||
Set<ExecutionVertexID> producersToIgnore); | ||
} |
90 changes: 90 additions & 0 deletions
90
...c/test/java/org/apache/flink/runtime/scheduler/AvailableInputsLocationsRetrieverTest.java
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -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)); | ||
} | ||
|
||
private static TestingInputsLocationsRetriever getOriginalLocationRetriever() { | ||
return new TestingInputsLocationsRetriever.Builder().connectConsumerToProducer(EV2, EV1).build(); | ||
} | ||
} |
Oops, something went wrong.