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-3257] Add Exactly-Once Processing Guarantees for Iterative DataStream Jobs #1668

Closed
wants to merge 1 commit into from

Conversation

senorcarbone
Copy link
Contributor

@senorcarbone senorcarbone commented Feb 18, 2016

Motivation and Algorithm

This is a first version of the adapted snapshot algorithm to support iterations. It is correct and works in practice...well, when memory capacity is enough for its logging requirements but I am working on that, hopefully with a little feedback from you. Before we go into the implementation details let me describe briefly the new algorithm.

Algorithm

Our existing checkpoint algorithm has a very fluid and straightforward protocol. It just makes sure that all checkpoint barriers are aligned in each operator so that all records before barriers (pre-shot) are processed before taking a snapshot. Since waiting indefinitely for all records in-transit within a cycle of an execution graph can violate termination (crucial liveness property) we have to...save any unprocessed records for later and get done with the snapshot. In this take of the algorithm on Flink we assign that role to the Iteration Head. The steps this version varies from the vanilla algorithm are simply the following:

(1) An Iteration Head receives a barrier from the system runtime (as before) and:

  • Goes into Logging Mode. That means that from that point on every record it receives from its Iteration Sink is buffered in its own operator state (log) and not forwarded further until it goes back to normal mode.
  • Forwards the barrier to its downstream nodes (this guarantees liveness, otherwise we have a deadlock).

(2) Eventually, the Iteration Head receives a barrier back from its Iteration Sink. At that point:

  • It checkpoints the log as its operator state.
  • Flushes to its outputs all pending records from the log and resets it.
  • Goes back to normal forwarding mode.

(3) When the Iteration Head starts/restarts it looks at its initial operator state (the log) and flushes any records that are possibly pending.

Example

This is just a very simple example topology. We have a single source S and a mapper M within an iteration with a head H and tail T. This is the bare minimum we need for now to check how this works.

ftloops-topology

In the diagram below you can see the sequence of possible events, containing both barrier and record transmissions. For completeness I included the Runtime as a separate entity, this is in our case the Checkpoint Coordinator who periodically initialises checkpointing in all tasks without inputs.

The point that this diagram tries to make is the following:
Record R1 (or any record received in normal mode) gets forwarded to M, the only consumer of H before the checkpoint barrier. On the other hand, R2 is not forwarded to M until H has finished snapshotting (or during the same atomic block anyway). In case of a failure R2 is not lost, rather than saved for later.

diagram

A very brief description of a consistent/correct snapshot in our context could be summed up in the following sentence:

Every record prior to a snapshot has been either processed by an operator (state depends on record) or included in the snapshot as event in transit (see Chandy Lamport algorithm).

This algorithm guarantees this property and as explained earlier, it also terminates.

Current Implementation Details

In the current prototype I tried to keep changes to a minimum until we agree on the pending issues (see below) so there is plenty of room for improvement, engineering-wise. The important changes are the following:

  • Obviously there is no more checking for iterations during a fault tolerant StreamGraph construction.
  • the triggerCheckpoint function of StreamTask was split (please do not freak out). That was necessary in order to extract the checkpointing logic from the barrier handling logic (which should be obviously decoupled in the updated algorithm). The actual state snapshotting now happens inside checkpointStatesInternal.
  • The StreamIterationTail has to process and forward Either<StreamRecord<IN>, CheckpointBarrier> objects to StreamIterationHead which takes the necessary steps depending on the event according to the algorithm. An internal task abstraction ForwardingOneInputStreamTask was introduced for forwarding barriers to the tail.
  • The StreamIterationHead uses a simple UpstreamLogger operator internally for its logging needs, nothing special about it.
  • A StreamIterationCheckpointingITCase was added that checks exactly-once guarantees, loops included. More tests will follow upon demand.

I am sure we can improve this implementation but let's focus on the non-trivial problems first.

Open/Pending Issues

