-
Notifications
You must be signed in to change notification settings - Fork 13.9k
[FLINK-12960] Introduce ShuffleDescriptor#ReleaseType and ShuffleDescriptor#getSupportedReleaseTypes #8857
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
[FLINK-12960] Introduce ShuffleDescriptor#ReleaseType and ShuffleDescriptor#getSupportedReleaseTypes #8857
Changes from all commits
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -24,6 +24,7 @@ | |
|
|
||
| import java.io.Serializable; | ||
| import java.util.Collection; | ||
| import java.util.EnumSet; | ||
| import java.util.Optional; | ||
|
|
||
| /** | ||
|
|
@@ -67,4 +68,32 @@ default boolean isUnknown() { | |
| * @return the resource id of the producing task executor if the partition occupies local resources there | ||
| */ | ||
| Optional<ResourceID> storesLocalResourcesOn(); | ||
|
|
||
| /** | ||
| * Return release types supported by Shuffle Service for this partition. | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. remove
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. why? if we leave
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. There might exist two options:
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. The
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Yes, I have no other concerns here now. |
||
| */ | ||
| EnumSet<ReleaseType> getSupportedReleaseTypes(); | ||
|
|
||
| /** | ||
| * Partition release type. | ||
| */ | ||
| enum ReleaseType { | ||
| /** | ||
| * Auto-release the partition after having been fully consumed once. | ||
| * | ||
| * <p>No additional actions required, like {@link ShuffleMaster#releasePartitionExternally(ShuffleDescriptor)} | ||
| * or {@link ShuffleEnvironment#releasePartitionsLocally(Collection)} | ||
| */ | ||
| AUTO, | ||
|
|
||
| /** | ||
| * Manually release the partition, the partition has to support consumption multiple times. | ||
| * | ||
| * <p>The partition requires manual release once all consumption is done: | ||
| * {@link ShuffleMaster#releasePartitionExternally(ShuffleDescriptor)} and | ||
| * if the partition occupies producer local resources ({@link #storesLocalResourcesOn()}) then also | ||
| * {@link ShuffleEnvironment#releasePartitionsLocally(Collection)}. | ||
| */ | ||
| MANUAL | ||
| } | ||
| } | ||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -19,6 +19,7 @@ | |
| package org.apache.flink.runtime.shuffle; | ||
|
|
||
| import org.apache.flink.runtime.deployment.ResultPartitionDeploymentDescriptor; | ||
| import org.apache.flink.runtime.shuffle.ShuffleDescriptor.ReleaseType; | ||
|
|
||
| import java.util.Collection; | ||
| import java.util.concurrent.CompletableFuture; | ||
|
|
@@ -51,6 +52,7 @@ CompletableFuture<T> registerPartitionWithProducer( | |
| * <p>This call triggers release of any resources which are occupied by the given partition in the external systems | ||
| * outside of the producer executor. This is mostly relevant for the batch jobs and blocking result partitions. | ||
| * This method is not called if {@link ResultPartitionDeploymentDescriptor#isReleasedOnConsumption()} is {@code true}. | ||
| * The partition has to support the {@link ReleaseType#MANUAL} in {@link ShuffleDescriptor#getSupportedReleaseTypes()}. | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. In theory different
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. The goal of PR is to let shuffle services decide what to support in Or do you mean, that we implicitly assume at the moment that
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Sorry I have not noticed that this javadoc change was for the method of |
||
| * The producer local resources are managed by {@link ShuffleDescriptor#storesLocalResourcesOn()} and | ||
| * {@link ShuffleEnvironment#releasePartitionsLocally(Collection)}. | ||
| * | ||
|
|
||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -27,8 +27,10 @@ | |
| import org.apache.flink.runtime.jobgraph.IntermediateDataSetID; | ||
| import org.apache.flink.runtime.jobgraph.IntermediateResultPartitionID; | ||
| import org.apache.flink.runtime.shuffle.NettyShuffleDescriptor; | ||
| import org.apache.flink.runtime.shuffle.NettyShuffleDescriptor.NetworkPartitionConnectionInfo; | ||
| import org.apache.flink.runtime.shuffle.PartitionDescriptor; | ||
| import org.apache.flink.runtime.shuffle.ShuffleDescriptor; | ||
| import org.apache.flink.runtime.shuffle.ShuffleDescriptor.ReleaseType; | ||
| import org.apache.flink.runtime.shuffle.UnknownShuffleDescriptor; | ||
| import org.apache.flink.runtime.util.NettyShuffleDescriptorBuilder; | ||
| import org.apache.flink.util.TestLogger; | ||
|
|
@@ -72,13 +74,9 @@ public class ResultPartitionDeploymentDescriptorTest extends TestLogger { | |
| * Tests simple de/serialization with {@link UnknownShuffleDescriptor}. | ||
| */ | ||
| @Test | ||
| public void testSerializationWithUnknownShuffleDescriptor() throws Exception { | ||
| public void testSerializationOfUnknownShuffleDescriptor() throws IOException { | ||
| ShuffleDescriptor shuffleDescriptor = new UnknownShuffleDescriptor(resultPartitionID); | ||
|
|
||
| ResultPartitionDeploymentDescriptor copy = | ||
| createCopyAndVerifyResultPartitionDeploymentDescriptor(shuffleDescriptor); | ||
|
|
||
| ShuffleDescriptor shuffleDescriptorCopy = copy.getShuffleDescriptor(); | ||
| ShuffleDescriptor shuffleDescriptorCopy = CommonTestUtils.createCopySerializable(shuffleDescriptor); | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. this change should be a separate hotfix |
||
| assertThat(shuffleDescriptorCopy, instanceOf(UnknownShuffleDescriptor.class)); | ||
| assertThat(shuffleDescriptorCopy.getResultPartitionID(), is(resultPartitionID)); | ||
| assertThat(shuffleDescriptorCopy.isUnknown(), is(true)); | ||
|
|
@@ -88,11 +86,12 @@ public void testSerializationWithUnknownShuffleDescriptor() throws Exception { | |
| * Tests simple de/serialization with {@link NettyShuffleDescriptor}. | ||
| */ | ||
| @Test | ||
| public void testSerializationWithNettyShuffleDescriptor() throws Exception { | ||
| public void testSerializationWithNettyShuffleDescriptor() throws IOException { | ||
| ShuffleDescriptor shuffleDescriptor = new NettyShuffleDescriptor( | ||
| producerLocation, | ||
| new NettyShuffleDescriptor.NetworkPartitionConnectionInfo(connectionID), | ||
| resultPartitionID); | ||
| new NetworkPartitionConnectionInfo(connectionID), | ||
| resultPartitionID, | ||
| false); | ||
|
|
||
| ResultPartitionDeploymentDescriptor copy = | ||
| createCopyAndVerifyResultPartitionDeploymentDescriptor(shuffleDescriptor); | ||
|
|
@@ -110,7 +109,7 @@ public void testReleasedOnConsumptionFlag() { | |
| for (ResultPartitionType partitionType : ResultPartitionType.values()) { | ||
| ResultPartitionDeploymentDescriptor partitionDescriptor = new ResultPartitionDeploymentDescriptor( | ||
| new PartitionDescriptor(resultId, partitionId, partitionType, numberOfSubpartitions, connectionIndex), | ||
| NettyShuffleDescriptorBuilder.newBuilder().buildLocal(), | ||
| NettyShuffleDescriptorBuilder.newBuilder().setBlocking(partitionType.isBlocking()).buildLocal(), | ||
| 1, | ||
| true | ||
| ); | ||
|
|
@@ -123,6 +122,16 @@ public void testReleasedOnConsumptionFlag() { | |
| } | ||
| } | ||
|
|
||
| @Test(expected = IllegalArgumentException.class) | ||
| public void testIncompatibleReleaseTypeManual() { | ||
| new ResultPartitionDeploymentDescriptor( | ||
| partitionDescriptor, | ||
| NettyShuffleDescriptorBuilder.newBuilder().setBlocking(false).buildLocal(), | ||
| 1, | ||
| true, | ||
| ReleaseType.MANUAL); | ||
| } | ||
|
|
||
| private static ResultPartitionDeploymentDescriptor createCopyAndVerifyResultPartitionDeploymentDescriptor( | ||
| ShuffleDescriptor shuffleDescriptor) throws IOException { | ||
| ResultPartitionDeploymentDescriptor orig = new ResultPartitionDeploymentDescriptor( | ||
|
|
||
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 it might be better to indicate which value true/false should support which
releaseType, otherwise it seems not clear the mapping between this value and release type.