-
Notifications
You must be signed in to change notification settings - Fork 13.8k
[FLINK-25696][datastream] Introduce metadataConsumer to InitContext in Sink #18412
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
Conversation
Thanks a lot for your contribution to the Apache Flink project. I'm the @flinkbot. I help the community Automated ChecksLast check on commit 91a7a14 (Thu Jan 20 06:00:01 UTC 2022) Warnings:
Mention the bot in a comment to re-run the automated checks. Review Progress
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 commandsThe @flinkbot bot supports the following commands:
|
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Thanks for drafting this PR. It looks mostly good I left two inline questions.
for (Consumer<RecordMetadata> metadataConsumer : metadataConsumers) { | ||
metadataConsumer.accept(metadata); |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Is this really safe? Currently one of the Kafka producer threads will update the metadata.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Good point, I've summarized below
class KafkaWriter<IN> implements SinkWriter<IN, KafkaCommittable, KafkaWriterState> { | ||
class KafkaWriter<IN> | ||
implements SinkWriter<IN, KafkaCommittable, KafkaWriterState>, | ||
MetadataPublisher<RecordMetadata> { |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Actually, my initial idea was to implement the MetadataPublisher
with the Sink and only pass the list of subscribers to the writer.
I am not sure about the final design of the table store but my assumption was that there are only new subscribers during the writer creation and not during runtime. So exposing adding new subscribers through the writer is unnecessary.
Does that make sense?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
The big question is whether the subscribe
call occurs before or after serialization
expected.add("testMetadataPublisher-0@" + i); | ||
} | ||
writer.prepareCommit(false); | ||
assertThat(metadataList, equalTo(expected)); |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Nit: please use assertJ for new assertions
Thanks @fapaul for the review, I've rethought it and it's probably not a good idea either way.
|
So I am thinking maybe we can move forward in JingsongLi#18 (comment)
Sink can decide for itself whether to publish meta information to subscriber. What do you think? |
I think here we have two different thread-safety issues. One is that the consumer is invoked by the KafkaProducer that you probably need to fix and trigger the callback via the Mailbox. The other issue is around adding the subscribers concurrently.
I agree the Consumer needs to be serializable but if we pass an unmodifiable list to the sink writer it solves a lot of the thread-safety issues because it is not possible to add new subscribers after the sink translation. I guess if that works depends on your implementation that you have in mind to use the metadata.
Isn't this approach very similar to option 2 you have outlined? You probably still need a serializable consumer that you can pass via the context to the sink writer. How is the consumer set in the init context? |
The
|
I updated in 256bff6 |
I don't think we need to put it inside the mailbox, it would be very performance intensive, it's a per record operation. A callback consumer, which I think has asynchronous processing reasonable.
Yes, the problem is that this is a runtime statistic, which needs to expose information to an external caller, and it's hard to implement by a serializable class. |
@flinkbot run azure |
From a Kafka connector side, the subscriber is not updated on every record, and just when the KafkaProducer is flushed it is only updated for a bulk of records (either during a checkpoint or if the internal buffer size is reached). Regarding adding a method to the InitContext I think that is okay. Do you think there will be ever multiple Subscribers? Maybe it is safer to already add a list instead of an optional. I am still a bit surprised that the |
Thanks for the information. I share my concern here:
I think we can let the implementer assemble it himself, if there is a need for list.
|
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
- Consistency issues: for example, the data flushed at the time of transaction commit, then re-stuff their meta into the mailbox, these data belong to the next checkpoint, no longer the current checkpoint, and TableStore wants the metas of the current checkpoint.
You can also have consistency problems with the current approach because updating the metadata consumer does not imply that the offsets are committed with the same checkpoint. The commit can either fail and be retried or the notifyCheckpointComplete
may get lost and the committable is committed with one of the next checkpoints.
Overall I am a bit skeptical about the benefit of the current metadata consumer if the sink works with transactions.
flink-core/src/main/java/org/apache/flink/api/connector/sink/Sink.java
Outdated
Show resolved
Hide resolved
* {@link MetaT} to the consumer. The consumer can accept metadata events in an asynchronous | ||
* thread, and the {@link Consumer#accept} method is executed very fast. | ||
*/ | ||
default <MetaT> Optional<Consumer<MetaT>> metadataConsumer() { |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I'd mark this @Experimental
to leave some room to change the threading model later if necessary.
And the default value of |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Thanks for providing all the helpful information. From my side, it is good to move forward with PR.
You only need to ensure please that you add the MetadataConsumer
to the new Sink [1] interface and not the old one. We will deprecate the old ones in this release.
[1]
interface InitContext { |
Thanks, I have reverted the modification of old one. |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Merging...
What is the purpose of the change
In Table Store, we want to get the offsets of kafka writer, only the offset returned by the callback inside the KafkaWriter is accurate, so we need this callback mechanism.
This ticket wants to add metadataConsumer to InitContext in Sink:
SinkWriter can get this consumer, and publish metadata to the consumer implemented by table store sink.
Brief change log
Verifying this change
KafkaWriterITCase.testMetadataPublisher
Does this pull request potentially affect one of the following parts:
@Public(Evolving)
: (yes / no)Documentation