During stress-testing I found the following issues that we need to discuss hoping that some of you are maybe eager to propose something.

  • I realised that while flooding my cyclic topology with records I was still losing data, even at the absence of task failures. The problem seemed to be the way we transferring events from the StreamIterationTail to the Head. Apparently the tail offers records to the head with a specified timeout on a blocking queue of size 1(!?). What is going on there? The Head makes sense to timeout on reading, but, why does the StreamIterationTail throw away events like there is no tomorrow?
    Since at-most-once processing guarantees are not what we want I just commented out the offer part until I understand why it is there. I think it is safe (deadlock-wise) to simply put records to the queue, right?...I can also fire a new JIRA issue reporting this if needed.
  • I feel that we really need to spill log state to disk (BufferSpiller?), or/and restrict iterative jobs to out-of-core backend and forbid in-memory. The upstream log size can very easily pass the boundaries of the allowed in-memory state (e.g. when millions records in transit are snapshotted). Any ideas on that are welcome!

Thanks a lot for looking into this. Really looking forward to your suggestions.

@gyfora
Copy link
Contributor

gyfora commented Feb 19, 2016

Thanks Paris, I like the idea, it's a correct modification of the original algorithm to make it much easier to implement on Flink for the price of buffering more records.

I have some comments to the implementation:

  • Why did you introduce the ForwardingOneInputStreamTask if only the Iteration tail will implement this behaviour? I am not sure if other StreamTasks will do this in the future, so it might be better to just put the logic in the StreamIterationTail instead of adding a new abstraction
  • I think the RecordPusher and UpstreamLogger embedded operators on the head and tail tasks are not really necessary and just add additional complexity to debugging and understanding the code. The Upstream logger only makes checkpointing "easier" for the implementer but we probably want to do some custom logic there anyways. So I would probably just overwrite the checkpointStatesInternal method directly.

I agree that we definitely need to spill the buffered records to disk or use managed memory for this. This can be similar to the BarrierBuffer logic. We can combine this with checkpointing to an output stream to never have to materialize the full state.

@senorcarbone
Copy link
Contributor Author

Thanks for going through it Gyula!

I agree, the ForwardingOneInputStreamTask can be integrated in the StreamIterationTail, I will probably do that. I just wanted to get the code look less messy in the tail part but you are right.

About the operators, the RecordPusher was already there (even though it was not initialised correctly). I just added the UpstreamLogger because I wanted it to follow the operator lifecycle. I think the way it is we do not need to override the checkpointStatesInternal to do the changes we discuss. We just need to change the operator callback method and this could also be more robust to changes in the StreamTask and operator interfaces and default behavior, just my personal view but I see your point too.

I agree with the spill buffer logic. I am only confused a bit with the output stream thing (the other part of the problem), is there something already I can use? I haven't check recent changes. How does this work if we use in-memory backend for example? The blob with all the messages will be anyway packaged and sent within the stateHandle to the job manager in a single message (potentially being over the limits), even if we use a stream API, or?

@tillrohrmann
Copy link
Contributor

Great idea @senorcarbone. I also really like it :-)

I agree with @gyfora to include the logic of the ForwardingOneInputStreamTask in the StreamIterationTail.

For the spillable state we can only use the RocksDBStateBackend at the moment. But I think that is fine for a first iteration. Eventually, we'll also add managed memory support for the memory based state backends which will make them equally suitable for the task.

@senorcarbone
Copy link
Contributor Author

Thanks @tillrohrmann for the feedback!
I merged ForwardingOneInputTask into StreamIterationSink.

One more think I missed pointing out is that when the iteration timeout occurs the StreamIterationHead flushes its log to its output before terminating. An alternative take would be to delay termination until the barrier arrives back to complete the snapshot. The problem with that version, even though it's correct, is that under frequent checkpoint intervals the iteration head could potentially never terminate because it will always be in snapshotting state when the timeout occurs.

Regarding the state snapshotting, should I switch to using the ListState representation instead or does it make no difference for the time being?

We anyway have to revisit the iteration implementation due to other pending problems such as backpressure deadlocks (see FLINK-2470). I guess this PR is just a first step towards improving them further.

@gyfora
Copy link
Contributor

gyfora commented Feb 23, 2016

