Skip to content
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-12331] Introduce partition/gate setup to decouple task registration with NetworkEnvironment #8416

Merged
merged 10 commits into from May 22, 2019

Conversation

azagrebin
Copy link
Contributor

What is the purpose of the change

Introduce partition/gate setup to decouple task registration with NetworkEnvironment.
The PR also does some preparation refactoring.

Brief change log

  • Introduce factories for ResultPartion and SingleInputgates
  • Introduce buffer pool factories for ResultPartion and SingleInputgates
  • Introduce MemorySegmentProvider for RemoteInputChannel to assign exclusive segments
  • Refactor NetworkEnvironment#setupXXX() to ResultPartitionWriter#setup() and InputGate#setup()

Verifying this change

existing unit tests.

Does this pull request potentially affect one of the following parts:

  • Dependencies (does it add or upgrade a dependency): (no)
  • The public API, i.e., is any changed class annotated with @Public(Evolving): (no)
  • The serializers: (no)
  • The runtime per-record code paths (performance sensitive): (no)
  • Anything that affects deployment or recovery: JobManager (and its components), Checkpointing, Yarn/Mesos, ZooKeeper: (no)
  • The S3 file system connector: (no)

Documentation

  • Does this pull request introduce a new feature? (no)
  • If yes, how is the feature documented? (not applicable)

@flinkbot
Copy link
Collaborator

flinkbot commented May 10, 2019

Thanks a lot for your contribution to the Apache Flink project. I'm the @flinkbot. I help the community
to review your pull request. We will use this comment to track the progress of the review.

Review Progress

  • ❓ 1. The [description] looks good.
  • ❓ 2. There is [consensus] that the contribution should go into to Flink.
  • ❗ 3. Needs [attention] from.
  • ❓ 4. The change fits into the overall [architecture].
  • ❓ 5. Overall code [quality] is good.

Please see the Pull Request Review Guide for a full explanation of the review process.


The 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 commands
The @flinkbot bot supports the following commands:

  • @flinkbot approve description to approve one or more aspects (aspects: description, consensus, architecture and quality)
  • @flinkbot approve all to approve all aspects
  • @flinkbot approve-until architecture to approve everything until architecture
  • @flinkbot attention @username1 [@username2 ..] to require somebody's attention
  • @flinkbot disapprove architecture to remove an approval you gave earlier

@azagrebin
Copy link
Contributor Author

@flinkbot attention @zhijiangW

