Skip to content

Conversation

@pnowojski
Copy link
Contributor

@pnowojski pnowojski commented Jun 20, 2019

This PR consists of series of changes that:

  • refactor existing CheckpointBarrierHandler to prepare a ground for using them in StreamTwoInputSelectableProcessor
  • use proper CheckpointBarrierHandler in StreamTwoInputSelectableProcessor

Next step would be to replace StreamTwoInputProcessor with StreamTwoInputSelectableProcessor.

For detailed changelog, please check individual commits.

There are quite a lot of refactoring and renaming commits there, I tried hard not to mix them with functional changes.

Because of the sheer amount of renames and refactorings and my attempt to keep them small, commits split might not be perfect :(

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, 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)

@flinkbot
Copy link
Collaborator

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.

Details
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

@pnowojski
Copy link
Contributor Author

@zhijiangW, @sunhaibotb and @Myasuka: you might be interested in this one.

@sunhaibotb
Copy link
Contributor

Thanks for the great work @pnowojski .

Copy link
Contributor

@sunhaibotb sunhaibotb left a comment

Choose a reason for hiding this comment

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

I have left two small comments about the "Rename BufferBlocker to BufferStorage" commit.

@zhijiangW
Copy link
Contributor

Thanks for notifying me about this PR @pnowojski !

Actually I am interested in it, but there are many review works in hands for release-1.9 atm. I am not sure whether have time for reviewing it in time before feature freeze, and I will try best. :)

@pnowojski
Copy link
Contributor Author

Thanks @sunhaibotb for the review. I have addressed the comments and resolved the conflicts.

Copy link
Contributor

@sunhaibotb sunhaibotb left a comment

Choose a reason for hiding this comment

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

I didn't find questions about the correctness, and only left some comments on other issues. I'll take the time to look at the key parts again. @pnowojski