I can't really add anything to the timeout question :D

As for the snapshotting. I would go with the ListState as that potentially provides very efficient implementation for this logging phase. (It is actually designed to handle this) Then we don't have to worry about spilling there either.

@senorcarbone
Copy link
Contributor Author

You can find an alternative version using ListState in the following branch:
https://github.com/senorcarbone/flink/tree/ftloopsalt
So I noticed that this version is quite slower than the one with custom operator state but it can support larger states apparently.

I am (ab)using the PartitionedState to store the ListState in the same key, as @gyfora suggested since it is the only way to obtain the nice representations at the moment. It would be nice to have them available for operator state snapshots as well - @aljoscha have you thought about it? When there is free time (after the release) it would be nice to see what @aljoscha and @StephanEwen think of the two takes as well. No hurries, just take a look when you have time!

The two annoying issues I noticed during testing and we need to check soon, probably after this PR, are the following:

  • The overhead of transmitting and finally delivering a barrier from the head to its consumers increases in time (for each subsequent checkpoint). That is due to having a single queue at the beginning of the iterative part of the job. Events coming from the backedge are pushed further behind the input queue. It would be nice to pick and deliver events in a round robin fashion among the two input gates (iteration source, regular input). Otherwise, checkpoints in iterative jobs can be really prolonged in time due to this.
  • We need a proper way to deal with deadlocks. I removed the part where we discard events in the tail upon timeout since that boils down to at most once semantics. This PR is not solving deadlocks but I think we should find a graceful way to tackle them. (@uce, any ideas? )

* Checkpoints all operator states of the current StreamTask.
* Thread-safety must be handled outside the scope of this function
*/
protected boolean checkpointStatesInternal(final long checkpointId, long timestamp) 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.

What about naming this as in the comments drawStateSnapshot? That it is internal is more or less communicated by the fact that it is a protected method.

Copy link
Contributor

Choose a reason for hiding this comment

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

Regarding the JavaDocs:

  • The idomiatic style is to have a short description and then a blank line for more details (the first line will be displayed as a summary in the IDE etc.) and info about the returned boolean and params is missing
  • The of the current StreamTask is clear from context
  • The Thread-safety part should be more explicit, for instance The caller has to make sure to call this method in scope of the task's checkpoint lock.

Copy link
Contributor

Choose a reason for hiding this comment

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

When rebasing we have to double check that nothing changed in this method when calling from triggerCheckpoint etc.

@StephanEwen
Copy link
Contributor

The core idea of this is very good, also the illustration is very nice.

After an offline chat with @senorcarbone, we concluded that a remaining problem in this is currently the way it integrates with the timeout-based termination detection.

Which brings us to the point that we should (in my opinion) change the way that loops terminate. It should probably be based on end-of-stream events, to make it deterministic and not susceptible to delays.

Question is now, does it make sense to do the termination change first, and base this on top of it, or to merge this irrespective of that...

@senorcarbone
Copy link
Contributor Author

Thanks @StephanEwen and @uce for looking into it! I really appreciate it. How about the following:

  1. I update this PR with the patch that uses ListState and apply some nice refactorings @gyfora made
  2. I will also address all your comments and then merge this to master
  3. We start working on perfecting stream finalization on loops and backpressure deadlock elimination in seperate PRs right away. These are different issues and we need to address them separately, in my view of course.

@uce
Copy link
Contributor

uce commented Apr 25, 2016

Sounds like a reasonable approach Paris! Just ping here after you addressed 1. and 2.

@senorcarbone
Copy link
Contributor Author

ok good to know @uce! Let me get back to it in a couple of weeks and make it right, now it is a bit impossible to find time.

@senorcarbone
Copy link
Contributor Author

senorcarbone commented Sep 28, 2016

Hey! Good to be back :) . Let's fix this properly, as @StephanEwen recommended, now that there is some time.
We are writing together with @fouadma a FLIP to address major loop fixes. Namely, termination determination and fault tolerance. The termination implementation is already in a good shape in my opinion and you can find it here so you want to take an early look. The description in the FLIP will make clear of how this works in detail.