@@ -108,6 +112,7 @@ private NetworkEnvironment(
this.taskEventPublisher = taskEventPublisher;
this.isShutdown = false;
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Put isShutdown=false at the end of this constructor?

@@ -128,13 +133,17 @@ public static NetworkEnvironment create(
ResultPartitionFactory resultPartitionFactory =
new ResultPartitionFactory(resultPartitionManager, checkNotNull(ioManager));

SingleInputGateFactory singleInputGateFactory =
new SingleInputGateFactory(config, connectionManager, resultPartitionManager);
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

TaskEventDispatcher could also be included in singleInputGateFactory?

InputChannelBuilder.newBuilder()
.setupFromNetworkEnvironment(network)
.buildUnknown(inputGate)
.toRemoteInputChannel(InputChannelBuilder.STUB_CONNECTION_ID);
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

why not build remote directly?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I tried to preserve the previous way of building remote but looking at it more, true, it does not seem to be relevant.

.buildUnknown(inputGate)
.toRemoteInputChannel(InputChannelBuilder.STUB_CONNECTION_ID);
IntermediateResultPartitionID resultPartitionId = remoteFromUnknown.getPartitionId().getPartitionId();
inputGate.setInputChannel(resultPartitionId, remoteFromUnknown);
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

setInputChannel could be removed?


network.setupInputGate(inputGate);

NetworkBufferPool bufferPool = network.getNetworkBufferPool();
if (enableCreditBasedFlowControl) {
RemoteInputChannel remote = (RemoteInputChannel) inputGate.getInputChannels()
.get(resultPartitionId.getPartitionId());
RemoteInputChannel remote = (RemoteInputChannel) inputGate.getInputChannels().get(resultPartitionId);
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Use the previous remoteFromUnknown directly, so IntermediateResultPartitionID resultPartitionId = remoteFromUnknown.getPartitionId().getPartitionId() could also be removed.

/** Requests default number of memory segments. */
Collection<MemorySegment> requestMemorySegments() throws IOException;

Collection<MemorySegment> requestMemorySegments(int numRequiredBuffers) throws IOException;
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

If we only consider the semantics of MemorySegmentProvider interface, the requestMemorySegments() is a bit redundant, which could be replaced by requestMemorySegments(int numRequiredBuffers). The only reason we keep both methods is that it seems not elegant to maintain networkBuffersPerChannel in SingleInputGate.

TBH, I do not think it is worth defining this new interface not very clean to accommodate the implementation. Because requestMemorySegments() seems blurry for the usage which relies on the default value, and we should also consider if the default value is 0, whether the returned collection is null or empty size.
In addition, requestMemorySegments(int numRequiredBuffers) is only used for tests ATM.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I think MemorySegmentProvider makes sense from point of interface segregation principle. This interface encapsulates segment management concern which is the only thing that remote channel needs atm. NetworkBufferPool looks like implementing 2 separate concerns: BufferPoolFactory and MemorySegmentProvider.

As for requestMemorySegments(int numRequiredBuffers), you are right, it is more clear to remove it from the interface and adjust the tests with explicit default value, e.g. 1.

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

We are indeed in the lack of semantics of requesting/releasing batch of buffers before, and I think it is reasonable to extend these methods.

  • SegmentProvider is not really needed because RemoteInputChannel still needs construct the Buffer internally after requesting MemorySegment.

  • We could extend the general BufferProvider interface to provide method of requestBuffers(int numRequiredBuffers, BufferRecycler), and RemoteInputChannel already implements BufferRecycler interface.

  • We could also extend the general BufferRecycler interface to provide method of recycle(Collection<MemorySegment>).

To do so we make use of the existing semantics and architecture. But the only concern is we would affect the previous implementations to bring more changes.

@@ -155,8 +155,8 @@
*/
private BufferPool bufferPool;

/** Global network buffer pool to request and recycle exclusive buffers (only for credit-based). */
private NetworkBufferPool networkBufferPool;
/** Global global memory segment provider to request and recycle exclusive buffers (only for credit-based). */
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

repeated global

@@ -535,7 +510,7 @@ public void requestPartitions() throws IOException, InterruptedException {
throws IOException, InterruptedException {
while (true) {
synchronized (inputChannelsWithData) {
while (inputChannelsWithData.size() == 0) {
while (inputChannelsWithData.isEmpty()) {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

why make this change?

metrics
);
metrics,
memorySegmentProvider);
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Whether to separate line for last ); which keeps the consistent with local and unknown cases?

static RemoteInputChannel createRemoteInputChannel(
SingleInputGate inputGate, MemorySegmentProvider memorySegmentProvider) throws Exception {

return createRemoteInputChannel(inputGate, mock(PartitionRequestClient.class), 0, 0, memorySegmentProvider);
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

It is better to avoid mock PartitionRequestClient here.

@@ -237,7 +254,8 @@ static RemoteInputChannel createRemoteInputChannel(
SingleInputGate inputGate,
PartitionRequestClient client,
int initialBackoff,
int maxBackoff) throws Exception {
int maxBackoff,
MemorySegmentProvider memorySegmentProvider) throws Exception {
final ConnectionManager connectionManager = mock(ConnectionManager.class);
when(connectionManager.createPartitionRequestClient(any(ConnectionID.class)))
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

There are already too many utils here to construct the RemoteInputChannel, but we still miss the units of {SingleInputGate, PartitionRequestClient, MemorySegmentProvider} which could avoid pass createRemoteInputChannel(inputGate, client, 0, 0, networkBufferPool) in CreditBasedPartitionRequestClientHandlerTest.

Maybe we need a separate hotfix for refactoring these parts to make connectionManager.createPartitionRequestClient call in separate tests if necessary, then we only need one type to construct RemoteInputChannel.

@@ -46,6 +48,7 @@
private int maxBackoff = 0;
private InputChannelMetrics metrics =
new InputChannelMetrics(UnregisteredMetricGroups.createUnregisteredTaskMetricGroup());
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Here we could use InputChannelTestUtils.newUnregisteredInputChannelMetrics

@@ -1010,8 +1032,9 @@ private RemoteInputChannel createRemoteInputChannel(

return InputChannelBuilder.newBuilder()
.setConnectionManager(connectionManager)
.setInitialBackoff(initialBackoff)
.setInitialBackoff(initialBackOff)
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

No need change?

private RemoteInputChannel createRemoteInputChannel(
SingleInputGate inputGate,
PartitionRequestClient partitionRequestClient,
int initialBackOff,
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

lowercase off to keep consistent with maxBackoff?

throws IOException, InterruptedException {

return createRemoteInputChannel(
inputGate, mock(PartitionRequestClient.class), 0, 0, memorySegmentProvider);
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

better avoid mock PartitionRequestClient.

The same issue for creating RemoteInputChannel in various units, if we do not need unify the createPartitionRequestClient with the constructing of RemoteInputChannel, all these createRemoteInputChannel could be covered by one InputChannelBuilder.

checkArgument(bufferPool.getNumberOfRequiredMemorySegments() >= getNumberOfSubpartitions(),
"Bug in result partition setup logic: Buffer pool has not enough guaranteed buffers for this result partition.");
"Bug in result partition setup logic: Buffer pool has not enough guaranteed buffers for this result partition.");

checkState(this.bufferPool == null, "Bug in result partition setup logic: Already registered buffer pool.");
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

this check should be put before resultPartitionBufferPoolFactory.create

@@ -836,6 +833,21 @@ else if (transitionState(current, ExecutionState.FAILED, t)) {
}
}

private void setupPartionsAndGates() throws IOException {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

why not remove this method to use the following static directly?

@@ -262,7 +268,29 @@ public void testExecutionFailsInBlobsMissing() throws Exception {
}

@Test
public void testExecutionFailsInNetworkRegistration() throws Exception {
public void testExecutionFailsInNetworkRegistrationFprPartitions() throws Exception {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Fpr->For

Copy link
Contributor

@zhijiangW zhijiangW left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Thanks for the work for these refactoring @azagrebin .

For the commits arrangement I have two concerns:

  • The hotfix of Introduce NetworkEnvironment.create factory should be put firstly because the other ones are relying on it. And the motivation for introducing static creation is not for tests purpose. I think it avoids the complex logics of constructing the factories in the constructor which could initialize the vars directly in easy way.

  • The hotfix of Introduce ResultPartitionBufferPoolFactory and Introduce InputGateBufferPoolFactory could be squashed with the last two setup commits, because they are only the temporary states, not final ones.

In addition, I have some other concerns for the motivation of introducing related factories:

  • PartitionBufferPoolFactory: I think it is no need to define this new factory for creating BufferPool, because we already have BufferPoolFactory for doing this, and the NetworkBufferPool implementation is just used for creating BufferPool for partition/gate. We might extend the BufferPoolFactory#createBufferPool to achieve the same motivation and make SingleInputGate/ResultPartition only sees the BufferPoolFactory instead of NetworkBufferPool. The new introduced PartitionBufferPoolFactory would make the relationship more complex and difficult to distinguish the semantic scope. In other words, I do not think two ShuffleService implementations would create the same SingleInputGate/ResultPartition in future. So even though the SingleInputGate sees the NetworkBufferPool which belongs to NetworkEnvironment ATM, it still might make sense, or we could break this tie if necessary in future.

  • ResultPartitionFactory: It seems reasonable to add the factory layer between ShuffleService and ResultPartitionWriter as one option. The current factory is just wrapping some components in NetworkEnvironment, but these components are still needed to be maintained separately in NetworkEnvironment for other usages. So I am not sure the specific advantages from the architecture view to do so except for the form view.


import java.io.IOException;

/** factory of {@link BufferPool} for network partitions or channels. */
Copy link
Contributor

@zhijiangW zhijiangW May 13, 2019

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This comment annotation /** */ is used for variable, and the class should be like following

/**
 * 
 */

I am not sure of it.

@@ -215,10 +215,9 @@ private void testReleaseMemory(final ResultPartitionType resultPartitionType) th
final int numAllBuffers = 10;
final NetworkEnvironment network = new NetworkEnvironmentBuilder()
.setNumNetworkBuffers(numAllBuffers).build();
final ResultPartitionConsumableNotifier notifier = new NoOpResultPartitionConsumableNotifier();
final ResultPartition resultPartition = createPartition(notifier, resultPartitionType, false);
final ResultPartition resultPartition = createPartition(network, resultPartitionType, 1);
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Actually the default value 1 is not needed, just because we could not give all the possible units of parameters in PartitionTestUtils. So it might be better to use ResultPartitionBuilder directly to delete intermediate PartitionTestUtils as I mentioned above.

config.isCreditBased(),
config.networkBuffersPerChannel(),
config.floatingNetworkBuffersPerGate(),
numberOfChannels, partitionType);
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

separate line for numberOfChannels and partitionType?

*/
public class ResultPartitionBuilder {

private JobID jobId = new JobID();
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

should be final for these vars.

@zhijiangW
Copy link
Contributor

Thanks for the updates @azagrebin ! It seems much better now.

I have other two concerns:

  • It is really nice now to introduce some builders for constructing ResultPartition, SingleInputGate and InputChannel. But there still exits various utils in tests to support various of parameter units for creating these items. It might bring unnecessary changes every time when make changes for partition/gate/channel. Even these scattered utils make hard to maintain and still could not provide everything. So I want to remove all these unnecessary utils in tests to unified use the proposed builder, maybe not in the scope of this PR.

  • I was ever a bit confused of the proposed MemorySegmentProvider interface, because I think the new introduced methods should be extended in existing BufferProvider and BufferRecycler to support the batch of buffer requesting/recycle. But now I think it seems make sense to do as now, because it is very simple to let NetworkBufferPool implement the new interface and NetworkBufferPool actually has the corresponding functions internally before.

@azagrebin
Copy link
Contributor Author

Thanks for the reviews @zentol @zhijiangW , I addressed the comments

@@ -115,7 +115,11 @@ public static NetworkEnvironment create(
TaskEventPublisher taskEventPublisher,
MetricGroup metricGroup,
IOManager ioManager) {
NettyConfig nettyConfig = checkNotNull(config).nettyConfig();
checkNotNull(ioManager);
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

the check at L 136 is now reedundant

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

true, done

Copy link
Contributor

@pnowojski pnowojski left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Thanks for the change. I have left couple of smaller comments. Mostly it LGTM.

I do not see a problems with removing this one extra lock, we should be fine without it.

ResultPartitionFactory resultPartitionFactory,
SingleInputGateFactory singleInputGateFactory) {

NetworkEnvironmentConfiguration config,
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I think you have squashed/amended/fixed ups wrong commits here. Why is this commit changing formatting?

@@ -191,7 +198,7 @@ public void registerTask(Task task) throws IOException {
}

for (final ResultPartition partition : producedPartitions) {
setupPartition(partition);
partition.setup();
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Could you explain more in the commit message of [FLINK-12331][network] Refactor NetworkEnvironment#setupPartition() to ResultPartitionWriter#setup() what is this commit supposed to achieve?

}

@VisibleForTesting
public void setupInputGate(SingleInputGate gate) throws IOException {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Ditto, about commit message: please explain more why & what are you trying to achieve.

@@ -327,7 +327,8 @@ public FairnessVerifyingInputGate(
boolean isCreditBased) {

super(owningTaskName, jobId, consumedResultId, ResultPartitionType.PIPELINED,
consumedSubpartitionIndex, numberOfInputChannels, taskActions, new SimpleCounter(), isCreditBased);
consumedSubpartitionIndex, numberOfInputChannels, taskActions, new SimpleCounter(),
isCreditBased, () -> null);
Copy link
Contributor

@pnowojski pnowojski May 20, 2019

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

null?

either:

  1. if you are not expecting this supplier function to be ever called, throw new UnsupportedOperationException().
  2. if null is supposed to be supported, change it to Optional.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I would prefer throw new UnsupportedOperationException(). There is no need for Optional in production code.

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

In that case throw new UnsupportedOperationException is of course better :)


import org.slf4j.Logger;
import org.slf4j.LoggerFactory;

import javax.annotation.Nonnull;
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

nit: I think you could drop Nonnull. We should follow an implicit assumption that everything is Nonnull unless marked differently.


private int floatingNetworkBuffersPerGate = 1;

private FunctionWithException<BufferPoolOwner, BufferPool, IOException> bufferPoolFactory;
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Optional instead of Nullable

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Is it related to the previous comment about UnsupportedOperationException?

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I think not. This builder is later performing != null check on this field

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

ok, I see, done

@azagrebin
Copy link
Contributor Author

azagrebin commented May 20, 2019

Thanks for the review @pnowojski, I've addressed comments.

Copy link
Contributor

@zhijiangW zhijiangW left a 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 LGTM!

Copy link
Contributor

@pnowojski pnowojski left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

LGTM % address comments commit should be probably squashed with some previous commit(s)

…o ResultPartitionWriter#setup()

Move partition setup from NetworkEnvironment to ResultPartition.
This eliminates tie between Task and NetworkEnvironment.
Task does not need depend on NetworkEnvironment
and can trigger setup from ResultPartitionWriter interface.
…o InputGate#setup()

Move input gate setup from NetworkEnvironment to InputGate.
This eliminates tie between Task and NetworkEnvironment.
Task does not need depend on NetworkEnvironment
and can trigger setup from InputGate directly.
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Projects
None yet
6 participants