Hi mates, I got a proposal about functionality of BucketingSink.
During implementation of one of our tasks we got the following need - create a meta-file, with the path and additional information about the file, created by BucketingSink, when it’s been moved into final place. Unfortunately such behaviour is currently not available for us. We’ve implemented our own Sink, that provides an opportunity to register notifiers, that will be called, when file state is changing, but current API doesn’t allow us to add such behaviour using inheritance ... It seems, that such functionality could be useful, and could be a part of BucketingSink API What do you sink, should I make a PR ? Sincerely yours, Rinat Sharipov Software Engineer at 1DMP CORE Team email: [hidden email] mobile: +7 (925) 416-37-26 CleverDATA make your data clever |
Hi,
I see that could be a useful feature. What exactly now is preventing you from inheriting from BucketingSink? Maybe it would be just enough to make the BucketingSink easier extendable. One thing now that could collide with such feature is that Kostas is now working on larger BucketingSink rework/refactor. Piotrek
|
Hi guys, thx for your reply.
The following code info is actual for release-1.5.0 tag, org.apache.flink.streaming.connectors.fs.bucketing.BucketingSink class For now, BucketingSink has the following lifecycle of files When moving files from opened to pending state:
So, the only way, that we have, is to call our hook from closeCurrentPartFile, that is private, so we copy-pasted the current impl and injected our logic there Files are moving from pending state into final, during checkpointing lifecycle, in notifyCheckpointComplete:657 line, that is public, and contains a lot of logic, including discovery of files in pending states, synchronization of state access and it’s modification, etc … So we couldn’t override it, or call super method and add some logic, because when current impl changes the state of files, it removes them from state, and we don’t have any opportunity to know, for which files state have been changed. To solve such problem, we've created the following interface /** And have added an ability to register this callbacks in BucketingSink impl in the following manner public BucketingSink<T> registerOnFinalStateChangeCallback(FileStateChangeCallback… callbacks) {...} public BucketingSink<T> registerOnPendingStateChangeCallback(FileStateChangeCallback... callbacks) {...} I’m ready to discuss the best ways, how such hooks could be implemented in the core impl or any other improvements, that will help us to add such functionality into our extension, using public api, instead of copy-pasting the source code. Thx for your help, mates =)
Sincerely yours, Rinat Sharipov Software Engineer at 1DMP CORE Team email: [hidden email] mobile: +7 (925) 416-37-26 CleverDATA make your data clever |
Hi,
Couple of things: 1. Please create a Jira ticket with this proposal, so we can move discussion from user mailing list. I haven’t thought it through, so take my comments with a grain of salt, however: 2. If we were to go with such callback, I would prefer to have one BucketStateChangeCallback, with methods like `onInProgressToPending(…)`, `onPendingToFinal`, `onPendingToCancelled(…)`, etc, in oppose to having one interface passed three times/four times for different purposes. 3. Other thing that I had in mind is that BucketingSink could be rewritten to extend TwoPhaseCommitSinkFunction. In that case, with public class BucketingSink2 extends TwoPhaseCommitSinkFunction<???> user could add his own hooks by overriding following methods BucketingSink2#beginTransaction, BucketingSink2#preCommit, BucketingSink2#commit, BucketingSink2#abort. For example: public class MyBucketingSink extends BucketingSink2 { @Override protected void commit(??? txn) { super.commit(txn); // My hook on moving file from pending to commit state }; } Alternatively, we could implement before mentioned callbacks support in TwoPhaseCommitSinkFunction and provide such feature to Kafka/Pravega/BucketingSink at once. Piotrek
|
Hi Piotr, I’ve create an issue https://issues.apache.org/jira/browse/FLINK-9592
The third proposal looks great, may I try to contribute this issue ?
Sincerely yours, Rinat Sharipov Software Engineer at 1DMP CORE Team email: [hidden email] mobile: +7 (925) 416-37-26 CleverDATA make your data clever |
Hi Piotr, during the migration to the latest Flink version, we’ve decided to try to contribute this functionality to the master branch.
PR is available here https://github.com/apache/flink/pull/6824 More details about hooking the state changes in BucketingSink are available in https://issues.apache.org/jira/browse/FLINK-9592 Thx !
Sincerely yours, Rinat Sharipov Software Engineer at 1DMP CORE Team email: [hidden email] mobile: +7 (925) 416-37-26 CleverDATA make your data clever |
Hi Ribat, I haven't checked your PR but we introduced a new connector in flink 1.6 called StreamingFileSink that is supposed to replace BucketingSink long term. I think it might solve a few problems of yours. Have you checked it by chance? Best, Dawid On Thu, 11 Oct 2018, 14:10 Rinat, <[hidden email]> wrote:
|
Hi Rinat,
I have commented on your PR and on the JIRA. Let me know what you think. Cheers, Kostas
|
Free forum by Nabble | Edit this page |