The FT update for loops will be rebased on top of the loop termination fix.
We hope that you will find this good too and btw thanks for your patience :)

EDIT: Here is a draft of the FLIP

@senorcarbone
Copy link
Contributor Author

senorcarbone commented Nov 22, 2016

Ok, so I am progressing this a bit independently from the termination stuff and then we rebase to the first PR that is merged. I just changed everything and rebased to the current master.

Some notable changes:

  • The StreamIterationCheckpointingITCase is now made deterministic, it fails on-purpose after the first successful checkpoint for just once and the job stops a bit after everything has been recovered appropriately (records in transit in feedback and operator states).
  • I am now using ListState which is supposed to work like a charm with the rocksdb file backend. Note that with the default in-memory backend there is a high chance to get "too large snapshot size" exceptions given the low memory capacity that it is given by default.
  • One tricky part that can be potentially done better is the way I set the logger in the StreamIterationHead. Also had to make checkpointing methods protected to access them through the StreamIterationHead without much of a hustle. Better alternatives are welcome.

Whenever you find time go ahead and check it out. It passes my super-strict test which is a good thing. :)

@senorcarbone
Copy link
Contributor Author

Hey again, @StephanEwen @uce .
When you have 10 min can you take a look to see if this is acceptable?
I would not like to leave this here for months again, it has been out way too long already.

The changes are just a few and straightforward so I really encourage you to skim them at your earliest convenience. Thanks!

@gyfora
Copy link
Contributor

gyfora commented Dec 12, 2016

I think the PR looks pretty good, and it sounds fair to to address termination in a later PR as this will still greatly improve the current guarantees without making the backpressure/termination problems any worse.

+1 from me

@senorcarbone
Copy link
Contributor Author

senorcarbone commented Dec 12, 2016

Exactly, these two issues do not depend on each other. No doubt loop FT is the first thing that can enable iterations in a production deployment so I would merge that first.

Thank you again @gyfora for looking into it :)

@StephanEwen
Copy link
Contributor

Thanks for the reminder, I went over the code today. The code looks mostly good, but here are some thoughts:

  • The head task supports only one concurrent checkpoint. In general, the tasks need to support multiple checkpoints being in progress at the same time. It frequently happens when people trigger savepoints concurrent to a running checkpoint. I think that is important to support.

  • There tail task offers the elements to the blocking queue. That means records are simply dropped if the capacity bound queue (one element) is not polled by the head task in time.

  • With the capacity bound in the feedback queue, it is pretty easy to build a full deadlock. Just use a loop function that explodes data into the feedback channel.

  • Recent code also introduced the ability to change parallelism. What are the semantics here when the parallelism of the loop is changed?

Since loops did not support any fault tolerance guarantees, I guess this does improve recovery behavior. But as long as the loops can either deadlock or drop data, the hard guarantees are in the end still a bit weak. So that leaves me a bit ambivalent what to do with this pull request.

@StephanEwen
Copy link
Contributor

To suggest some way to fix the guarantees: To my mind, the crux lies in the way that the feedback channel is implemented - a simple blocking queue just does not cut it for that case. To make this proper, I think we need to do the following:

  • Have an elastic feedback channel (unbounded) with a certain memory budget, that can spill if needed. I think it would be best implemented holding data serialized.
  • On checkpoint, one simply adds the feedback channel data (already bytes) to the checkpoint
  • The source task should probably prioritize reading from the feedback channel, to keep it always as small as possible.

@senorcarbone
Copy link
Contributor Author

