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-19026][network] Improve threading model of CheckpointBarrierUnaligner #13228

Merged
merged 19 commits into from Sep 23, 2020
Merged

[FLINK-19026][network] Improve threading model of CheckpointBarrierUnaligner #13228

merged 19 commits into from Sep 23, 2020

Conversation

AHeise
Copy link
Contributor

@AHeise AHeise commented Aug 24, 2020

What is the purpose of the change

Currently, netty threads and task thread are both accessing Unaligner. Netty thread access it indirectly through BufferReceivedListener. Thus, we need some costly synchronization and weaken the threading model of the Unaligner.

This PR mitigates it by letting priority buffers use the same handover protocol as regular buffers. Instead buffers are reordered where needed an additional wake-up mechanism is used for priority events, such that they can interrupt regular buffer processing.

Brief change log

  • Adding specific buffer data type for priority events.
  • Adding priority flags to hand-over protocols (BufferAndAvailability, BufferAndBacklog, InputWithData, ...) directly or indirectly.
  • Reordering buffers, channels, and gates on priority buffers, such that they are pulled before any non-priority buffer.
  • Adding a data structure PriorityDeque to support such reordering.
  • Adding priority notifications on input/output.
  • Handling priority notifications in CheckpointedInputGate (moved out from Unaligner). Generalized hand-over of events from CheckpointedInputGate to StreamTaskNetworkInput, such that all events are forwarded.
  • Removing synchronizations from Unaligner.
  • Moving spill handling from StreamTaskNetworkInput to Unaligner.
  • Removing BufferReceivedListener and letting Unaligner react on in-flight data.

Verifying this change

  • Added/modified a bunch of unit tests.
  • UnalignedCheckpointITCase still covering along with all other ITCases and e2e (using unaligned checkpoints by default).

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

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

Documentation

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

This change is Reviewable

@flinkbot
Copy link
Collaborator

flinkbot commented Aug 24, 2020

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.

Automated Checks

Last check on commit 36c3664 (Fri Feb 19 07:24:38 UTC 2021)

Warnings:

  • No documentation files were touched! Remember to keep the Flink docs up to date!

Mention the bot in a comment to re-run the automated checks.

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

@flinkbot
Copy link
Collaborator

flinkbot commented Aug 24, 2020

CI report:

Bot commands The @flinkbot bot supports the following commands:
  • @flinkbot run travis re-run the last Travis build
  • @flinkbot run azure re-run the last Azure build

@zentol
Copy link
Contributor

zentol commented Aug 25, 2020

CI shows a timeout in {{DataSinkTaskTest.testDataSinkTask}}.

@AHeise AHeise marked this pull request as draft August 25, 2020 12:33
@AHeise
Copy link
Contributor Author

AHeise commented Aug 25, 2020

CI shows a timeout in {{DataSinkTaskTest.testDataSinkTask}}.

Thanks. I converted to draft until I resolved it.

I also wanted to collect some feedback anyways before considering it a serious PR (in the sense of being mergeable).

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 draft @AHeise. I've left some comments/questions.

Comment on lines 88 to 87
while (pollNext().map(BufferOrEvent::morePriorityEvents).orElse(false)) {
}
Copy link
Contributor

Choose a reason for hiding this comment

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

As I understand it, it assumes that this pollNext() can not return anything else besides a priority event?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Yes, first this method checks if there is at least one priority event (priority future completed). If there is at least one, it starts processing the first one. At this point, it relies on BufferOrEvent::morePriorityEvents to be correct in both directions (no false positives or negatives; although a false negative would just be a tad slower).

Copy link
Contributor

Choose a reason for hiding this comment

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

maybe add a checkState, that we are not loosing some unexpected data?

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 update @AHeise. So far I haven't got to the new parts, mostly revisited the first commits to refresh my memory.

note to myself, I've finished reading the code until:

Use futures to listen to priority events and handle them in StreamTaskNetworkInput.

