-
Notifications
You must be signed in to change notification settings - Fork 13.9k
[FLINK-11391] Introduce shuffle master interface #8362
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Conversation
|
Thanks a lot for your contribution to the Apache Flink project. I'm the @flinkbot. I help the community Automated ChecksLast check on commit f7014b7 (Wed Aug 07 16:30:56 UTC 2019) Warnings:
Mention the bot in a comment to re-run the automated checks. Review Progress
Please see the Pull Request Review Guide for a full explanation of the review process. DetailsThe Bot is tracking the review progress through labels. Labels are applied according to the order of the review items. For consensus, approval by a Flink committer of PMC member is required Bot commandsThe @flinkbot bot supports the following commands:
|
|
@flinkbot attention @tillrohrmann @zhijiangW |
d9129e5 to
1c84635
Compare
| } | ||
|
|
||
| @Override | ||
| public String toString() { |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I am not sure whether to add the left infos of numberOfSubpartitions and maxParallelism here.
| return maxParallelism; | ||
| } | ||
|
|
||
| public int getConnectionIndex() { |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Currently it could be only package private. I am not sure whether it could be used outside of package future.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
it could be potentially used outside of this package in other shuffle implementation. I would leave methods public here because it is basically part of shuffle API then.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I would recommend following the conservative approach Zhijiang proposed and make things only public if they are really needed. Decreasing visibility is always much harder than increasing it.
|
|
||
| return new PartitionShuffleDescriptor( | ||
| resultId, partitionId, partitionType, numberOfSubpartitions, maxParallelism, | ||
| partition.getIntermediateResult().getConnectionIndex()); |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I am not sure whether it is worth defining a private connectionIndex var in front for partition.getIntermediateResult().getConnectionIndex() to keep the same way as other vars.
| } | ||
|
|
||
| @Nonnull | ||
| public ShuffleDeploymentDescriptor getShuffleDeploymentDescriptor() { |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
package private
| private final ConnectionID producerConnection; | ||
|
|
||
| @Nonnull | ||
| private final ResultPartitionID resultPartitionID; |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Maybe better to name resultPartitionId consistent with producerResourceId
| import java.util.List; | ||
| import java.util.Map; | ||
|
|
||
| /** Factory of {@link TaskDeploymentDescriptor} to deploy {@link Execution}. */ |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
The annotation format /** */ seems mainly used in variable not in class. I am not very sure of it.
zhijiangW
left a comment
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Thanks for opening this PR and it seems really good @azagrebin .
I just left some minor inline comments and thought of two concerns currently. I have not finished the whole review and would continue on it.
-
The class
PartitionInfoseems a bit redundant which might be replaced byInputGateDeploymentDescriptor(IGDD)completely. Because they cover most of the same informations, onlyResultPartitionTypeandconsumedSubpartitionIndexinIGDDseems not necessary forPartitionInfonow, but theResultPartitionTypemight also make sense inPartitionInfoif we support the dynamically determine the type future. The currentTaskExecutor#updatePartitions(IGDD)could also work well to make a bit change. And it seems also consistent to use the same structure during deploying and updating. -
I am not sure it is a good way for using
instanceofinSingleInputGateto check the instance ofShuffleDeploymentDescriptorand make the transformation. Another option is we define some methods in the interfaceShuffleDeploymentDescriptor, such asboolean isUnknown(),getResultPartitionID(),getConnectionID()to avoid this.
|
Thanks for the review @zhijiangW ! I have addressed smaller comments.
|
|
Thanks for the confirmation @azagrebin .
BTW, I have not finished the whole review yet. I would continue on it later. |
tillrohrmann
left a comment
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Thanks for opening this PR @azagrebin. I gave it a first pass mainly having minor comments (many style comments which do not affect correctness). I will give it another pass to look in detail at the abstractions. I will post comments regarding the abstractions next.
| @Nonnull ResultPartitionType consumedPartitionType, | ||
| @Nonnegative int consumedSubpartitionIndex, | ||
| @Nonnull ShuffleDeploymentDescriptor[] inputChannels, | ||
| @Nonnull ResourceID consumerResourceId) { |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Flink's code base uses double indentation to distinguish the parameters from the function body.
| } | ||
|
|
||
| public static TaskDeploymentDescriptorFactory fromExecutionVertex( | ||
| ExecutionVertex executionVertex, ExecutionAttemptID executionId, int attemptNumber) { |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
If we break the parameter list, then I would suggest to break all parameters.
| JobID jobID, | ||
| boolean lazyScheduling, | ||
| int subtaskIndex, | ||
| ExecutionEdge[][] inputEdges) { |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
It's not super consistent in the Flink code base but newer code tries to indent parameters lists an additional level to distinguish them from the method body.
| this.inputEdges = inputEdges; | ||
| } | ||
|
|
||
| public static TaskDeploymentDescriptorFactory fromExecutionVertex( |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Usually static functions go to the bottom of the class. The order is roughly
Static fields;
Field;
Constructors;
Methods;
Static functions;
|
|
||
| /** | ||
| * Creates a task deployment descriptor to deploy a subtask to the given target slot. | ||
| */ |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
These JavaDocs don't add much information.
| } | ||
|
|
||
| private TaskDeploymentDescriptor createReceiver( | ||
| DefaultShuffleDeploymentDescriptor sdd, ResourceID location) throws IOException { |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
formatting
|
|
||
| PartitionShuffleDescriptor psd = new PartitionShuffleDescriptor( | ||
| new IntermediateDataSetID(), sdd.getResultPartitionID().getPartitionId(), | ||
| ResultPartitionType.PIPELINED, 1, 1, 0); |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
formatting
| } | ||
|
|
||
| public static InputGateDeploymentDescriptor createInputGateDeploymentDescriptor( | ||
| ShuffleDeploymentDescriptor sdd, ResourceID consumerLocation) { |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
line breaks
| ResultPartitionType.PIPELINED_BOUNDED, | ||
| channel, | ||
| channelDescriptors); | ||
| channelDescriptors, localLocation); |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
line break
| } | ||
| } | ||
|
|
||
| private static ShuffleDeploymentDescriptor createLocalSdd(ResultPartitionID resultPartitionID, ResourceID location) { |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Could ShuffleTestUtils#createSddWithLocalConnection be used here?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
we can also use NettyShuffleDescriptorBuilder here.
tillrohrmann
left a comment
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I like the abstractions you've put in place @azagrebin. Good work! I've had a couple of minor comments.
| /** | ||
| * Default implementation of {@link ShuffleDeploymentDescriptor} for {@link DefaultShuffleMaster}. | ||
| */ | ||
| public class DefaultShuffleDeploymentDescriptor implements ShuffleDeploymentDescriptor { |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Serial version UID is missing. I recommend to activate IntelliJ's inspections which will mark this as an error.
| /** | ||
| * Partition producer descriptor for {@link ShuffleMaster} to obtain {@link ShuffleDeploymentDescriptor}. | ||
| */ | ||
| public class ProducerShuffleDescriptor { |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
We could rename this class into ProducerDescriptor because it is already part of the shuffle package.
| /** | ||
| * Partition descriptor for {@link ShuffleMaster} to obtain {@link ShuffleDeploymentDescriptor}. | ||
| */ | ||
| public class PartitionShuffleDescriptor implements Serializable { |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
We could do the same here with PartitionShuffleDescriptor --> PartitionDescriptor
| * Intermediate result partition registry to use in {@link org.apache.flink.runtime.jobmaster.JobMaster}. | ||
| */ | ||
| public interface ShuffleMaster { | ||
| CompletableFuture<ShuffleDeploymentDescriptor> registerPartitionWithProducer( |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Could think about introducing a T extends ShuffleDeploymentDescriptor and then letting the specific implementations define the T. This could avoid type casting when testing the implementations.
| this.intermediateDataSetID = Preconditions.checkNotNull(intermediateResultPartitionID); | ||
| this.inputChannelDeploymentDescriptor = Preconditions.checkNotNull(inputChannelDeploymentDescriptor); | ||
| @Nonnull | ||
| private final ResourceID consumerResourceID; |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I think this information should come from the TaskExecutor where it is used to update the channels.
| /** | ||
| * Default {@link ShuffleMaster} for netty and local file based shuffle implementation. | ||
| */ | ||
| public class DefaultShuffleMaster implements ShuffleMaster { |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Shall we name this class and all related classes directly NettyShuffleMaster?
| Execution producer = consumedPartition.getProducer().getCurrentExecutionAttempt(); | ||
| Map<IntermediateResultPartitionID, ResultPartitionDeploymentDescriptor> producedPartitions = | ||
| producer.getProducedPartitions(); | ||
| Preconditions.checkArgument(checkInputReady(consumedPartition.getPartitionId(), producedPartitions), |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
checkState would fit better here I think.
|
|
||
| public Map<IntermediateResultPartitionID, ResultPartitionDeploymentDescriptor> getProducedPartitions() { | ||
| return producedPartitions; | ||
| } |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
We could hide this implementation detail by offering a Optional<ResultPartitionDeploymentDescriptor> getProducedPartition(IntermediateResultPartitionID intermediateResultPartitionId)
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
The return type could even be Optional<ShuffleDeploymentDescriptor> if I'm not mistaken.
| /** | ||
| * Unknown {@link ShuffleDeploymentDescriptor}. | ||
| */ | ||
| public final class UnknownShuffleDeploymentDescriptor implements ShuffleDeploymentDescriptor { |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Missing serial version UID
| @Nonnull | ||
| private final ResourceID producerResourceId; | ||
|
|
||
| /** The address to use to request the remote partition. */ |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
For external shuffle service implementations, this does not necessarily be true.
8a6079d to
cdf56b1
Compare
| * Intermediate result partition registry to use in {@link org.apache.flink.runtime.jobmaster.JobMaster}. | ||
| */ | ||
| public interface ShuffleMaster<T extends ShuffleDescriptor> { | ||
| CompletableFuture<T> registerPartitionWithProducer( |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
missing javadoc, in particular for the return value
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I will add class parameter description to class level javadoc, not sure method comment will add too much value to it. The method name already says basically what is happening, unless you think that some specific detail should be mentioned.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I think interface methods should get JavaDocs especially if they are public API. Since we want to offer external shuffle service implementations this is the case.
| @Nonnegative int consumedSubpartitionIndex, | ||
| ShuffleDescriptor[] inputChannels, | ||
| ResourceID consumerLocation) { | ||
| this.consumedResultId = consumedResultId; |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
keep checkNotNull for these parameters?
| * Interface for shuffle deployment descriptor of result partition resource. | ||
| */ | ||
| public interface ShuffleDescriptor extends Serializable { | ||
| ResultPartitionID getResultPartitionID(); |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Add empty line before this method?
| * <p>It can be used e.g. to compare with partition producer {@link ResourceID} in | ||
| * {@link ProducerDescriptor} to determine producer/consumer co-location. | ||
| */ | ||
| private final ResourceID consumerLocation; |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
consumerLocation -> consumerResourceId? Because in ProducerDescriptor or NettyShuffleDescriptor, we also name producerResourceId
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
actually I think it is better to rename it in ProducerDescriptor & NettyShuffleDescriptor
| * Intermediate result partition registry to use in {@link org.apache.flink.runtime.jobmaster.JobMaster}. | ||
| */ | ||
| public interface ShuffleMaster<T extends ShuffleDescriptor> { | ||
| CompletableFuture<T> registerPartitionWithProducer( |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
empty line before method?
| */ | ||
| private final int dataPort; | ||
|
|
||
| public ProducerDescriptor( |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
| ShuffleDescriptor remoteSdd = createRemoteWithIdAndLocation( | ||
| remoteResultPartitionId.getPartitionId(), | ||
| ResourceID.generate()); | ||
| inputGate.updateInputChannel(localLocation, new PartitionInfo(new IntermediateDataSetID(), remoteSdd)); |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
ditto: reuse createPartitionInfo()
| ShuffleDescriptor sdd = createRemoteWithIdAndLocation( | ||
| resultPartitionId.getPartitionId(), | ||
| ResourceID.generate()); | ||
| inputGate.updateInputChannel(ResourceID.generate(), new PartitionInfo(new IntermediateDataSetID(), sdd)); |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
could reuse createPartitionInfo(resultPartitionId, ResourceID.generate()) to create PartitionInfo here.
| } | ||
|
|
||
| public static NettyShuffleDescriptor createRemoteWithIdAndLocation( | ||
| IntermediateResultPartitionID partitionId, |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
add one more indentation for parameters
| Collection<InputGateDeploymentDescriptor> inputGates) throws Exception { | ||
| Collection<ResultPartitionDeploymentDescriptor> resultPartitions, | ||
| Collection<InputGateDeploymentDescriptor> inputGates) throws Exception { | ||
| String errorMessage = "Network buffer pool has already been destroyed."; |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
not necessary changes for this method.
|
|
||
| private TaskDeploymentDescriptor createReceiver( | ||
| NettyShuffleDescriptor shuffleDescriptor, | ||
| ResourceID location) throws IOException { |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I guess location is not needed in the parameter which could be created directly inside.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
some tests rely on it to be the same as in some other components outside of this method.
| slotProvider, | ||
| new NoRestartStrategy(), | ||
| jobVertex); | ||
| executionGraph.start(TestingComponentMainThreadExecutorServiceAdapter.forMainThread()); |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
why need this change?
| public void testSendCancelAndReceiveFail() throws Exception { | ||
| final ExecutionGraph graph = ExecutionGraphTestUtils.createSimpleTestGraph(); | ||
|
|
||
| graph.start(TestingComponentMainThreadExecutorServiceAdapter.forMainThread()); |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
ditto
zhijiangW
left a comment
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Thanks for the updates @azagrebin !
I have finished the whole reviews. Sorry for intermediate suspend reviews these days.
|
Thanks for the review @zhijiangW ! I have addressed the comments. |
34a400a to
0b09ef0
Compare
tillrohrmann
left a comment
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Thanks for addressing my comments @azagrebin. I had a couple of additional comments. Once these are resolved I think we are good to merge this PR :-)
| private final ShuffleDescriptor[] inputChannels; | ||
|
|
||
| /** | ||
| * {@link ResourceID} of partition consume to identify its location. |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
typo: consume -> consumer
| * <p>It can be used e.g. to compare with partition producer {@link ResourceID} in | ||
| * {@link ProducerDescriptor} to determine producer/consumer co-location. | ||
| */ | ||
| private final ResourceID consumerLocation; |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Why do we need this field? I thought that the InputGateDeploymentDescriptor is being used to create an InputGate on the TaskExecutor. If this is the case, then the information about the TaskExecutor's ResourceID should already be there. No need to transmit this additional information.
| } | ||
|
|
||
| public InputChannelDeploymentDescriptor[] getInputChannelDeploymentDescriptors() { | ||
| public ShuffleDescriptor[] getInputChannelDescriptors() { |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Better to call getShuffleDescriptors
| return inputChannels; | ||
| } | ||
|
|
||
| public ResourceID getConsumerLocation() { |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
It looks as if this method + the field were introduced to make the creation of the InputGate a bit more convenient. I'm wondering whether this is not the wrong place to get this information from. I think it should come from the TaskExecutor.
| new TaskDeploymentDescriptor.Offloaded<>(taskInfo.right()); | ||
| } | ||
|
|
||
| private List<InputGateDeploymentDescriptor> createInputGateDeploymentDescriptors(ResourceID location) { |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Here we are mixing static methods with methods. Feels a bit weird given that we us this function in createDeploymentDescriptor
| * Intermediate result partition registry to use in {@link org.apache.flink.runtime.jobmaster.JobMaster}. | ||
| */ | ||
| public interface ShuffleMaster<T extends ShuffleDescriptor> { | ||
| CompletableFuture<T> registerPartitionWithProducer( |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I think interface methods should get JavaDocs especially if they are public API. Since we want to offer external shuffle service implementations this is the case.
| /** | ||
| * Tests for the {@link ResultPartitionDeploymentDescriptor}. | ||
| */ | ||
| public class ResultPartitionDeploymentDescriptorTest { |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Missing extends TestLogger
| createCopyAndVerifyResultPartitionDeploymentDescriptor(shuffleDescriptor); | ||
|
|
||
| assertThat(copy.getShuffleDescriptor(), instanceOf(UnknownShuffleDescriptor.class)); | ||
| UnknownShuffleDescriptor copySdd = (UnknownShuffleDescriptor) copy.getShuffleDescriptor(); |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
why is this cast needed?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
leftover, will remove
| } | ||
|
|
||
| private TaskDeploymentDescriptor createSender( | ||
| NettyShuffleDescriptor shuffleDeploymentDescriptor, |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Naming of this parameter is not consistent with createReceiver.
| /** | ||
| * Builder to mock {@link NettyShuffleDescriptor} in tests. | ||
| */ | ||
| public class NettyShuffleDescriptorBuilder { |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Nice :-)
6a40912 to
4820219
Compare
tillrohrmann
left a comment
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Thanks for addressing my comments @azagrebin. LGTM. Merging this PR once Travis gives green light.
|
Thanks for the review @tillrohrmann @zhijiangW ! |
c50c30d to
669df9e
Compare
Introduce PartitionLocation in NettyShuffleDescriptor and NettyShuffleDescriptorBuilder for tests Add ShuffleDescriptor.getResultPartitionID and isUnknown Use NettyShuffleDescriptorBuilder in StreamNetworkBenchmarkEnvironment Introduce ShuffleUtils.applyWithShuffleTypeCheck to isolate inout channel shuffle descriptor 'known' check and cast This closes apache#8362.
Introduce PartitionLocation in NettyShuffleDescriptor and NettyShuffleDescriptorBuilder for tests Add ShuffleDescriptor.getResultPartitionID and isUnknown Use NettyShuffleDescriptorBuilder in StreamNetworkBenchmarkEnvironment Introduce ShuffleUtils.applyWithShuffleTypeCheck to isolate inout channel shuffle descriptor 'known' check and cast This closes apache#8362.
669df9e to
f7014b7
Compare
|
Failing test case seems to be unrelated. Merging this PR now. |
What is the purpose of the change
This PR introduces ShuffleMaster interface. JobMaster and ExecutionGraph use ShuffleMaster to register resources for produced partition of intermediate results. This is part of the overall effort to make shuffle implementation pluggable.
Brief change log
to hand over to task and its local shuffle service
when producer is unknown yet
PartitionDescriptor and ShuffleDeploymentDescriptor
instead of InputChannelDeploymentDescriptor
for existing shuffle service based on Netty communication and local files
consumer resource id and producer id from NettyShuffleDescriptor
Verifying this change
unit tests
Does this pull request potentially affect one of the following parts:
@Public(Evolving): (no)Documentation