These are some good points @StephanEwen, thanks for checking it.
How about the following, regarding each issue:

  • Concurrent Checkpoints: Looks like an improvement but I can sure do it in this PR if it is a crucial one. Can you elaborate a bit more or point me out to other concurrent checkpointing operator state examples to get an idea of how you want to do it?
  • Reconfiguration : Sounds interesting...but I am not really aware of it from the devlist. If it is simple enough I could add support for it here. Otherwise I would suggest we address this in a seperate JIRA and PR as an improvement. Is there a design document on how we plan to achieve reconfiguration and repartitioning for operator state specifically somewhere?
  • At-most-once blocking queue : It is obvious from my previous comments that I do not approve this part, but that is something we already got rid of in FLIP-15 already (this commit). How about we address this together with the deadlocks in FLIP-15?
  • Deadlocks: I like the elastic spilling channel idea to resolve deadlocks. I need time to dig a bit more into this and make sure we solve deadlocks and not just improve. Is it ok with you if we address that in FLIP-15? I need more time for this part, plus, we need to combine the absense of expiring queues with a proper termination algorithm (otherwise we just solve the deadlocks and the jobs never terminate).

What do you think?

@StephanEwen
Copy link
Contributor

@senorcarbone I agree, let's fix the multiple checkpoints issue and do the rest in FLIP-15

The other operators have a pretty simply way of doing this:

  • for synchronous checkpointed operators, no need to do anything, the synchronous part of one checkpoint is over when the next starts (because it is synchronous ;-))
  • for asynchronously checkpointed state, the state backend needs to be able to hold multiple snapshots, which are saved by multiple background threads
  • none of the operators deal with in-flight data, which makes their job easy

Dealing with in-flight data probably means that you need to open a ListState for each checkpoint that arrives and add the feed back values to each state, until that particular checkpoints barrier comes back through the feedback channel. I think that should be sufficient.

@senorcarbone
Copy link
Contributor Author

senorcarbone commented Dec 16, 2016

agreed @StephanEwen! I will do that.

EDIT 1:
Working on it atm . I decided to make the following optimisations but want to very quickly make sure that async checkpointing works the way I believe it does:

  • Most importantly, I am changing the iteration head to always forward records. Their effects are not present in any in-progress snapshot anyway so that I should had done from the very beginning. :)
  • If ListState is checkpointed asynchronously, depending on the backend I suppose, then the current version of it, during the snapshot, will be persisted as a copy, which means that we can reset it (clear and rebuild) right after invoking the snapshot to the beginning of the next in-progress snapshot (some indexing involved). That way we do not need to open new ListStates in the first place. Does this make sense?

@StephanEwen Please correct me if I am wrong, regarding the second point. I am just not very familiar with async snapshotting for ListState (this is not clear in the documentation for me). Mind also that I do not use the CheckpointedAsychronously interface, it seems to be heading towards deprecation. Thanks!

EDIT 2:
On a second look I had today, the second optimisation cannot be implemented with the append-only ListState so I will do it in separate ones as suggested. The first optimisation still holds though. @StephanEwen I would really appreciate it if you fill me in or point out an example on how to open and maintain multiple versions of a ListState for async checkpointing to make it right. Should I use descriptors with a different name?

@StephanEwen could you check my question above when you can?

@senorcarbone
Copy link
Contributor Author

senorcarbone commented Jan 19, 2017

The last update implements a variant of what @StephanEwen proposes to support concurrency under asynchronous snapshots (i.e. in rocksdb). We have put some more thought on this offline too thanks to @gyfora ! The idea is that instead of putting records to each ListState, the output log is partitioned into multiple log "slices", one per concurrent checkpoint.

More specifically, the UpstreamLogger operator at the StreamIterationHead slices logs proportionally to the number of concurrent snapshots. This also allows committed output logs to be uniquely identified and cleared after each complete checkpoint. The design is based on the following assumptions:

  • A slice is named after a checkpoint ID. Checkpoint IDs are numerically ordered within an execution.
  • Each checkpoint barrier arrives back in FIFO order, thus we discard log slices in respective FIFO order.
  • Upon restoration the logger sorts sliced logs in the same FIFO order and returns an Iterable that gives a singular view of the log.

Before I polish this we need to close a memory leak. The clear operation of State cleans the state under the registered id but it does not seem to unregister the key itself. Does anyone have an idea on how to unregister state properly and generically (not backend specific)? Hope this gets some attention to wrap it up, it's been too long :).

@addisonj
Copy link

addisonj commented Mar 9, 2017