Comment on lines 200 to 213
private boolean isUnalignedCheckpoint(BufferConsumer bufferConsumer) {
boolean unalignedCheckpoint;
try (BufferConsumer bc = bufferConsumer.copy()) {
Buffer buffer = bc.build();
try {
final AbstractEvent event = EventSerializer.fromBuffer(buffer, getClass().getClassLoader());
unalignedCheckpoint = event instanceof CheckpointBarrier;
} catch (IOException e) {
throw new IllegalStateException("Should always be able to deserialize in-memory event", e);
} finally {
buffer.recycleBuffer();
}
}
return unalignedCheckpoint;
Copy link
Contributor

Choose a reason for hiding this comment

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

this method is only isCheckpointBarrier and it seems to not care if it's aligned or not, right?

Besides, do we really need to deserialise the event? Previously we were snapshotting in-flight data every time we were inserting buffer as a head. I think it was just as not elegant, but simpler.

I guess this is currently a dead code, but would change if we ever want to have priority cancelation markers? If that's a sole motivation, I would revisit this problem in the future. Who knows if we will need this with checkpoint abort RPC. And if we will do, there is also another option:

Inserting priority UC barrier, could go through a separate method , that would return overtaken in-flight data:

Collection<Buffer> insertAsHeadAndGetInFlightData(checkpointBarrier)

which would also eliminate the currently existing assumption/hack that requestInflightBufferSnapshot has to be always called immediately after inserting as a head.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Moved this method to the commit that spills immediately. We need it in that method to retrieve the checkpoint id to spill correctly.

Deserialization is only necessary for priority events, which are very rare and rather cheap (30 bytes). I'd argue that adding a new call chain just to optimize it is not warranted.

* @param priority flag indicating if it's a priority or non-priority element
* @param alreadyContained flag that hints that the element is already in this deque, potentially as non-priority element.
*/
public void add(T element, boolean priority, boolean alreadyContained) {
Copy link
Contributor

Choose a reason for hiding this comment

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

is this method being used? I think at least not in this commit

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'm moving it to the commit that starts using it.


CompletableFuture<?> toNotifyPriority = null;
Copy link
Contributor

Choose a reason for hiding this comment

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

optional nit:
extract toNotify and toNotifyPriority pair to some simple inner class `

public static class DataNotification() {
  @Nullable
  CompletableFuture<?> toNotifyPriority = null;
  @Nullable
  CompletableFuture<?> toNotify = null;

  // two setters
  setXYZ(...);

  void complete() {
 		if (toNotifyPriority != null) {
			toNotifyPriority.complete(null);
		}
		if (toNotify != null) {
			toNotify.complete(null);
		}
  }
}

and re-use in UnionInputGate as well?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Extracted into GateNotificationHelper, please check if it's actually helping to reduce complexity.

*/
default boolean notifyPriorityEvent(BufferConsumer eventBufferConsumer) throws IOException {
return false;
default void notifyPriorityEvent() {
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 elaborate a bit more in the commit message, what has been simplified, why and what are the benefits?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Added a few thoughts. Let me know if it makes things clearer.

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 update @AHeise .

I've left a couple of comments more. I think there are no ground braking issues.

Comment on lines 205 to 209
channelStateWriter.addOutputData(
barrier.getId(),
subpartitionInfo,
ChannelStateWriter.SEQUENCE_NUMBER_UNKNOWN,
inflightBuffers.toArray(new Buffer[0]));
Copy link
Contributor

Choose a reason for hiding this comment

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

Again I would point to the previous comment:
#13228 (comment)

Collection<Buffer> insertAsHeadAndGetInFlightData(checkpointBarrier)

might be a better option. (It might not, as I haven't tried to implement it)

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Not relevant in the final version where the channel spills by itself (no return value on this method). I can make it clearer in the commit message if you want.

@@ -369,9 +369,7 @@ public void testMissingCancellationBarriers() throws Exception {
inputGate = createBarrierBuffer(2, sequence, validator);
Copy link
Contributor

Choose a reason for hiding this comment

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

Why has this test and CheckpointBarrierTrackerTest.java changed in this commit? Rebasing/squashing mistake, or am I missing something about this commit (I thought it's a pure clean up without functional changes).

Copy link
Contributor Author

Choose a reason for hiding this comment

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

A side-effect of this commit is that all events are handed over from CheckpointedInputGate to StreamTaskNetworkInput and break up the poll loop. However, since events are rare, it should have no visible impact on the throughput.

The changes to the tests are now handling the additionally emitted events. Imho tests are easier to read now (no magically disappearing buffers in the sequence).

mainMailboxExecutor.execute(() -> {
prioritySelectionHandler.setAvailableInput(index);
priorityAvailability.getUnavailableToResetAvailable().complete(null);
}, "priority event {}", index);
Copy link
Contributor

Choose a reason for hiding this comment

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

nit: add StreamMultipleInputProcessor to the mail's name?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Commit removed; on CheckpointedInputGate, I'm adding the gate.toString().

// set the priority flag in a mail before notifying StreamTask of availability
mainMailboxExecutor.execute(() -> {
prioritySelectionHandler.setAvailableInput(index);
priorityAvailability.getUnavailableToResetAvailable().complete(null);
Copy link
Contributor

Choose a reason for hiding this comment

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

Can not you maybe handle the priority message directly here, in this mail? Instead of relaying on the processDefaultAction to pick this up?

(I'm asking/loudly thinking)

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Moved it even further up towards CheckpointedInputGate. At this point, we need to make sure that a priority event is really at the top (hence the optimistic lock protocol for notification).

Comment on lines 116 to 120
for (final InputGate gate : inputGates) {
for (int index = 0, numChannels = gate.getNumberOfInputChannels(); index < numChannels; index++) {
gate.getChannel(index).checkpointStopped(currentCheckpointId);
}
}
Copy link
Contributor

Choose a reason for hiding this comment

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

What's the story behind this change?

  1. it seems it differs only by a single line numBarriersReceived = 0;, so at the very least we should deduplicate some code here
  2. can you explain what's the functional change?
  3. aren't we missing a unit test for that? It would help answer point 2., and if there was a bug discovered in e2e test, it would be nice to have a faster unit test for that as well.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Sorry that was just a test commit to see if the stuck e2e failed because of this change. I removed it. The original change is covered by a few unit tests already.

Copy link
Contributor Author

@AHeise AHeise left a comment

Choose a reason for hiding this comment

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

Addressed all comments but also did major changes as a result:

  • Priority events are completely handled in CheckpointedInputGate; no processing on StreamTaskNetworkInput, Processors, StreamTask. The last commit is dropped for that reason.
  • To avoid spurious priority notifications, there is a new commit [FLINK-19026][network] Move sequence number into PipelinedSubpartition and relay through BufferAndAvailability and BufferAndBacklog. I'd like to squash it with the previous commit as it touches very similar files. I left it unsquashed for easier review.

Comment on lines 200 to 213
private boolean isUnalignedCheckpoint(BufferConsumer bufferConsumer) {
boolean unalignedCheckpoint;
try (BufferConsumer bc = bufferConsumer.copy()) {
Buffer buffer = bc.build();
try {
final AbstractEvent event = EventSerializer.fromBuffer(buffer, getClass().getClassLoader());
unalignedCheckpoint = event instanceof CheckpointBarrier;
} catch (IOException e) {
throw new IllegalStateException("Should always be able to deserialize in-memory event", e);
} finally {
buffer.recycleBuffer();
}
}
return unalignedCheckpoint;
Copy link
Contributor Author

Choose a reason for hiding this comment

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

Moved this method to the commit that spills immediately. We need it in that method to retrieve the checkpoint id to spill correctly.

Deserialization is only necessary for priority events, which are very rare and rather cheap (30 bytes). I'd argue that adding a new call chain just to optimize it is not warranted.

* @param priority flag indicating if it's a priority or non-priority element
* @param alreadyContained flag that hints that the element is already in this deque, potentially as non-priority element.
*/
public void add(T element, boolean priority, boolean alreadyContained) {
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'm moving it to the commit that starts using it.

@@ -369,9 +369,7 @@ public void testMissingCancellationBarriers() throws Exception {
inputGate = createBarrierBuffer(2, sequence, validator);
Copy link
Contributor Author

Choose a reason for hiding this comment

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

A side-effect of this commit is that all events are handed over from CheckpointedInputGate to StreamTaskNetworkInput and break up the poll loop. However, since events are rare, it should have no visible impact on the throughput.

The changes to the tests are now handling the additionally emitted events. Imho tests are easier to read now (no magically disappearing buffers in the sequence).

@@ -22,13 +22,10 @@
import org.apache.flink.annotation.VisibleForTesting;
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 admit that it's an awkward cut.

However, it's only spilled in one place as the BufferReceivedListener methods are effectively not called in the previous commits anymore. I will make a later pass to see that all tests pass though.

@@ -812,242 +788,13 @@ public void testQueuedBuffers() throws Exception {
}
}

@Test
public void testBufferReceivedListener() 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.

Can you double check if indeed all of those test should be removed? I don't see how things like testPartitionNotFoundExceptionWhileGetNextBuffer should be related to this commit/PR?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Good catch, my intent was to delete testBufferReceivedListener and testPartitionNotFoundExceptionWhileGetNextBuffer but not the test in between them.

testBufferReceivedListener tests BufferReceivedListener which this commits renders useless (and is later removed).

testPartitionNotFoundExceptionWhileGetNextBuffer tests concurrent spilling of lingering buffers and receiving of such lingering buffers. Both now happens in the same thread, so the test does not make any sense.

Comment on lines 88 to 87
while (pollNext().map(BufferOrEvent::morePriorityEvents).orElse(false)) {
}
Copy link
Contributor

Choose a reason for hiding this comment

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

maybe add a checkState, that we are not loosing some unexpected data?

…atorContext#unregisterSourceReader.

If an error happens during startup, the reader may not be registered (yet), but cleanup is triggered anyways.
It generalizes the current special treatment of unaligned checkpoints and will allow a consistent treatment of priority events on both input and output.
The new type facilitates detection of priority events without the need to inspect the contents of a buffer on input side.
It also eases the special treatment of priority event on output side as the contextual priority flag is now inlined after the buffer has been created.
…dSubpartition.

PrioritizedDeque supports enqueue elements with priority such that it will be polled after all existing priority elements but before any non-priority element.
It is a building block for fair scheduling with priority elevation that will be used also on input side in the next commits.
…ide.

The priority information is fully incorporated in Buffer.DataType now.
…ndBacklog to capture the DataType of the next record.

The data type of the next record allows to check for availability, whether it's an event, and whether the event has a priority.
It also will allow handling future data types more smoothly.
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 udpates @AHeise . I think the code LGTM %:

…n and relay through BufferAndAvailability and BufferAndBacklog.

The sequence number will be used in input side to avoid spurious priority notification in later commits.
Better distinguishes between optional and non-optional variables. Add short cut for empty buffers.
Next commit will build on this refactoring to incorporate priority events.
…polling.

This is an alternative fix for FLINK-12510, where a cyclic deadlock can happen when a subpartition is being requested, another subpartition is freed, and data is being polled at the same time on an UnionInputGate. This fix avoids double-lock acquisition on polling by moving the availability notification for a newly acquired subpartitions outside of the lock of ResultPartitionManager. This change may trigger a availability notification on Subpartition without data being available, however, all relevant components are guarded against spurious wakeups.
PriorityDeque is also used in InputGates. BufferOrEvent and InputWithData are enriched with a flag indicating that there are more priority events.
(Note relaying the DataType as on the output side would require lock acquisitions which are not warranted at this point in time)
BufferAvailabilityListener#notifyPriorityEvent now is a simple notification to avoid any kind of secondary data flow on output side as it was originally intended before.
For remote channels, notifyPriorityEvent behaves like an extra notifyDataAvailable call as an event is always pollable. For local channels, notifyPriorityEvent ultimately informs InputGate that the respective channel has a priority event.
A future commit moves the Unaligner completely into task thread, which would result in late spilling of in-flight data during polling and potentially delay un
Because channels are now responsible for spilling in-flight data during unaligned checkpoint, in-flight data can be spilled as soon as the checkpoint has been.
…ivedListener to CheckpointedInputGate.

This commit renders BufferReceivedListener obsolete and will allow a following commit to remove it entirely.
A side-effect of this commit is that all events are handed over from CheckpointedInputGate to StreamTaskNetworkInput and break up the poll loop. However, since events are rare, it should have no visible impact on the throughput.
…rrierUnaligner.

This concludes the refactoring: All priority events use the same buffer hand-over as normal events; the buffers are just reordered.
Notification of priority event bypasses CheckpointBarrierHandler and directly triggers CheckpointedInputGate#processPriorityEvents.
Note that checkpoints are not cancelled anymore if Unaligner received all barriers. This behavior is now in line with Aligner.
…ource interface.

The rewritten test induces heavy backpressure which would not work at all with aligned checkpoints or legacy sources during the timeout period.
@AHeise
Copy link
Contributor Author

AHeise commented Sep 23, 2020

* https://github.com/apache/flink/pull/13228/files#r493039856 is still not addressed I think?

* azure green

* benchmark request will not show a visible regression :)

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Projects
None yet
5 participants