public void close() {
public void close() throws IOException {
BufferOrEvent boe;
while ((boe = currentBuffers.poll()) != null) {
Copy link
Contributor

Choose a reason for hiding this comment

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

Nit: If you can add a few comments like "this part should not ever fail", I don't need to find out why there's no try... finally here.

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 not sure if that's the contract here. I guess this is on a best effort basis clean up.

Regardless of that, this is a separate issue independent of my change. If you think that something is not correct here, can you create a new issue and investigate it there?

Copy link
Contributor

Choose a reason for hiding this comment

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

I mean, the reason why it don't need to add try...finally like the following code is that the while {...} parts will never fail, and the code is clearer if we can add some comments to express this point.

	public void close() throws IOException {
		try {
			BufferOrEvent boe;
			while ((boe = currentBuffers.poll()) != null) {
				if (boe.isBuffer()) {
					boe.getBuffer().recycleBuffer();
				}
			}
		} finally {
			super.close();
		}
	}

Copy link
Contributor Author

@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 review @sunhaibotb. I've addressed and/or responded to your comments.

public void close() {
public void close() throws IOException {
BufferOrEvent boe;
while ((boe = currentBuffers.poll()) != null) {
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 not sure if that's the contract here. I guess this is on a best effort basis clean up.

Regardless of that, this is a separate issue independent of my change. If you think that something is not correct here, can you create a new issue and investigate it there?

Copy link
Contributor

@StephanEwen StephanEwen left a comment

Choose a reason for hiding this comment

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

This generally looks like a very good implementation.

One thing that strikes me as slightly odd is the LinkedBufferStorage.

  1. Is this very much tailored towards two inputs, when it would be good to keep it generic enough for N inputs (think side inputs in the future).
  2. if the linking is done in two places that need to be kept carefully in sync (BarrierHandler and LinkedBufferStorage), rather than in one place.

Could the barrier handler (or the specific barrier aligner implementation) not trigger the roll over of the buffer storage? That would eliminate the many if clauses (and rollover logic) out of the CheckpointedInputGate and abstract the entire linking between multiple input gates behind the fact that there is a shared barrier handler.

There are some minor issues in the code, which might be artifacts of prior work:

  • InputGate.pollNext() throws an InterruptedException even though it is a non-blocking operation.
  • creating a BufferStorage for AT_LEAST_ONCE mode creates also a CachingBufferStorage. While this is never used (no buffer is stored in that way), should this maybe be an implementation that throws unsupported operation exceptions, to fail fast in case a buggy code path actually tries to store a buffer there.

@pnowojski
Copy link
Contributor Author

pnowojski commented Jul 1, 2019

Thanks for the review @StephanEwen.

Is this very much tailored towards two inputs, when it would be good to keep it generic enough for N inputs (think side inputs in the future).

I was considering making it more general, but currently we don't have general NInputStreamTask or NInputStreamProcessor partially for the performance reasons. So:

  1. I wanted to be consistent with existing TwoInput*** classes
  2. I didn't want to risk spending time investigating & make sure that performance of n input storage would be good enough
  3. There is also a drawback of keeping a more general classes/interfaces when they are never used. It confuses code reader, making potential future refactoring/bug fixing/expanding/maintaining more expensive (a good example was int[] ChannelSelector#selectChannel)

Because of those reasons I thought that it's better to keep it specialised now and maybe revisit this in the future.

Could the barrier handler (or the specific barrier aligner implementation) not trigger the roll over of the buffer storage?

That I was also considering. The problem is that buffer storage can not be hidden from CheckpointedInputGate inside BarrierHandler class, because buffers must be stored independently for either of the two inputs (in order to make input selection work). Once this out of the question, the only alternative solution would be to keep BufferStorage instance in the CheckpointedInputGate, but "automatically" roll over it from BarrierHandler. This would unfortunately add another dependency to the BarrierHandler (making it more complicated & harder to insatiate/test) and would also add some implicit side effects: BufferStorage would be implicitly modified by actions on BarrierHandler, which I think is worse compared to the current setup.

All in all, I don't like the current setup too much, but I think it's a lesser evil and at least BarrierHandler and BufferStorage classes are mostly independent of one another.

InputGate.pollNext() throws an InterruptedException even though it is a non-blocking operation.

I think I will add a java doc above this method:

// Please increase the counter if you spent your time investigating 
// why InterruptedException might be thrown and realised that it can not be removed
//
// total_people_count = 4  

@StephanEwen + @StefanRRichter + @NicoK + @pnowojski = 4 :)

The catch is that this method might block if there is some data, but the buffers pool is empty and we are waiting for some buffer to be recycled (via LocalInputChannel).

implementation that throws unsupported operation exceptions

Done

@pnowojski
Copy link
Contributor Author

pnowojski commented Jul 1, 2019

Re InterruptedException

Actually, with your @StephanEwen new BoundedBlockingSubpartitionReader this is no longer needed, so technically speaking as a part of FLINK-12070 effort both InterruptedException and IOException could have been removed from the ResultSubpartitionView#getNextBuffer exception's list (and thanks to that from InputGate#pollNext as well). I've created a ticket for that.

@StephanEwen
Copy link
Contributor

Forwarding the results of an offline discussion between @pnowojski and me:

We sketched an option where BufferStorage would be a component used by the BarrierAligner and not directly exposed to the CheckpointedInputGate. That variant would spare the "reaching from one input gate into the other" via the LinkedBufferStorage and probably generalize to N inputs.

However, there are a lot of open questions about that approach as well. For example, the BarrierHandler would need to know about the different inputs and potentially handle the N input availability notifications (which might interfere with the responsibility of the SelectableTwoInputStreamOperator.

Given the uncertainty about the shortcomings or design complexity of the alternative implementation, I concur with Piotr that this implementation is the better approach for the time being, especially when avoiding the risk of regression of the current TwoInput* code paths.

For the introduction of the N input operators and side inputs, we should revisit this design.

@StephanEwen
Copy link
Contributor

With decision mentioned in the previous comment, I have no more concerns about this PR.
The code quality is overall very good.

+1 to merge from my side.

@pnowojski
Copy link
Contributor Author

Thanks for the review @StephanEwen and +1 for what you have written.

PR has passed tests on my private travis instance and previous version (without the EmptyBufferStorage class) had green travis before, so I'm merging it as it is.

pnowojski added 9 commits July 1, 2019 16:40
…ith the refactor

1. Rename BarrierBuffer to CheckpointedInputGate
CheckpointedInputGate was an interface, while BarrierBuffer was
it's implementation. This rename means that we are dropping the interface
and keeping only the concrete class.

2. Rename BarrierBuffer and BarrierTracker tests to match this rename
and previous refactorings.
This reference was introducing unwanted dependency between CachedBufferStorage and a
class that was using it.
@pnowojski pnowojski merged commit 6311a65 into apache:master Jul 1, 2019
@pnowojski pnowojski deleted the f12777 branch July 1, 2019 14:41
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Projects

None yet

Development

Successfully merging this pull request may close these issues.

6 participants