Very interested in this work. It sounds like there are few loose ends and then some cleanup before it might be ready for merge, @senorcarbone or @StephanEwen anything that can be supported by someone else? Would love to help wherever possible

@senorcarbone
Copy link
Contributor Author

senorcarbone commented Mar 10, 2017

Hey @addisonj.
Sure! You could perhaps review the changes and/or maybe see how to discard empty operator states if you are motivated with a seperate PR (I dont think there is a JIRA for this either). This is the only pending issue to merge this PR . thanks!

@senorcarbone
Copy link
Contributor Author

senorcarbone commented Mar 15, 2017

I have just rebased to the current master and tests pass locally.
Please let me wrap this up. It has been over a year :)

Unregistering state in the OperatorStateStore is a very tiny fix.
@StephanEwen @StefanRRichter Is it ok with you to make this small addition in this PR or should I create a separate issue? Hope you have a spare few minutes to take a quick look this time.

@senorcarbone
Copy link
Contributor Author

senorcarbone commented Mar 24, 2017

Hey. Any update/opinion/something anyone?
Just a gentle reminder to check this maybe, my apologies if this sounds a bit desperate :)

Copy link
Contributor

@gyfora gyfora left a comment

Choose a reason for hiding this comment

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

Thanks Paris, I added some minor comments inline.

I like the current approach with the slices for the different checkpoints, too bad we can't completely delete states registered in the backends.

The only problem I see with the current implementation is that it will hold the elements in the list states in-memory before the checkpoint but this should not be a serious limitation.


@Override
public void open() throws Exception {
super.open();
Copy link
Contributor

Choose a reason for hiding this comment

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

why override if nothing changes?


@Override
public void setup(StreamTask<?, ?> containingTask, StreamConfig config, Output<StreamRecord<IN>> output) {
super.setup(containingTask, config, output);
Copy link
Contributor

Choose a reason for hiding this comment

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

same

}

private void progressLog() {
while (!currentIterator.hasNext() && ++indx < wrappedIterators.size()) {
Copy link
Contributor

Choose a reason for hiding this comment

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

You do a lot of checking against !currentIterator.hasNext() you could probably have it only in progressLog and call that without the if in the other places (maybe rename to progressIfNecessary)

}

public void clearLog() throws Exception {
for (String outputLogs : getOperatorStateBackend().getRegisteredStateNames()) {
Copy link
Contributor

Choose a reason for hiding this comment

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

It's kind of bad that we can't remove the state completely and keep iterating over them when replaying the log...

this.lock = getCheckpointLock();
getConfiguration().setStreamOperator(new UpstreamLogger(getConfiguration()));
operatorChain = new OperatorChain<>(this);
this.upstreamLogger = (UpstreamLogger<IN>) operatorChain.getHeadOperator();
Copy link
Contributor

Choose a reason for hiding this comment

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

if this is the same UpstreamLogger instance that you pass 2 lines above then why not use that? :)

Copy link
Contributor Author

Choose a reason for hiding this comment

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

it's another instance that is why I was fetching it back there.
The OperatorChain basically deserialises and sets up another instance through the configuration.

output.emitWatermark(new Watermark(Long.MAX_VALUE));
}
}

synchronized (lock) {
//emit in-flight events in the upstream log upon recovery
for (StreamRecord<IN> rec : upstreamLogger.getReplayLog()) {
Copy link
Contributor

Choose a reason for hiding this comment

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

Maybe it would make sense to put this in a while loop and check "running" as well, to cancel early if the job is cancelled during replay

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.

I think the algorithm is good. For the implementation, I have some suggestions:

List State Checkpoints

Storing the feedback channel data in a ListState means that each element in the list is tracked by metadata in the checkpoint (the list elements are the granularity of redistribution).
Since we probably do not need fine-grained redistribution, each operator can simply create one entry in the list (which has all the feedback elements). That would make checkpoint metadata much smaller and more robust.

Ideally, we actually do not checkpoint at all to an actual list, but simply write to raw operator state directly (then we are also not memory/rpc-payload-size constrained). The operator state should have the "list redistribution pattern".

@StefanRRichter Can you share a few pointers how to write to raw operator state from a task / operator?

Upstream Logger

Can we solve the access to the ListState or the OperatorState in general slightly different than creating a mock operator (UpStreamLogger). It works, but I feel it is a workaround due to the fact that an operator is missing.

We could simply give the IterationHead always an operator (that forwards input and feedback elements). That operator then checkpoints the feedback data.
As a nice extra, this would eliminate the case that a task can have null as the head operator (a case that always is a bit nasty to work around).

synchronized (getCheckpointLock()) {
if (isRunning()) {
dataChannel.put(new Either.Right(new CheckpointBarrier(checkpointMetaData.getCheckpointId(), checkpointMetaData.getTimestamp(), checkpointOptions)));
getEnvironment().acknowledgeCheckpoint(checkpointMetaData.getCheckpointId(), checkpointMetrics);
Copy link
Contributor

Choose a reason for hiding this comment

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

Can the IterationTailTask contain operators as well, or is it always a task without operators? If it has operators, we cannot immediately acknowledge here, but need to delegate to superclass checkpoint method instead.

Copy link
Contributor Author

@senorcarbone senorcarbone Mar 27, 2017

Choose a reason for hiding this comment

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

Hm, not really. I cannot think of a possible usage of loaded operators in IterationSink tasks. Perhaps structured window iterations will involve these special operators in the future (if we decide to keep them anyway) but I can only see the loop tracking logic to be implemented in the Heads only.

@senorcarbone
Copy link
Contributor Author

senorcarbone commented Mar 27, 2017

Thanks for the review @gyfora and @StephanEwen , these are very good points.

@StephanEwen makes sense to not really index/keep metadata of individual records in log slices, it is extra overhead. Writing raw operator state also makes sense to balance the serialisation overhead, so I will do that once @StefanRRichter gives me some pointers, that would be great.

Any redistribution of the checkpoint slices would violate causality so I hope the "list redistribution pattern" actually keeps the set of registered operator states per instance intact. The garbage collection issue still remains but maybe (if @StefanRRichter approves) I can add an unregister functionality to the OperatorStateStore.

I can also add preconfigured operators (not that they will be reused anywhere). It is more clean but I really need to see how can I get full control of the task checkpointing behaviour from the operator level (since the checkpointing behaviour is altered at the task-level).

@StefanRRichter
Copy link
Contributor

For raw operator state, override AbstractStreamOperator::snapshotState(StateSnapshotContext context) inside your operator. Your implementation calls to super, then it can obtain the raw stream via context.getRawOperatorStateOutput(). This stream works like a normal output stream, except that you can also call stream.startNewPartition(). This signals that a partition is started and previous partitions are finalized/immutable. Partitions are the atomic units of state redistribution, think of them as the indiviual elements in a ListCheckpointed state.

For restoring, override AbstractStreamOperator::initializeState(StateInitializationContext context). After calling super, context.getRawOperatorStateInputs() provides an iterable with one input stream per partition that your operator should restore.

@senorcarbone
Copy link
Contributor Author

sweet! thanks @StefanRRichter

@kien-truong
Copy link

Hi. Do we have any updates on this :)

@senorcarbone
Copy link
Contributor Author

well..major runtime changes are coming with FLIP-6, 15 and 16 so I would suggest you watch those. Loop FT will be included in one of these along with other loop redesign features.

@tzanko-matev
Copy link

Hi, are there any updates on this feature?

@john-tipper
Copy link

Hi, are there any updates on this feature please? Loop fault tolerance enables lots of very cool use cases and it would be great to see this incorporated.

@aljoscha
Copy link
Contributor

I'm closing this as "Abandoned", since there is no more activity and the code base has moved on quite a bit.

I think the feature is still very needed, the Jira issue remains open, and there is also a new FLIP about it: https://cwiki.apache.org/confluence/display/FLINK/FLIP-16%3A+Loop+Fault+Tolerance

@aljoscha aljoscha closed this Oct 14, 2019
@aljoscha aljoscha self-assigned this Oct 20, 2019
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.