[DISCUSS] FLIP-143: Unified Sink API

classic Classic list List threaded Threaded
61 messages Options
1234
Reply | Threaded
Open this post in threaded view
|

Re: [DISCUSS] FLIP-143: Unified Sink API

Steven Wu
> AFAIK the committer would not see the file-1-2 when ck1 happens in the
ExactlyOnce mode.

@Guowei Ma <[hidden email]> I think you are right for exactly once
checkpoint semantics. what about "at least once"? I guess we can argue that
it is fine to commit file-1-2 for at least once mode.

I still like the concept of grouping data files per checkpoint for
streaming mode. it is cleaner and probably easier to manage and deal with
commit failures. Plus, it can reduce dupes for the at least once mode.  I
understand checkpoint is not an option for batch execution. We don't have
to expose the checkpointId in API, as long as  the internal bookkeeping
groups data files by checkpoints for streaming mode.


On Tue, Sep 15, 2020 at 6:58 AM Steven Wu <[hidden email]> wrote:

> > images don't make it through to the mailing lists. You would need to
> host the file somewhere and send a link.
>
> Sorry about that. Here is the sample DAG in google drawings.
>
> https://docs.google.com/drawings/d/1-P8F2jF9RG9HHTtAfWEBRuU_2uV9aDTdqEt5dLs2JPk/edit?usp=sharing
>
>
> On Tue, Sep 15, 2020 at 4:58 AM Guowei Ma <[hidden email]> wrote:
>
>> Hi, Dawid
>>
>> >>I still find the merging case the most confusing. I don't necessarily
>> understand why do you need the "SingleFileCommit" step in this scenario.
>> The way I
>> >> understand "commit" operation is that it makes some data/artifacts
>> visible to the external system, thus it should be immutable from a point
>> of
>> view of a single >>process. Having an additional step in the same process
>> that works on committed data contradicts with those assumptions. I might
>> be
>> missing something though. >> Could you elaborate >why can't it be
>> something
>> like FileWriter -> FileMergeWriter -> Committer (either global or
>> non-global)? Again it might be just me not getting the example.
>>
>> I think you are right. The topology
>> "FileWriter->FileMergeWriter->Committer" could meet the merge requirement.
>> The topology "FileWriter-> SingleFileCommitter -> FileMergeWriter ->
>> GlobalCommitter" reuses some code of the StreamingFileSink(For example
>> rolling policy) so it has the "SingleFileCommitter" in the topology. In
>> general I want to use the case to show that there are different topologies
>> according to the requirements.
>>
>> BTW: IIRC, @Jingsong Lee <[hidden email]> telled me that the
>> actual topology of merged supported HiveSink is more complicated than
>> that.
>>
>>
>> >> I've just briefly skimmed over the proposed interfaces. I would suggest
>> one
>> >> addition to the Writer interface (as I understand this is the runtime
>> >> interface in this proposal?): add some availability method, to avoid,
>> if
>> >> possible, blocking calls on the sink. We already have similar
>> >> availability methods in the new sources [1] and in various places in
>> the
>> >> network stack [2].
>> >> BTW Let's not forget about Piotr's comment. I think we could add the
>> isAvailable or similar method to the Writer interface in the FLIP.
>>
>> Thanks @Dawid Wysakowicz <[hidden email]>  for your reminder.
>> There
>> are two many issues at the same time.
>>
>> In addition to what Ajjoscha said : there is very little system support
>> it.   Another thing I worry about is that: Does the sink's snapshot return
>> immediately when the sink's status is unavailable? Maybe we could do it by
>> dedupe some element in the state but I think it might be too complicated.
>> For me I want to know is what specific sink will benefit from this
>> feature.  @piotr <[hidden email]>  Please correct me if  I
>> misunderstand you. thanks.
>>
>> Best,
>> Guowei
>>
>>
>> On Tue, Sep 15, 2020 at 3:55 PM Dawid Wysakowicz <[hidden email]>
>> wrote:
>>
>> > What I understand is that HiveSink's implementation might need the local
>> > committer(FileCommitter) because the file rename is needed.
>> > But the iceberg only needs to write the manifest file.  Would you like
>> to
>> > enlighten me why the Iceberg needs the local committer?
>> > Thanks
>> >
>> > Sorry if I caused a confusion here. I am not saying the Iceberg sink
>> needs
>> > a local committer. What I had in mind is that prior to the Iceberg
>> example
>> > I did not see a need for a "GlobalCommitter" in the streaming case. I
>> > thought it is always enough to have the "normal" committer in that case.
>> > Now I understand that this differentiation is not really about logical
>> > separation. It is not really about the granularity with which we commit,
>> > i.e. answering the "WHAT" question. It is really about the performance
>> and
>> > that in the end we will have a single "transaction", so it is about
>> > answering the question "HOW".
>> >
>> >
>> >    -
>> >
>> >    Commit a directory with merged files(Some user want to merge the
>> files
>> >    in a directory before committing the directory to Hive meta store)
>> >
>> >
>> >    1.
>> >
>> >    FileWriter -> SingleFileCommit -> FileMergeWriter  -> GlobalCommitter
>> >
>> > I still find the merging case the most confusing. I don't necessarily
>> > understand why do you need the "SingleFileCommit" step in this scenario.
>> > The way I understand "commit" operation is that it makes some
>> > data/artifacts visible to the external system, thus it should be
>> immutable
>> > from a point of view of a single process. Having an additional step in
>> the
>> > same process that works on committed data contradicts with those
>> > assumptions. I might be missing something though. Could you elaborate
>> why
>> > can't it be something like FileWriter -> FileMergeWriter -> Committer
>> > (either global or non-global)? Again it might be just me not getting the
>> > example.
>> >
>> > I've just briefly skimmed over the proposed interfaces. I would suggest
>> one
>> > addition to the Writer interface (as I understand this is the runtime
>> > interface in this proposal?): add some availability method, to avoid, if
>> > possible, blocking calls on the sink. We already have similar
>> > availability methods in the new sources [1] and in various places in the
>> > network stack [2].
>> >
>> > BTW Let's not forget about Piotr's comment. I think we could add the
>> > isAvailable or similar method to the Writer interface in the FLIP.
>> >
>> > Best,
>> >
>> > Dawid
>> > On 15/09/2020 08:06, Guowei Ma wrote:
>> >
>> > I would think that we only need flush() and the semantics are that it
>> > prepares for a commit, so on a physical level it would be called from
>> > "prepareSnapshotPreBarrier". Now that I'm thinking about it more I
>> > think flush() should be renamed to something like "prepareCommit()".
>> >
>> > Generally speaking it is a good point that emitting the committables
>> > should happen before emitting the checkpoint barrier downstream.
>> > However, if I remember offline discussions well, the idea behind
>> > Writer#flush and Writer#snapshotState was to differentiate commit on
>> > checkpoint vs final checkpoint at the end of the job. Both of these
>> > methods could emit committables, but the flush should not leave any in
>> > progress state (e.g. in case of file sink in STREAM mode, in
>> > snapshotState it could leave some open files that would be committed in
>> > a subsequent cycle, however flush should close all files). The
>> > snapshotState as it is now can not be called in
>> > prepareSnapshotPreBarrier as it can store some state, which should
>> > happen in Operator#snapshotState as otherwise it would always be
>> > synchronous. Therefore I think we would need sth like:
>> >
>> > void prepareCommit(boolean flush, WriterOutput<CommT> output);
>> >
>> > ver 1:
>> >
>> > List<StateT> snapshotState();
>> >
>> > ver 2:
>> >
>> > void snapshotState(); // not sure if we need that method at all in
>> option
>> >
>> > 2
>> >
>> > I second Dawid's proposal. This is a valid scenario. And version2 does
>> not
>> > need the snapshotState() any more.
>> >
>> >
>> > The Committer is as described in the FLIP, it's basically a function
>> > "void commit(Committable)". The GobalCommitter would be a function "void
>> > commit(List<Committable>)". The former would be used by an S3 sink where
>> > we can individually commit files to S3, a committable would be the list
>> > of part uploads that will form the final file and the commit operation
>> > creates the metadata in S3. The latter would be used by something like
>> > Iceberg where the Committer needs a global view of all the commits to be
>> > efficient and not overwhelm the system.
>> >
>> > I don't know yet if sinks would only implement on type of commit
>> > function or potentially both at the same time, and maybe Commit can
>> > return some CommitResult that gets shipped to the GlobalCommit function.
>> > I must admit it I did not get the need for Local/Normal + Global
>> > committer at first. The Iceberg example helped a lot. I think it makes a
>> > lot of sense.
>> >
>> > @Dawid
>> > What I understand is that HiveSink's implementation might need the local
>> > committer(FileCommitter) because the file rename is needed.
>> > But the iceberg only needs to write the manifest file.  Would you like
>> to
>> > enlighten me why the Iceberg needs the local committer?
>> > Thanks
>> >
>> > Best,
>> > Guowei
>> >
>> >
>> > On Mon, Sep 14, 2020 at 11:19 PM Dawid Wysakowicz <
>> [hidden email]> <[hidden email]>
>> > wrote:
>> >
>> >
>> > Hi all,
>> >
>> >
>> > I would think that we only need flush() and the semantics are that it
>> > prepares for a commit, so on a physical level it would be called from
>> > "prepareSnapshotPreBarrier". Now that I'm thinking about it more I
>> > think flush() should be renamed to something like "prepareCommit()".
>> >
>> > Generally speaking it is a good point that emitting the committables
>> > should happen before emitting the checkpoint barrier downstream.
>> > However, if I remember offline discussions well, the idea behind
>> > Writer#flush and Writer#snapshotState was to differentiate commit on
>> > checkpoint vs final checkpoint at the end of the job. Both of these
>> > methods could emit committables, but the flush should not leave any in
>> > progress state (e.g. in case of file sink in STREAM mode, in
>> > snapshotState it could leave some open files that would be committed in
>> > a subsequent cycle, however flush should close all files). The
>> > snapshotState as it is now can not be called in
>> > prepareSnapshotPreBarrier as it can store some state, which should
>> > happen in Operator#snapshotState as otherwise it would always be
>> > synchronous. Therefore I think we would need sth like:
>> >
>> > void prepareCommit(boolean flush, WriterOutput<CommT> output);
>> >
>> > ver 1:
>> >
>> > List<StateT> snapshotState();
>> >
>> > ver 2:
>> >
>> > void snapshotState(); // not sure if we need that method at all in
>> option 2
>> >
>> >
>> > The Committer is as described in the FLIP, it's basically a function
>> > "void commit(Committable)". The GobalCommitter would be a function "void
>> > commit(List<Committable>)". The former would be used by an S3 sink where
>> > we can individually commit files to S3, a committable would be the list
>> > of part uploads that will form the final file and the commit operation
>> > creates the metadata in S3. The latter would be used by something like
>> > Iceberg where the Committer needs a global view of all the commits to be
>> > efficient and not overwhelm the system.
>> >
>> > I don't know yet if sinks would only implement on type of commit
>> > function or potentially both at the same time, and maybe Commit can
>> > return some CommitResult that gets shipped to the GlobalCommit function.
>> >
>> > I must admit it I did not get the need for Local/Normal + Global
>> > committer at first. The Iceberg example helped a lot. I think it makes a
>> > lot of sense.
>> >
>> >
>> > For Iceberg, writers don't need any state. But the GlobalCommitter
>> > needs to
>> > checkpoint StateT. For the committer, CommT is "DataFile". Since a
>> single
>> > committer can collect thousands (or more) data files in one checkpoint
>> > cycle, as an optimization we checkpoint a single "ManifestFile" (for the
>> > collected thousands data files) as StateT. This allows us to absorb
>> > extended commit outages without losing written/uploaded data files, as
>> > operator state size is as small as one manifest file per checkpoint
>> cycle
>> > [2].
>> > ------------------
>> > StateT snapshotState(SnapshotContext context) throws Exception;
>> >
>> > That means we also need the restoreCommitter API in the Sink interface
>> > ---------------
>> > Committer<CommT, StateT> restoreCommitter(InitContext context, StateT
>> > state);
>> >
>> > I think this might be a valid case. Not sure though if I would go with a
>> > "state" there. Having a state in a committer would imply we need a
>> > collect method as well. So far we needed a single method commit(...) and
>> > the bookkeeping of the committables could be handled by the framework. I
>> > think something like an optional combiner in the GlobalCommitter would
>> > be enough. What do you think?
>> >
>> > GlobalCommitter<CommT, GlobalCommT> {
>> >
>> >     void commit(GlobalCommT globalCommittables);
>> >
>> >     GlobalCommT combine(List<CommT> committables);
>> >
>> > }
>> >
>> > A different problem that I see here is how do we handle commit failures.
>> > Should the committables (both normal and global be included in the next
>> > cycle, shall we retry it, ...) I think it would be worth laying it out
>> > in the FLIP.
>> >
>> > @Aljoscha I think you can find the code Steven was referring in here:
>> >
>> https://github.com/Netflix-Skunkworks/nfflink-connector-iceberg/blob/master/nfflink-connector-iceberg/src/main/java/com/netflix/spaas/nfflink/connector/iceberg/sink/IcebergCommitter.java
>> >
>> > Best,
>> >
>> > Dawid
>> >
>> > On 14/09/2020 15:19, Aljoscha Krettek wrote:
>> >
>> > On 14.09.20 01:23, Steven Wu wrote:
>> >
>> > ## Writer interface
>> >
>> > For the Writer interface, should we add "*prepareSnapshot"* before the
>> > checkpoint barrier emitted downstream?  IcebergWriter would need it. Or
>> > would the framework call "*flush*" before the barrier emitted
>> > downstream?
>> > that guarantee would achieve the same goal.
>> >
>> > I would think that we only need flush() and the semantics are that it
>> > prepares for a commit, so on a physical level it would be called from
>> > "prepareSnapshotPreBarrier". Now that I'm thinking about it more I
>> > think flush() should be renamed to something like "prepareCommit()".
>> >
>> > @Guowei, what do you think about this?
>> >
>> >
>> > In [1], we discussed the reason for Writer to emit (checkpointId, CommT)
>> > tuple to the committer. The committer needs checkpointId to separate out
>> > data files for different checkpoints if concurrent checkpoints are
>> > enabled.
>> >
>> > When can this happen? Even with concurrent checkpoints the snapshot
>> > barriers would still cleanly segregate the input stream of an operator
>> > into tranches that should manifest in only one checkpoint. With
>> > concurrent checkpoints, all that can happen is that we start a
>> > checkpoint before a last one is confirmed completed.
>> >
>> > Unless there is some weirdness in the sources and some sources start
>> > chk1 first and some other ones start chk2 first?
>> >
>> > @Piotrek, do you think this is a problem?
>> >
>> >
>> > For the Committer interface, I am wondering if we should split the
>> > single
>> > commit method into separate "*collect"* and "*commit"* methods? This
>> > way,
>> > it can handle both single and multiple CommT objects.
>> >
>> > I think we can't do this. If the sink only needs a regular Commiter,
>> > we can perform the commits in parallel, possibly on different
>> > machines. Only when the sink needs a GlobalCommitter would we need to
>> > ship all commits to a single process and perform the commit there. If
>> > both methods were unified in one interface we couldn't make the
>> > decision of were to commit in the framework code.
>> >
>> >
>> > For Iceberg, writers don't need any state. But the GlobalCommitter
>> > needs to
>> > checkpoint StateT. For the committer, CommT is "DataFile". Since a
>> > single
>> > committer can collect thousands (or more) data files in one checkpoint
>> > cycle, as an optimization we checkpoint a single "ManifestFile" (for the
>> > collected thousands data files) as StateT. This allows us to absorb
>> > extended commit outages without losing written/uploaded data files, as
>> > operator state size is as small as one manifest file per checkpoint
>> > cycle
>> >
>> > You could have a point here. Is the code for this available in
>> > open-source? I was checking out
>> >
>> >
>> >
>> https://github.com/apache/iceberg/blob/master/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergFilesCommitter.java
>> >
>> > and didn't find the ManifestFile optimization there.
>> >
>> > Best,
>> > Aljoscha
>> >
>> >
>> >
>>
>
Reply | Threaded
Open this post in threaded view
|

Re: [DISCUSS] FLIP-143: Unified Sink API

Guowei Ma
Hi, Steven
Thanks you for your thoughtful ideas and concerns.

>>I still like the concept of grouping data files per checkpoint for
streaming mode. it is cleaner and probably easier to manage and deal with
commit failures. Plus, it >>can reduce dupes for the at least once
>>mode.  I understand checkpoint is not an option for batch execution. We
don't have to expose the checkpointId in API, as >>long as  the internal
bookkeeping groups data files by checkpoints for streaming >>mode.

I think this problem(How to dedupe the combined committed data) also
depends on where to place the agg/combine logic .

1. If the agg/combine takes place in the “commit” maybe we need to figure
out how to give the aggregated committable a unique and auto-increment id
in the committer.
2. If the agg/combine takes place in a separate operator maybe sink
developer could maintain the id itself by using the state.

I think this problem is also decided by what the topology pattern the sink
API should support. Actually there are already many other topology
requirements. :)

Best,
Guowei


On Wed, Sep 16, 2020 at 7:46 AM Steven Wu <[hidden email]> wrote:

> > AFAIK the committer would not see the file-1-2 when ck1 happens in the
> ExactlyOnce mode.
>
> @Guowei Ma <[hidden email]> I think you are right for exactly once
> checkpoint semantics. what about "at least once"? I guess we can argue that
> it is fine to commit file-1-2 for at least once mode.
>
> I still like the concept of grouping data files per checkpoint for
> streaming mode. it is cleaner and probably easier to manage and deal with
> commit failures. Plus, it can reduce dupes for the at least once mode.  I
> understand checkpoint is not an option for batch execution. We don't have
> to expose the checkpointId in API, as long as  the internal bookkeeping
> groups data files by checkpoints for streaming mode.
>
>
> On Tue, Sep 15, 2020 at 6:58 AM Steven Wu <[hidden email]> wrote:
>
>> > images don't make it through to the mailing lists. You would need to
>> host the file somewhere and send a link.
>>
>> Sorry about that. Here is the sample DAG in google drawings.
>>
>> https://docs.google.com/drawings/d/1-P8F2jF9RG9HHTtAfWEBRuU_2uV9aDTdqEt5dLs2JPk/edit?usp=sharing
>>
>>
>> On Tue, Sep 15, 2020 at 4:58 AM Guowei Ma <[hidden email]> wrote:
>>
>>> Hi, Dawid
>>>
>>> >>I still find the merging case the most confusing. I don't necessarily
>>> understand why do you need the "SingleFileCommit" step in this scenario.
>>> The way I
>>> >> understand "commit" operation is that it makes some data/artifacts
>>> visible to the external system, thus it should be immutable from a point
>>> of
>>> view of a single >>process. Having an additional step in the same process
>>> that works on committed data contradicts with those assumptions. I might
>>> be
>>> missing something though. >> Could you elaborate >why can't it be
>>> something
>>> like FileWriter -> FileMergeWriter -> Committer (either global or
>>> non-global)? Again it might be just me not getting the example.
>>>
>>> I think you are right. The topology
>>> "FileWriter->FileMergeWriter->Committer" could meet the merge
>>> requirement.
>>> The topology "FileWriter-> SingleFileCommitter -> FileMergeWriter ->
>>> GlobalCommitter" reuses some code of the StreamingFileSink(For example
>>> rolling policy) so it has the "SingleFileCommitter" in the topology. In
>>> general I want to use the case to show that there are different
>>> topologies
>>> according to the requirements.
>>>
>>> BTW: IIRC, @Jingsong Lee <[hidden email]> telled me that the
>>> actual topology of merged supported HiveSink is more complicated than
>>> that.
>>>
>>>
>>> >> I've just briefly skimmed over the proposed interfaces. I would
>>> suggest
>>> one
>>> >> addition to the Writer interface (as I understand this is the runtime
>>> >> interface in this proposal?): add some availability method, to avoid,
>>> if
>>> >> possible, blocking calls on the sink. We already have similar
>>> >> availability methods in the new sources [1] and in various places in
>>> the
>>> >> network stack [2].
>>> >> BTW Let's not forget about Piotr's comment. I think we could add the
>>> isAvailable or similar method to the Writer interface in the FLIP.
>>>
>>> Thanks @Dawid Wysakowicz <[hidden email]>  for your reminder.
>>> There
>>> are two many issues at the same time.
>>>
>>> In addition to what Ajjoscha said : there is very little system support
>>> it.   Another thing I worry about is that: Does the sink's snapshot
>>> return
>>> immediately when the sink's status is unavailable? Maybe we could do it
>>> by
>>> dedupe some element in the state but I think it might be too complicated.
>>> For me I want to know is what specific sink will benefit from this
>>> feature.  @piotr <[hidden email]>  Please correct me if  I
>>> misunderstand you. thanks.
>>>
>>> Best,
>>> Guowei
>>>
>>>
>>> On Tue, Sep 15, 2020 at 3:55 PM Dawid Wysakowicz <[hidden email]
>>> >
>>> wrote:
>>>
>>> > What I understand is that HiveSink's implementation might need the
>>> local
>>> > committer(FileCommitter) because the file rename is needed.
>>> > But the iceberg only needs to write the manifest file.  Would you like
>>> to
>>> > enlighten me why the Iceberg needs the local committer?
>>> > Thanks
>>> >
>>> > Sorry if I caused a confusion here. I am not saying the Iceberg sink
>>> needs
>>> > a local committer. What I had in mind is that prior to the Iceberg
>>> example
>>> > I did not see a need for a "GlobalCommitter" in the streaming case. I
>>> > thought it is always enough to have the "normal" committer in that
>>> case.
>>> > Now I understand that this differentiation is not really about logical
>>> > separation. It is not really about the granularity with which we
>>> commit,
>>> > i.e. answering the "WHAT" question. It is really about the performance
>>> and
>>> > that in the end we will have a single "transaction", so it is about
>>> > answering the question "HOW".
>>> >
>>> >
>>> >    -
>>> >
>>> >    Commit a directory with merged files(Some user want to merge the
>>> files
>>> >    in a directory before committing the directory to Hive meta store)
>>> >
>>> >
>>> >    1.
>>> >
>>> >    FileWriter -> SingleFileCommit -> FileMergeWriter  ->
>>> GlobalCommitter
>>> >
>>> > I still find the merging case the most confusing. I don't necessarily
>>> > understand why do you need the "SingleFileCommit" step in this
>>> scenario.
>>> > The way I understand "commit" operation is that it makes some
>>> > data/artifacts visible to the external system, thus it should be
>>> immutable
>>> > from a point of view of a single process. Having an additional step in
>>> the
>>> > same process that works on committed data contradicts with those
>>> > assumptions. I might be missing something though. Could you elaborate
>>> why
>>> > can't it be something like FileWriter -> FileMergeWriter -> Committer
>>> > (either global or non-global)? Again it might be just me not getting
>>> the
>>> > example.
>>> >
>>> > I've just briefly skimmed over the proposed interfaces. I would
>>> suggest one
>>> > addition to the Writer interface (as I understand this is the runtime
>>> > interface in this proposal?): add some availability method, to avoid,
>>> if
>>> > possible, blocking calls on the sink. We already have similar
>>> > availability methods in the new sources [1] and in various places in
>>> the
>>> > network stack [2].
>>> >
>>> > BTW Let's not forget about Piotr's comment. I think we could add the
>>> > isAvailable or similar method to the Writer interface in the FLIP.
>>> >
>>> > Best,
>>> >
>>> > Dawid
>>> > On 15/09/2020 08:06, Guowei Ma wrote:
>>> >
>>> > I would think that we only need flush() and the semantics are that it
>>> > prepares for a commit, so on a physical level it would be called from
>>> > "prepareSnapshotPreBarrier". Now that I'm thinking about it more I
>>> > think flush() should be renamed to something like "prepareCommit()".
>>> >
>>> > Generally speaking it is a good point that emitting the committables
>>> > should happen before emitting the checkpoint barrier downstream.
>>> > However, if I remember offline discussions well, the idea behind
>>> > Writer#flush and Writer#snapshotState was to differentiate commit on
>>> > checkpoint vs final checkpoint at the end of the job. Both of these
>>> > methods could emit committables, but the flush should not leave any in
>>> > progress state (e.g. in case of file sink in STREAM mode, in
>>> > snapshotState it could leave some open files that would be committed in
>>> > a subsequent cycle, however flush should close all files). The
>>> > snapshotState as it is now can not be called in
>>> > prepareSnapshotPreBarrier as it can store some state, which should
>>> > happen in Operator#snapshotState as otherwise it would always be
>>> > synchronous. Therefore I think we would need sth like:
>>> >
>>> > void prepareCommit(boolean flush, WriterOutput<CommT> output);
>>> >
>>> > ver 1:
>>> >
>>> > List<StateT> snapshotState();
>>> >
>>> > ver 2:
>>> >
>>> > void snapshotState(); // not sure if we need that method at all in
>>> option
>>> >
>>> > 2
>>> >
>>> > I second Dawid's proposal. This is a valid scenario. And version2 does
>>> not
>>> > need the snapshotState() any more.
>>> >
>>> >
>>> > The Committer is as described in the FLIP, it's basically a function
>>> > "void commit(Committable)". The GobalCommitter would be a function
>>> "void
>>> > commit(List<Committable>)". The former would be used by an S3 sink
>>> where
>>> > we can individually commit files to S3, a committable would be the list
>>> > of part uploads that will form the final file and the commit operation
>>> > creates the metadata in S3. The latter would be used by something like
>>> > Iceberg where the Committer needs a global view of all the commits to
>>> be
>>> > efficient and not overwhelm the system.
>>> >
>>> > I don't know yet if sinks would only implement on type of commit
>>> > function or potentially both at the same time, and maybe Commit can
>>> > return some CommitResult that gets shipped to the GlobalCommit
>>> function.
>>> > I must admit it I did not get the need for Local/Normal + Global
>>> > committer at first. The Iceberg example helped a lot. I think it makes
>>> a
>>> > lot of sense.
>>> >
>>> > @Dawid
>>> > What I understand is that HiveSink's implementation might need the
>>> local
>>> > committer(FileCommitter) because the file rename is needed.
>>> > But the iceberg only needs to write the manifest file.  Would you like
>>> to
>>> > enlighten me why the Iceberg needs the local committer?
>>> > Thanks
>>> >
>>> > Best,
>>> > Guowei
>>> >
>>> >
>>> > On Mon, Sep 14, 2020 at 11:19 PM Dawid Wysakowicz <
>>> [hidden email]> <[hidden email]>
>>> > wrote:
>>> >
>>> >
>>> > Hi all,
>>> >
>>> >
>>> > I would think that we only need flush() and the semantics are that it
>>> > prepares for a commit, so on a physical level it would be called from
>>> > "prepareSnapshotPreBarrier". Now that I'm thinking about it more I
>>> > think flush() should be renamed to something like "prepareCommit()".
>>> >
>>> > Generally speaking it is a good point that emitting the committables
>>> > should happen before emitting the checkpoint barrier downstream.
>>> > However, if I remember offline discussions well, the idea behind
>>> > Writer#flush and Writer#snapshotState was to differentiate commit on
>>> > checkpoint vs final checkpoint at the end of the job. Both of these
>>> > methods could emit committables, but the flush should not leave any in
>>> > progress state (e.g. in case of file sink in STREAM mode, in
>>> > snapshotState it could leave some open files that would be committed in
>>> > a subsequent cycle, however flush should close all files). The
>>> > snapshotState as it is now can not be called in
>>> > prepareSnapshotPreBarrier as it can store some state, which should
>>> > happen in Operator#snapshotState as otherwise it would always be
>>> > synchronous. Therefore I think we would need sth like:
>>> >
>>> > void prepareCommit(boolean flush, WriterOutput<CommT> output);
>>> >
>>> > ver 1:
>>> >
>>> > List<StateT> snapshotState();
>>> >
>>> > ver 2:
>>> >
>>> > void snapshotState(); // not sure if we need that method at all in
>>> option 2
>>> >
>>> >
>>> > The Committer is as described in the FLIP, it's basically a function
>>> > "void commit(Committable)". The GobalCommitter would be a function
>>> "void
>>> > commit(List<Committable>)". The former would be used by an S3 sink
>>> where
>>> > we can individually commit files to S3, a committable would be the list
>>> > of part uploads that will form the final file and the commit operation
>>> > creates the metadata in S3. The latter would be used by something like
>>> > Iceberg where the Committer needs a global view of all the commits to
>>> be
>>> > efficient and not overwhelm the system.
>>> >
>>> > I don't know yet if sinks would only implement on type of commit
>>> > function or potentially both at the same time, and maybe Commit can
>>> > return some CommitResult that gets shipped to the GlobalCommit
>>> function.
>>> >
>>> > I must admit it I did not get the need for Local/Normal + Global
>>> > committer at first. The Iceberg example helped a lot. I think it makes
>>> a
>>> > lot of sense.
>>> >
>>> >
>>> > For Iceberg, writers don't need any state. But the GlobalCommitter
>>> > needs to
>>> > checkpoint StateT. For the committer, CommT is "DataFile". Since a
>>> single
>>> > committer can collect thousands (or more) data files in one checkpoint
>>> > cycle, as an optimization we checkpoint a single "ManifestFile" (for
>>> the
>>> > collected thousands data files) as StateT. This allows us to absorb
>>> > extended commit outages without losing written/uploaded data files, as
>>> > operator state size is as small as one manifest file per checkpoint
>>> cycle
>>> > [2].
>>> > ------------------
>>> > StateT snapshotState(SnapshotContext context) throws Exception;
>>> >
>>> > That means we also need the restoreCommitter API in the Sink interface
>>> > ---------------
>>> > Committer<CommT, StateT> restoreCommitter(InitContext context, StateT
>>> > state);
>>> >
>>> > I think this might be a valid case. Not sure though if I would go with
>>> a
>>> > "state" there. Having a state in a committer would imply we need a
>>> > collect method as well. So far we needed a single method commit(...)
>>> and
>>> > the bookkeeping of the committables could be handled by the framework.
>>> I
>>> > think something like an optional combiner in the GlobalCommitter would
>>> > be enough. What do you think?
>>> >
>>> > GlobalCommitter<CommT, GlobalCommT> {
>>> >
>>> >     void commit(GlobalCommT globalCommittables);
>>> >
>>> >     GlobalCommT combine(List<CommT> committables);
>>> >
>>> > }
>>> >
>>> > A different problem that I see here is how do we handle commit
>>> failures.
>>> > Should the committables (both normal and global be included in the next
>>> > cycle, shall we retry it, ...) I think it would be worth laying it out
>>> > in the FLIP.
>>> >
>>> > @Aljoscha I think you can find the code Steven was referring in here:
>>> >
>>> https://github.com/Netflix-Skunkworks/nfflink-connector-iceberg/blob/master/nfflink-connector-iceberg/src/main/java/com/netflix/spaas/nfflink/connector/iceberg/sink/IcebergCommitter.java
>>> >
>>> > Best,
>>> >
>>> > Dawid
>>> >
>>> > On 14/09/2020 15:19, Aljoscha Krettek wrote:
>>> >
>>> > On 14.09.20 01:23, Steven Wu wrote:
>>> >
>>> > ## Writer interface
>>> >
>>> > For the Writer interface, should we add "*prepareSnapshot"* before the
>>> > checkpoint barrier emitted downstream?  IcebergWriter would need it. Or
>>> > would the framework call "*flush*" before the barrier emitted
>>> > downstream?
>>> > that guarantee would achieve the same goal.
>>> >
>>> > I would think that we only need flush() and the semantics are that it
>>> > prepares for a commit, so on a physical level it would be called from
>>> > "prepareSnapshotPreBarrier". Now that I'm thinking about it more I
>>> > think flush() should be renamed to something like "prepareCommit()".
>>> >
>>> > @Guowei, what do you think about this?
>>> >
>>> >
>>> > In [1], we discussed the reason for Writer to emit (checkpointId,
>>> CommT)
>>> > tuple to the committer. The committer needs checkpointId to separate
>>> out
>>> > data files for different checkpoints if concurrent checkpoints are
>>> > enabled.
>>> >
>>> > When can this happen? Even with concurrent checkpoints the snapshot
>>> > barriers would still cleanly segregate the input stream of an operator
>>> > into tranches that should manifest in only one checkpoint. With
>>> > concurrent checkpoints, all that can happen is that we start a
>>> > checkpoint before a last one is confirmed completed.
>>> >
>>> > Unless there is some weirdness in the sources and some sources start
>>> > chk1 first and some other ones start chk2 first?
>>> >
>>> > @Piotrek, do you think this is a problem?
>>> >
>>> >
>>> > For the Committer interface, I am wondering if we should split the
>>> > single
>>> > commit method into separate "*collect"* and "*commit"* methods? This
>>> > way,
>>> > it can handle both single and multiple CommT objects.
>>> >
>>> > I think we can't do this. If the sink only needs a regular Commiter,
>>> > we can perform the commits in parallel, possibly on different
>>> > machines. Only when the sink needs a GlobalCommitter would we need to
>>> > ship all commits to a single process and perform the commit there. If
>>> > both methods were unified in one interface we couldn't make the
>>> > decision of were to commit in the framework code.
>>> >
>>> >
>>> > For Iceberg, writers don't need any state. But the GlobalCommitter
>>> > needs to
>>> > checkpoint StateT. For the committer, CommT is "DataFile". Since a
>>> > single
>>> > committer can collect thousands (or more) data files in one checkpoint
>>> > cycle, as an optimization we checkpoint a single "ManifestFile" (for
>>> the
>>> > collected thousands data files) as StateT. This allows us to absorb
>>> > extended commit outages without losing written/uploaded data files, as
>>> > operator state size is as small as one manifest file per checkpoint
>>> > cycle
>>> >
>>> > You could have a point here. Is the code for this available in
>>> > open-source? I was checking out
>>> >
>>> >
>>> >
>>> https://github.com/apache/iceberg/blob/master/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergFilesCommitter.java
>>> >
>>> > and didn't find the ManifestFile optimization there.
>>> >
>>> > Best,
>>> > Aljoscha
>>> >
>>> >
>>> >
>>>
>>
Reply | Threaded
Open this post in threaded view
|

Re: [DISCUSS] FLIP-143: Unified Sink API

Guowei Ma
Hi,all

Thanks for all your valuable options and ideas.Currently there are many
topics in the mail. I try to summarize what is consensus and what is not.
Correct me if I am wrong.

## Consensus

1. The motivation of the unified sink API is to decouple the sink
implementation from the different runtime execution mode.
2. The initial scope of the unified sink API only covers the file system
type, which supports the real transactions. The FLIP focuses more on the
semantics the new sink api should support.
3. We prefer the first alternative API, which could give the framework a
greater opportunity to optimize.
4. The `Writer` needs to add a method `prepareCommit`, which would be
called from `prepareSnapshotPreBarrier`. And remove the `Flush` method.
5. The FLIP could move the `Snapshot & Drain` section in order to be more
focused.

## Not Consensus

1. What should the “Unified Sink API” support/cover? The API can
“unified”(decoupe) the commit operation in the term of supporting exactly
once semantics. However, even if we narrow down the initial supported
system to the file system there would be different topology requirements.
These requirements come from performance optimization
(IceBergSink/MergeHiveSink) or functionality(i.e. whether a bucket is
“finished”).  Should the unified sink API support these requirements?
2. The API does not expose the checkpoint-id because the batch execution
mode does not have the normal checkpoint. But there still some
implementations depend on this.(IceBergSink uses this to do some dedupe).
I think how to support this requirement depends on the first open question.
3. Whether the `Writer` supports async functionality or not. Currently I do
not know which sink could benefit from it. Maybe it is just my own problem.

Best,
Guowei


On Wed, Sep 16, 2020 at 12:02 PM Guowei Ma <[hidden email]> wrote:

>
> Hi, Steven
> Thanks you for your thoughtful ideas and concerns.
>
> >>I still like the concept of grouping data files per checkpoint for
> streaming mode. it is cleaner and probably easier to manage and deal with
> commit failures. Plus, it >>can reduce dupes for the at least once
> >>mode.  I understand checkpoint is not an option for batch execution. We
> don't have to expose the checkpointId in API, as >>long as  the internal
> bookkeeping groups data files by checkpoints for streaming >>mode.
>
> I think this problem(How to dedupe the combined committed data) also
> depends on where to place the agg/combine logic .
>
> 1. If the agg/combine takes place in the “commit” maybe we need to figure
> out how to give the aggregated committable a unique and auto-increment id
> in the committer.
> 2. If the agg/combine takes place in a separate operator maybe sink
> developer could maintain the id itself by using the state.
>
> I think this problem is also decided by what the topology pattern the sink
> API should support. Actually there are already many other topology
> requirements. :)
>
> Best,
> Guowei
>
>
> On Wed, Sep 16, 2020 at 7:46 AM Steven Wu <[hidden email]> wrote:
>
>> > AFAIK the committer would not see the file-1-2 when ck1 happens in the
>> ExactlyOnce mode.
>>
>> @Guowei Ma <[hidden email]> I think you are right for exactly once
>> checkpoint semantics. what about "at least once"? I guess we can argue that
>> it is fine to commit file-1-2 for at least once mode.
>>
>> I still like the concept of grouping data files per checkpoint for
>> streaming mode. it is cleaner and probably easier to manage and deal with
>> commit failures. Plus, it can reduce dupes for the at least once mode.  I
>> understand checkpoint is not an option for batch execution. We don't have
>> to expose the checkpointId in API, as long as  the internal bookkeeping
>> groups data files by checkpoints for streaming mode.
>>
>>
>> On Tue, Sep 15, 2020 at 6:58 AM Steven Wu <[hidden email]> wrote:
>>
>>> > images don't make it through to the mailing lists. You would need to
>>> host the file somewhere and send a link.
>>>
>>> Sorry about that. Here is the sample DAG in google drawings.
>>>
>>> https://docs.google.com/drawings/d/1-P8F2jF9RG9HHTtAfWEBRuU_2uV9aDTdqEt5dLs2JPk/edit?usp=sharing
>>>
>>>
>>> On Tue, Sep 15, 2020 at 4:58 AM Guowei Ma <[hidden email]> wrote:
>>>
>>>> Hi, Dawid
>>>>
>>>> >>I still find the merging case the most confusing. I don't necessarily
>>>> understand why do you need the "SingleFileCommit" step in this scenario.
>>>> The way I
>>>> >> understand "commit" operation is that it makes some data/artifacts
>>>> visible to the external system, thus it should be immutable from a
>>>> point of
>>>> view of a single >>process. Having an additional step in the same
>>>> process
>>>> that works on committed data contradicts with those assumptions. I
>>>> might be
>>>> missing something though. >> Could you elaborate >why can't it be
>>>> something
>>>> like FileWriter -> FileMergeWriter -> Committer (either global or
>>>> non-global)? Again it might be just me not getting the example.
>>>>
>>>> I think you are right. The topology
>>>> "FileWriter->FileMergeWriter->Committer" could meet the merge
>>>> requirement.
>>>> The topology "FileWriter-> SingleFileCommitter -> FileMergeWriter ->
>>>> GlobalCommitter" reuses some code of the StreamingFileSink(For example
>>>> rolling policy) so it has the "SingleFileCommitter" in the topology. In
>>>> general I want to use the case to show that there are different
>>>> topologies
>>>> according to the requirements.
>>>>
>>>> BTW: IIRC, @Jingsong Lee <[hidden email]> telled me that
>>>> the
>>>> actual topology of merged supported HiveSink is more complicated than
>>>> that.
>>>>
>>>>
>>>> >> I've just briefly skimmed over the proposed interfaces. I would
>>>> suggest
>>>> one
>>>> >> addition to the Writer interface (as I understand this is the runtime
>>>> >> interface in this proposal?): add some availability method, to
>>>> avoid, if
>>>> >> possible, blocking calls on the sink. We already have similar
>>>> >> availability methods in the new sources [1] and in various places in
>>>> the
>>>> >> network stack [2].
>>>> >> BTW Let's not forget about Piotr's comment. I think we could add the
>>>> isAvailable or similar method to the Writer interface in the FLIP.
>>>>
>>>> Thanks @Dawid Wysakowicz <[hidden email]>  for your reminder.
>>>> There
>>>> are two many issues at the same time.
>>>>
>>>> In addition to what Ajjoscha said : there is very little system support
>>>> it.   Another thing I worry about is that: Does the sink's snapshot
>>>> return
>>>> immediately when the sink's status is unavailable? Maybe we could do it
>>>> by
>>>> dedupe some element in the state but I think it might be too
>>>> complicated.
>>>> For me I want to know is what specific sink will benefit from this
>>>> feature.  @piotr <[hidden email]>  Please correct me if  I
>>>> misunderstand you. thanks.
>>>>
>>>> Best,
>>>> Guowei
>>>>
>>>>
>>>> On Tue, Sep 15, 2020 at 3:55 PM Dawid Wysakowicz <
>>>> [hidden email]>
>>>> wrote:
>>>>
>>>> > What I understand is that HiveSink's implementation might need the
>>>> local
>>>> > committer(FileCommitter) because the file rename is needed.
>>>> > But the iceberg only needs to write the manifest file.  Would you
>>>> like to
>>>> > enlighten me why the Iceberg needs the local committer?
>>>> > Thanks
>>>> >
>>>> > Sorry if I caused a confusion here. I am not saying the Iceberg sink
>>>> needs
>>>> > a local committer. What I had in mind is that prior to the Iceberg
>>>> example
>>>> > I did not see a need for a "GlobalCommitter" in the streaming case. I
>>>> > thought it is always enough to have the "normal" committer in that
>>>> case.
>>>> > Now I understand that this differentiation is not really about logical
>>>> > separation. It is not really about the granularity with which we
>>>> commit,
>>>> > i.e. answering the "WHAT" question. It is really about the
>>>> performance and
>>>> > that in the end we will have a single "transaction", so it is about
>>>> > answering the question "HOW".
>>>> >
>>>> >
>>>> >    -
>>>> >
>>>> >    Commit a directory with merged files(Some user want to merge the
>>>> files
>>>> >    in a directory before committing the directory to Hive meta store)
>>>> >
>>>> >
>>>> >    1.
>>>> >
>>>> >    FileWriter -> SingleFileCommit -> FileMergeWriter  ->
>>>> GlobalCommitter
>>>> >
>>>> > I still find the merging case the most confusing. I don't necessarily
>>>> > understand why do you need the "SingleFileCommit" step in this
>>>> scenario.
>>>> > The way I understand "commit" operation is that it makes some
>>>> > data/artifacts visible to the external system, thus it should be
>>>> immutable
>>>> > from a point of view of a single process. Having an additional step
>>>> in the
>>>> > same process that works on committed data contradicts with those
>>>> > assumptions. I might be missing something though. Could you elaborate
>>>> why
>>>> > can't it be something like FileWriter -> FileMergeWriter -> Committer
>>>> > (either global or non-global)? Again it might be just me not getting
>>>> the
>>>> > example.
>>>> >
>>>> > I've just briefly skimmed over the proposed interfaces. I would
>>>> suggest one
>>>> > addition to the Writer interface (as I understand this is the runtime
>>>> > interface in this proposal?): add some availability method, to avoid,
>>>> if
>>>> > possible, blocking calls on the sink. We already have similar
>>>> > availability methods in the new sources [1] and in various places in
>>>> the
>>>> > network stack [2].
>>>> >
>>>> > BTW Let's not forget about Piotr's comment. I think we could add the
>>>> > isAvailable or similar method to the Writer interface in the FLIP.
>>>> >
>>>> > Best,
>>>> >
>>>> > Dawid
>>>> > On 15/09/2020 08:06, Guowei Ma wrote:
>>>> >
>>>> > I would think that we only need flush() and the semantics are that it
>>>> > prepares for a commit, so on a physical level it would be called from
>>>> > "prepareSnapshotPreBarrier". Now that I'm thinking about it more I
>>>> > think flush() should be renamed to something like "prepareCommit()".
>>>> >
>>>> > Generally speaking it is a good point that emitting the committables
>>>> > should happen before emitting the checkpoint barrier downstream.
>>>> > However, if I remember offline discussions well, the idea behind
>>>> > Writer#flush and Writer#snapshotState was to differentiate commit on
>>>> > checkpoint vs final checkpoint at the end of the job. Both of these
>>>> > methods could emit committables, but the flush should not leave any in
>>>> > progress state (e.g. in case of file sink in STREAM mode, in
>>>> > snapshotState it could leave some open files that would be committed
>>>> in
>>>> > a subsequent cycle, however flush should close all files). The
>>>> > snapshotState as it is now can not be called in
>>>> > prepareSnapshotPreBarrier as it can store some state, which should
>>>> > happen in Operator#snapshotState as otherwise it would always be
>>>> > synchronous. Therefore I think we would need sth like:
>>>> >
>>>> > void prepareCommit(boolean flush, WriterOutput<CommT> output);
>>>> >
>>>> > ver 1:
>>>> >
>>>> > List<StateT> snapshotState();
>>>> >
>>>> > ver 2:
>>>> >
>>>> > void snapshotState(); // not sure if we need that method at all in
>>>> option
>>>> >
>>>> > 2
>>>> >
>>>> > I second Dawid's proposal. This is a valid scenario. And version2
>>>> does not
>>>> > need the snapshotState() any more.
>>>> >
>>>> >
>>>> > The Committer is as described in the FLIP, it's basically a function
>>>> > "void commit(Committable)". The GobalCommitter would be a function
>>>> "void
>>>> > commit(List<Committable>)". The former would be used by an S3 sink
>>>> where
>>>> > we can individually commit files to S3, a committable would be the
>>>> list
>>>> > of part uploads that will form the final file and the commit operation
>>>> > creates the metadata in S3. The latter would be used by something like
>>>> > Iceberg where the Committer needs a global view of all the commits to
>>>> be
>>>> > efficient and not overwhelm the system.
>>>> >
>>>> > I don't know yet if sinks would only implement on type of commit
>>>> > function or potentially both at the same time, and maybe Commit can
>>>> > return some CommitResult that gets shipped to the GlobalCommit
>>>> function.
>>>> > I must admit it I did not get the need for Local/Normal + Global
>>>> > committer at first. The Iceberg example helped a lot. I think it
>>>> makes a
>>>> > lot of sense.
>>>> >
>>>> > @Dawid
>>>> > What I understand is that HiveSink's implementation might need the
>>>> local
>>>> > committer(FileCommitter) because the file rename is needed.
>>>> > But the iceberg only needs to write the manifest file.  Would you
>>>> like to
>>>> > enlighten me why the Iceberg needs the local committer?
>>>> > Thanks
>>>> >
>>>> > Best,
>>>> > Guowei
>>>> >
>>>> >
>>>> > On Mon, Sep 14, 2020 at 11:19 PM Dawid Wysakowicz <
>>>> [hidden email]> <[hidden email]>
>>>> > wrote:
>>>> >
>>>> >
>>>> > Hi all,
>>>> >
>>>> >
>>>> > I would think that we only need flush() and the semantics are that it
>>>> > prepares for a commit, so on a physical level it would be called from
>>>> > "prepareSnapshotPreBarrier". Now that I'm thinking about it more I
>>>> > think flush() should be renamed to something like "prepareCommit()".
>>>> >
>>>> > Generally speaking it is a good point that emitting the committables
>>>> > should happen before emitting the checkpoint barrier downstream.
>>>> > However, if I remember offline discussions well, the idea behind
>>>> > Writer#flush and Writer#snapshotState was to differentiate commit on
>>>> > checkpoint vs final checkpoint at the end of the job. Both of these
>>>> > methods could emit committables, but the flush should not leave any in
>>>> > progress state (e.g. in case of file sink in STREAM mode, in
>>>> > snapshotState it could leave some open files that would be committed
>>>> in
>>>> > a subsequent cycle, however flush should close all files). The
>>>> > snapshotState as it is now can not be called in
>>>> > prepareSnapshotPreBarrier as it can store some state, which should
>>>> > happen in Operator#snapshotState as otherwise it would always be
>>>> > synchronous. Therefore I think we would need sth like:
>>>> >
>>>> > void prepareCommit(boolean flush, WriterOutput<CommT> output);
>>>> >
>>>> > ver 1:
>>>> >
>>>> > List<StateT> snapshotState();
>>>> >
>>>> > ver 2:
>>>> >
>>>> > void snapshotState(); // not sure if we need that method at all in
>>>> option 2
>>>> >
>>>> >
>>>> > The Committer is as described in the FLIP, it's basically a function
>>>> > "void commit(Committable)". The GobalCommitter would be a function
>>>> "void
>>>> > commit(List<Committable>)". The former would be used by an S3 sink
>>>> where
>>>> > we can individually commit files to S3, a committable would be the
>>>> list
>>>> > of part uploads that will form the final file and the commit operation
>>>> > creates the metadata in S3. The latter would be used by something like
>>>> > Iceberg where the Committer needs a global view of all the commits to
>>>> be
>>>> > efficient and not overwhelm the system.
>>>> >
>>>> > I don't know yet if sinks would only implement on type of commit
>>>> > function or potentially both at the same time, and maybe Commit can
>>>> > return some CommitResult that gets shipped to the GlobalCommit
>>>> function.
>>>> >
>>>> > I must admit it I did not get the need for Local/Normal + Global
>>>> > committer at first. The Iceberg example helped a lot. I think it
>>>> makes a
>>>> > lot of sense.
>>>> >
>>>> >
>>>> > For Iceberg, writers don't need any state. But the GlobalCommitter
>>>> > needs to
>>>> > checkpoint StateT. For the committer, CommT is "DataFile". Since a
>>>> single
>>>> > committer can collect thousands (or more) data files in one checkpoint
>>>> > cycle, as an optimization we checkpoint a single "ManifestFile" (for
>>>> the
>>>> > collected thousands data files) as StateT. This allows us to absorb
>>>> > extended commit outages without losing written/uploaded data files, as
>>>> > operator state size is as small as one manifest file per checkpoint
>>>> cycle
>>>> > [2].
>>>> > ------------------
>>>> > StateT snapshotState(SnapshotContext context) throws Exception;
>>>> >
>>>> > That means we also need the restoreCommitter API in the Sink interface
>>>> > ---------------
>>>> > Committer<CommT, StateT> restoreCommitter(InitContext context, StateT
>>>> > state);
>>>> >
>>>> > I think this might be a valid case. Not sure though if I would go
>>>> with a
>>>> > "state" there. Having a state in a committer would imply we need a
>>>> > collect method as well. So far we needed a single method commit(...)
>>>> and
>>>> > the bookkeeping of the committables could be handled by the
>>>> framework. I
>>>> > think something like an optional combiner in the GlobalCommitter would
>>>> > be enough. What do you think?
>>>> >
>>>> > GlobalCommitter<CommT, GlobalCommT> {
>>>> >
>>>> >     void commit(GlobalCommT globalCommittables);
>>>> >
>>>> >     GlobalCommT combine(List<CommT> committables);
>>>> >
>>>> > }
>>>> >
>>>> > A different problem that I see here is how do we handle commit
>>>> failures.
>>>> > Should the committables (both normal and global be included in the
>>>> next
>>>> > cycle, shall we retry it, ...) I think it would be worth laying it out
>>>> > in the FLIP.
>>>> >
>>>> > @Aljoscha I think you can find the code Steven was referring in here:
>>>> >
>>>> https://github.com/Netflix-Skunkworks/nfflink-connector-iceberg/blob/master/nfflink-connector-iceberg/src/main/java/com/netflix/spaas/nfflink/connector/iceberg/sink/IcebergCommitter.java
>>>> >
>>>> > Best,
>>>> >
>>>> > Dawid
>>>> >
>>>> > On 14/09/2020 15:19, Aljoscha Krettek wrote:
>>>> >
>>>> > On 14.09.20 01:23, Steven Wu wrote:
>>>> >
>>>> > ## Writer interface
>>>> >
>>>> > For the Writer interface, should we add "*prepareSnapshot"* before the
>>>> > checkpoint barrier emitted downstream?  IcebergWriter would need it.
>>>> Or
>>>> > would the framework call "*flush*" before the barrier emitted
>>>> > downstream?
>>>> > that guarantee would achieve the same goal.
>>>> >
>>>> > I would think that we only need flush() and the semantics are that it
>>>> > prepares for a commit, so on a physical level it would be called from
>>>> > "prepareSnapshotPreBarrier". Now that I'm thinking about it more I
>>>> > think flush() should be renamed to something like "prepareCommit()".
>>>> >
>>>> > @Guowei, what do you think about this?
>>>> >
>>>> >
>>>> > In [1], we discussed the reason for Writer to emit (checkpointId,
>>>> CommT)
>>>> > tuple to the committer. The committer needs checkpointId to separate
>>>> out
>>>> > data files for different checkpoints if concurrent checkpoints are
>>>> > enabled.
>>>> >
>>>> > When can this happen? Even with concurrent checkpoints the snapshot
>>>> > barriers would still cleanly segregate the input stream of an operator
>>>> > into tranches that should manifest in only one checkpoint. With
>>>> > concurrent checkpoints, all that can happen is that we start a
>>>> > checkpoint before a last one is confirmed completed.
>>>> >
>>>> > Unless there is some weirdness in the sources and some sources start
>>>> > chk1 first and some other ones start chk2 first?
>>>> >
>>>> > @Piotrek, do you think this is a problem?
>>>> >
>>>> >
>>>> > For the Committer interface, I am wondering if we should split the
>>>> > single
>>>> > commit method into separate "*collect"* and "*commit"* methods? This
>>>> > way,
>>>> > it can handle both single and multiple CommT objects.
>>>> >
>>>> > I think we can't do this. If the sink only needs a regular Commiter,
>>>> > we can perform the commits in parallel, possibly on different
>>>> > machines. Only when the sink needs a GlobalCommitter would we need to
>>>> > ship all commits to a single process and perform the commit there. If
>>>> > both methods were unified in one interface we couldn't make the
>>>> > decision of were to commit in the framework code.
>>>> >
>>>> >
>>>> > For Iceberg, writers don't need any state. But the GlobalCommitter
>>>> > needs to
>>>> > checkpoint StateT. For the committer, CommT is "DataFile". Since a
>>>> > single
>>>> > committer can collect thousands (or more) data files in one checkpoint
>>>> > cycle, as an optimization we checkpoint a single "ManifestFile" (for
>>>> the
>>>> > collected thousands data files) as StateT. This allows us to absorb
>>>> > extended commit outages without losing written/uploaded data files, as
>>>> > operator state size is as small as one manifest file per checkpoint
>>>> > cycle
>>>> >
>>>> > You could have a point here. Is the code for this available in
>>>> > open-source? I was checking out
>>>> >
>>>> >
>>>> >
>>>> https://github.com/apache/iceberg/blob/master/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergFilesCommitter.java
>>>> >
>>>> > and didn't find the ManifestFile optimization there.
>>>> >
>>>> > Best,
>>>> > Aljoscha
>>>> >
>>>> >
>>>> >
>>>>
>>>
Reply | Threaded
Open this post in threaded view
|

Re: [DISCUSS] FLIP-143: Unified Sink API

Piotr Nowojski-4
Hey

Thanks Dawid for bringing up my suggestion :)

> I'm not so sure about this, the sinks I'm aware of would not be able to
> implement this method: Kafka doesn't have this, I didn't see it in the
> Iceberg interfaces, and HDFS/S3 also don't have it.

Aljoscha, as I wrote, FlinkKafkaProducer is actually one for which we could
do some magic. At the very least we could use
`FlinkKafkaProducer#pendingRecords` to make the sink unavailable when some
threshold is exceeded. Alternatively, maybe we could hook in to the
KafkaProducer's buffer state [1]:

> The buffer.memory controls the total amount of memory available to the
producer for buffering.
> If records are sent faster than they can be transmitted to the server
then this buffer space will be exhausted.
> When the buffer space is exhausted additional send calls will block.

As far as I can see, Kafka is exposing the `buffer-available-bytes` metric,
which we might use instead of `pendingRecords`. Heck, we are already
hacking KafkaProducer with reflections, we could access
`org.apache.kafka.clients.producer.KafkaProducer#accumulator` field to
call  `accumulator.bufferPoolAvailableMemory()` method, if metric would be
to expensive to check per every record.

Furthermore, I'm pretty sure other sinks (maybe not all) provide similar
features. If we are desperate, we could always contribute something to
those systems to make them expose the internal buffer's state.

If we are really desperate, we could provide a generic records handover
wrapper sink, that would have a buffer of N (5? 10? ) records and would be
handing over those records to the blocking sink running in another thread.
If the buffer is full, the sink would be unavailable.

Guowei
> Does the sink's snapshot return immediately when the sink's status is
unavailable?

State snapshot call is generally speaking non blocking already, so it
should not be an issue. If it's blocking and if it will be solving some
problem, we could later decide in the runtime code to not execute snapshot
calls if a sink is unavailable. Think about isAvailable more like a hint
from the operator to the runtime, which we can use to make better
decisions. Also take a look at the FLIP-27 sources (`SourceReader`), where
there already is `isAvailable()` method. It would be best if new sinks
would just duplicate the same contract.

> For me I want to know is what specific sink will benefit from this feature

It's not the sinks that would benefit from this, but other parts of the
system. Currently task thread is blocked on backpressured Sink, it's
blocking some things from happening (checkpointing, closing, ...). If we
make sinks non blocking (as is the network stack in the most part and as
are the FLIP-27 sources), we will be able to snapshot state of the operator
immediately. For example, change from blocking to non blocking sources was
speeding up unaligned checkpoints from ~30seconds down to ~5seconds in
our benchmarks, but the difference can be even more profound (hours instead
of seconds/minutes as reported by some users).

Piotrek

[1]
https://kafka.apache.org/10/javadoc/org/apache/kafka/clients/producer/KafkaProducer.html

śr., 16 wrz 2020 o 06:29 Guowei Ma <[hidden email]> napisał(a):

> Hi,all
>
> Thanks for all your valuable options and ideas.Currently there are many
> topics in the mail. I try to summarize what is consensus and what is not.
> Correct me if I am wrong.
>
> ## Consensus
>
> 1. The motivation of the unified sink API is to decouple the sink
> implementation from the different runtime execution mode.
> 2. The initial scope of the unified sink API only covers the file system
> type, which supports the real transactions. The FLIP focuses more on the
> semantics the new sink api should support.
> 3. We prefer the first alternative API, which could give the framework a
> greater opportunity to optimize.
> 4. The `Writer` needs to add a method `prepareCommit`, which would be
> called from `prepareSnapshotPreBarrier`. And remove the `Flush` method.
> 5. The FLIP could move the `Snapshot & Drain` section in order to be more
> focused.
>
> ## Not Consensus
>
> 1. What should the “Unified Sink API” support/cover? The API can
> “unified”(decoupe) the commit operation in the term of supporting exactly
> once semantics. However, even if we narrow down the initial supported
> system to the file system there would be different topology requirements.
> These requirements come from performance optimization
> (IceBergSink/MergeHiveSink) or functionality(i.e. whether a bucket is
> “finished”).  Should the unified sink API support these requirements?
> 2. The API does not expose the checkpoint-id because the batch execution
> mode does not have the normal checkpoint. But there still some
> implementations depend on this.(IceBergSink uses this to do some dedupe).
> I think how to support this requirement depends on the first open question.
> 3. Whether the `Writer` supports async functionality or not. Currently I do
> not know which sink could benefit from it. Maybe it is just my own problem.
>
> Best,
> Guowei
>
>
> On Wed, Sep 16, 2020 at 12:02 PM Guowei Ma <[hidden email]> wrote:
>
> >
> > Hi, Steven
> > Thanks you for your thoughtful ideas and concerns.
> >
> > >>I still like the concept of grouping data files per checkpoint for
> > streaming mode. it is cleaner and probably easier to manage and deal with
> > commit failures. Plus, it >>can reduce dupes for the at least once
> > >>mode.  I understand checkpoint is not an option for batch execution. We
> > don't have to expose the checkpointId in API, as >>long as  the internal
> > bookkeeping groups data files by checkpoints for streaming >>mode.
> >
> > I think this problem(How to dedupe the combined committed data) also
> > depends on where to place the agg/combine logic .
> >
> > 1. If the agg/combine takes place in the “commit” maybe we need to figure
> > out how to give the aggregated committable a unique and auto-increment id
> > in the committer.
> > 2. If the agg/combine takes place in a separate operator maybe sink
> > developer could maintain the id itself by using the state.
> >
> > I think this problem is also decided by what the topology pattern the
> sink
> > API should support. Actually there are already many other topology
> > requirements. :)
> >
> > Best,
> > Guowei
> >
> >
> > On Wed, Sep 16, 2020 at 7:46 AM Steven Wu <[hidden email]> wrote:
> >
> >> > AFAIK the committer would not see the file-1-2 when ck1 happens in the
> >> ExactlyOnce mode.
> >>
> >> @Guowei Ma <[hidden email]> I think you are right for exactly
> once
> >> checkpoint semantics. what about "at least once"? I guess we can argue
> that
> >> it is fine to commit file-1-2 for at least once mode.
> >>
> >> I still like the concept of grouping data files per checkpoint for
> >> streaming mode. it is cleaner and probably easier to manage and deal
> with
> >> commit failures. Plus, it can reduce dupes for the at least once mode.
> I
> >> understand checkpoint is not an option for batch execution. We don't
> have
> >> to expose the checkpointId in API, as long as  the internal bookkeeping
> >> groups data files by checkpoints for streaming mode.
> >>
> >>
> >> On Tue, Sep 15, 2020 at 6:58 AM Steven Wu <[hidden email]> wrote:
> >>
> >>> > images don't make it through to the mailing lists. You would need to
> >>> host the file somewhere and send a link.
> >>>
> >>> Sorry about that. Here is the sample DAG in google drawings.
> >>>
> >>>
> https://docs.google.com/drawings/d/1-P8F2jF9RG9HHTtAfWEBRuU_2uV9aDTdqEt5dLs2JPk/edit?usp=sharing
> >>>
> >>>
> >>> On Tue, Sep 15, 2020 at 4:58 AM Guowei Ma <[hidden email]>
> wrote:
> >>>
> >>>> Hi, Dawid
> >>>>
> >>>> >>I still find the merging case the most confusing. I don't
> necessarily
> >>>> understand why do you need the "SingleFileCommit" step in this
> scenario.
> >>>> The way I
> >>>> >> understand "commit" operation is that it makes some data/artifacts
> >>>> visible to the external system, thus it should be immutable from a
> >>>> point of
> >>>> view of a single >>process. Having an additional step in the same
> >>>> process
> >>>> that works on committed data contradicts with those assumptions. I
> >>>> might be
> >>>> missing something though. >> Could you elaborate >why can't it be
> >>>> something
> >>>> like FileWriter -> FileMergeWriter -> Committer (either global or
> >>>> non-global)? Again it might be just me not getting the example.
> >>>>
> >>>> I think you are right. The topology
> >>>> "FileWriter->FileMergeWriter->Committer" could meet the merge
> >>>> requirement.
> >>>> The topology "FileWriter-> SingleFileCommitter -> FileMergeWriter ->
> >>>> GlobalCommitter" reuses some code of the StreamingFileSink(For example
> >>>> rolling policy) so it has the "SingleFileCommitter" in the topology.
> In
> >>>> general I want to use the case to show that there are different
> >>>> topologies
> >>>> according to the requirements.
> >>>>
> >>>> BTW: IIRC, @Jingsong Lee <[hidden email]> telled me that
> >>>> the
> >>>> actual topology of merged supported HiveSink is more complicated than
> >>>> that.
> >>>>
> >>>>
> >>>> >> I've just briefly skimmed over the proposed interfaces. I would
> >>>> suggest
> >>>> one
> >>>> >> addition to the Writer interface (as I understand this is the
> runtime
> >>>> >> interface in this proposal?): add some availability method, to
> >>>> avoid, if
> >>>> >> possible, blocking calls on the sink. We already have similar
> >>>> >> availability methods in the new sources [1] and in various places
> in
> >>>> the
> >>>> >> network stack [2].
> >>>> >> BTW Let's not forget about Piotr's comment. I think we could add
> the
> >>>> isAvailable or similar method to the Writer interface in the FLIP.
> >>>>
> >>>> Thanks @Dawid Wysakowicz <[hidden email]>  for your reminder.
> >>>> There
> >>>> are two many issues at the same time.
> >>>>
> >>>> In addition to what Ajjoscha said : there is very little system
> support
> >>>> it.   Another thing I worry about is that: Does the sink's snapshot
> >>>> return
> >>>> immediately when the sink's status is unavailable? Maybe we could do
> it
> >>>> by
> >>>> dedupe some element in the state but I think it might be too
> >>>> complicated.
> >>>> For me I want to know is what specific sink will benefit from this
> >>>> feature.  @piotr <[hidden email]>  Please correct me if  I
> >>>> misunderstand you. thanks.
> >>>>
> >>>> Best,
> >>>> Guowei
> >>>>
> >>>>
> >>>> On Tue, Sep 15, 2020 at 3:55 PM Dawid Wysakowicz <
> >>>> [hidden email]>
> >>>> wrote:
> >>>>
> >>>> > What I understand is that HiveSink's implementation might need the
> >>>> local
> >>>> > committer(FileCommitter) because the file rename is needed.
> >>>> > But the iceberg only needs to write the manifest file.  Would you
> >>>> like to
> >>>> > enlighten me why the Iceberg needs the local committer?
> >>>> > Thanks
> >>>> >
> >>>> > Sorry if I caused a confusion here. I am not saying the Iceberg sink
> >>>> needs
> >>>> > a local committer. What I had in mind is that prior to the Iceberg
> >>>> example
> >>>> > I did not see a need for a "GlobalCommitter" in the streaming case.
> I
> >>>> > thought it is always enough to have the "normal" committer in that
> >>>> case.
> >>>> > Now I understand that this differentiation is not really about
> logical
> >>>> > separation. It is not really about the granularity with which we
> >>>> commit,
> >>>> > i.e. answering the "WHAT" question. It is really about the
> >>>> performance and
> >>>> > that in the end we will have a single "transaction", so it is about
> >>>> > answering the question "HOW".
> >>>> >
> >>>> >
> >>>> >    -
> >>>> >
> >>>> >    Commit a directory with merged files(Some user want to merge the
> >>>> files
> >>>> >    in a directory before committing the directory to Hive meta
> store)
> >>>> >
> >>>> >
> >>>> >    1.
> >>>> >
> >>>> >    FileWriter -> SingleFileCommit -> FileMergeWriter  ->
> >>>> GlobalCommitter
> >>>> >
> >>>> > I still find the merging case the most confusing. I don't
> necessarily
> >>>> > understand why do you need the "SingleFileCommit" step in this
> >>>> scenario.
> >>>> > The way I understand "commit" operation is that it makes some
> >>>> > data/artifacts visible to the external system, thus it should be
> >>>> immutable
> >>>> > from a point of view of a single process. Having an additional step
> >>>> in the
> >>>> > same process that works on committed data contradicts with those
> >>>> > assumptions. I might be missing something though. Could you
> elaborate
> >>>> why
> >>>> > can't it be something like FileWriter -> FileMergeWriter ->
> Committer
> >>>> > (either global or non-global)? Again it might be just me not getting
> >>>> the
> >>>> > example.
> >>>> >
> >>>> > I've just briefly skimmed over the proposed interfaces. I would
> >>>> suggest one
> >>>> > addition to the Writer interface (as I understand this is the
> runtime
> >>>> > interface in this proposal?): add some availability method, to
> avoid,
> >>>> if
> >>>> > possible, blocking calls on the sink. We already have similar
> >>>> > availability methods in the new sources [1] and in various places in
> >>>> the
> >>>> > network stack [2].
> >>>> >
> >>>> > BTW Let's not forget about Piotr's comment. I think we could add the
> >>>> > isAvailable or similar method to the Writer interface in the FLIP.
> >>>> >
> >>>> > Best,
> >>>> >
> >>>> > Dawid
> >>>> > On 15/09/2020 08:06, Guowei Ma wrote:
> >>>> >
> >>>> > I would think that we only need flush() and the semantics are that
> it
> >>>> > prepares for a commit, so on a physical level it would be called
> from
> >>>> > "prepareSnapshotPreBarrier". Now that I'm thinking about it more I
> >>>> > think flush() should be renamed to something like "prepareCommit()".
> >>>> >
> >>>> > Generally speaking it is a good point that emitting the committables
> >>>> > should happen before emitting the checkpoint barrier downstream.
> >>>> > However, if I remember offline discussions well, the idea behind
> >>>> > Writer#flush and Writer#snapshotState was to differentiate commit on
> >>>> > checkpoint vs final checkpoint at the end of the job. Both of these
> >>>> > methods could emit committables, but the flush should not leave any
> in
> >>>> > progress state (e.g. in case of file sink in STREAM mode, in
> >>>> > snapshotState it could leave some open files that would be committed
> >>>> in
> >>>> > a subsequent cycle, however flush should close all files). The
> >>>> > snapshotState as it is now can not be called in
> >>>> > prepareSnapshotPreBarrier as it can store some state, which should
> >>>> > happen in Operator#snapshotState as otherwise it would always be
> >>>> > synchronous. Therefore I think we would need sth like:
> >>>> >
> >>>> > void prepareCommit(boolean flush, WriterOutput<CommT> output);
> >>>> >
> >>>> > ver 1:
> >>>> >
> >>>> > List<StateT> snapshotState();
> >>>> >
> >>>> > ver 2:
> >>>> >
> >>>> > void snapshotState(); // not sure if we need that method at all in
> >>>> option
> >>>> >
> >>>> > 2
> >>>> >
> >>>> > I second Dawid's proposal. This is a valid scenario. And version2
> >>>> does not
> >>>> > need the snapshotState() any more.
> >>>> >
> >>>> >
> >>>> > The Committer is as described in the FLIP, it's basically a function
> >>>> > "void commit(Committable)". The GobalCommitter would be a function
> >>>> "void
> >>>> > commit(List<Committable>)". The former would be used by an S3 sink
> >>>> where
> >>>> > we can individually commit files to S3, a committable would be the
> >>>> list
> >>>> > of part uploads that will form the final file and the commit
> operation
> >>>> > creates the metadata in S3. The latter would be used by something
> like
> >>>> > Iceberg where the Committer needs a global view of all the commits
> to
> >>>> be
> >>>> > efficient and not overwhelm the system.
> >>>> >
> >>>> > I don't know yet if sinks would only implement on type of commit
> >>>> > function or potentially both at the same time, and maybe Commit can
> >>>> > return some CommitResult that gets shipped to the GlobalCommit
> >>>> function.
> >>>> > I must admit it I did not get the need for Local/Normal + Global
> >>>> > committer at first. The Iceberg example helped a lot. I think it
> >>>> makes a
> >>>> > lot of sense.
> >>>> >
> >>>> > @Dawid
> >>>> > What I understand is that HiveSink's implementation might need the
> >>>> local
> >>>> > committer(FileCommitter) because the file rename is needed.
> >>>> > But the iceberg only needs to write the manifest file.  Would you
> >>>> like to
> >>>> > enlighten me why the Iceberg needs the local committer?
> >>>> > Thanks
> >>>> >
> >>>> > Best,
> >>>> > Guowei
> >>>> >
> >>>> >
> >>>> > On Mon, Sep 14, 2020 at 11:19 PM Dawid Wysakowicz <
> >>>> [hidden email]> <[hidden email]>
> >>>> > wrote:
> >>>> >
> >>>> >
> >>>> > Hi all,
> >>>> >
> >>>> >
> >>>> > I would think that we only need flush() and the semantics are that
> it
> >>>> > prepares for a commit, so on a physical level it would be called
> from
> >>>> > "prepareSnapshotPreBarrier". Now that I'm thinking about it more I
> >>>> > think flush() should be renamed to something like "prepareCommit()".
> >>>> >
> >>>> > Generally speaking it is a good point that emitting the committables
> >>>> > should happen before emitting the checkpoint barrier downstream.
> >>>> > However, if I remember offline discussions well, the idea behind
> >>>> > Writer#flush and Writer#snapshotState was to differentiate commit on
> >>>> > checkpoint vs final checkpoint at the end of the job. Both of these
> >>>> > methods could emit committables, but the flush should not leave any
> in
> >>>> > progress state (e.g. in case of file sink in STREAM mode, in
> >>>> > snapshotState it could leave some open files that would be committed
> >>>> in
> >>>> > a subsequent cycle, however flush should close all files). The
> >>>> > snapshotState as it is now can not be called in
> >>>> > prepareSnapshotPreBarrier as it can store some state, which should
> >>>> > happen in Operator#snapshotState as otherwise it would always be
> >>>> > synchronous. Therefore I think we would need sth like:
> >>>> >
> >>>> > void prepareCommit(boolean flush, WriterOutput<CommT> output);
> >>>> >
> >>>> > ver 1:
> >>>> >
> >>>> > List<StateT> snapshotState();
> >>>> >
> >>>> > ver 2:
> >>>> >
> >>>> > void snapshotState(); // not sure if we need that method at all in
> >>>> option 2
> >>>> >
> >>>> >
> >>>> > The Committer is as described in the FLIP, it's basically a function
> >>>> > "void commit(Committable)". The GobalCommitter would be a function
> >>>> "void
> >>>> > commit(List<Committable>)". The former would be used by an S3 sink
> >>>> where
> >>>> > we can individually commit files to S3, a committable would be the
> >>>> list
> >>>> > of part uploads that will form the final file and the commit
> operation
> >>>> > creates the metadata in S3. The latter would be used by something
> like
> >>>> > Iceberg where the Committer needs a global view of all the commits
> to
> >>>> be
> >>>> > efficient and not overwhelm the system.
> >>>> >
> >>>> > I don't know yet if sinks would only implement on type of commit
> >>>> > function or potentially both at the same time, and maybe Commit can
> >>>> > return some CommitResult that gets shipped to the GlobalCommit
> >>>> function.
> >>>> >
> >>>> > I must admit it I did not get the need for Local/Normal + Global
> >>>> > committer at first. The Iceberg example helped a lot. I think it
> >>>> makes a
> >>>> > lot of sense.
> >>>> >
> >>>> >
> >>>> > For Iceberg, writers don't need any state. But the GlobalCommitter
> >>>> > needs to
> >>>> > checkpoint StateT. For the committer, CommT is "DataFile". Since a
> >>>> single
> >>>> > committer can collect thousands (or more) data files in one
> checkpoint
> >>>> > cycle, as an optimization we checkpoint a single "ManifestFile" (for
> >>>> the
> >>>> > collected thousands data files) as StateT. This allows us to absorb
> >>>> > extended commit outages without losing written/uploaded data files,
> as
> >>>> > operator state size is as small as one manifest file per checkpoint
> >>>> cycle
> >>>> > [2].
> >>>> > ------------------
> >>>> > StateT snapshotState(SnapshotContext context) throws Exception;
> >>>> >
> >>>> > That means we also need the restoreCommitter API in the Sink
> interface
> >>>> > ---------------
> >>>> > Committer<CommT, StateT> restoreCommitter(InitContext context,
> StateT
> >>>> > state);
> >>>> >
> >>>> > I think this might be a valid case. Not sure though if I would go
> >>>> with a
> >>>> > "state" there. Having a state in a committer would imply we need a
> >>>> > collect method as well. So far we needed a single method commit(...)
> >>>> and
> >>>> > the bookkeeping of the committables could be handled by the
> >>>> framework. I
> >>>> > think something like an optional combiner in the GlobalCommitter
> would
> >>>> > be enough. What do you think?
> >>>> >
> >>>> > GlobalCommitter<CommT, GlobalCommT> {
> >>>> >
> >>>> >     void commit(GlobalCommT globalCommittables);
> >>>> >
> >>>> >     GlobalCommT combine(List<CommT> committables);
> >>>> >
> >>>> > }
> >>>> >
> >>>> > A different problem that I see here is how do we handle commit
> >>>> failures.
> >>>> > Should the committables (both normal and global be included in the
> >>>> next
> >>>> > cycle, shall we retry it, ...) I think it would be worth laying it
> out
> >>>> > in the FLIP.
> >>>> >
> >>>> > @Aljoscha I think you can find the code Steven was referring in
> here:
> >>>> >
> >>>>
> https://github.com/Netflix-Skunkworks/nfflink-connector-iceberg/blob/master/nfflink-connector-iceberg/src/main/java/com/netflix/spaas/nfflink/connector/iceberg/sink/IcebergCommitter.java
> >>>> >
> >>>> > Best,
> >>>> >
> >>>> > Dawid
> >>>> >
> >>>> > On 14/09/2020 15:19, Aljoscha Krettek wrote:
> >>>> >
> >>>> > On 14.09.20 01:23, Steven Wu wrote:
> >>>> >
> >>>> > ## Writer interface
> >>>> >
> >>>> > For the Writer interface, should we add "*prepareSnapshot"* before
> the
> >>>> > checkpoint barrier emitted downstream?  IcebergWriter would need it.
> >>>> Or
> >>>> > would the framework call "*flush*" before the barrier emitted
> >>>> > downstream?
> >>>> > that guarantee would achieve the same goal.
> >>>> >
> >>>> > I would think that we only need flush() and the semantics are that
> it
> >>>> > prepares for a commit, so on a physical level it would be called
> from
> >>>> > "prepareSnapshotPreBarrier". Now that I'm thinking about it more I
> >>>> > think flush() should be renamed to something like "prepareCommit()".
> >>>> >
> >>>> > @Guowei, what do you think about this?
> >>>> >
> >>>> >
> >>>> > In [1], we discussed the reason for Writer to emit (checkpointId,
> >>>> CommT)
> >>>> > tuple to the committer. The committer needs checkpointId to separate
> >>>> out
> >>>> > data files for different checkpoints if concurrent checkpoints are
> >>>> > enabled.
> >>>> >
> >>>> > When can this happen? Even with concurrent checkpoints the snapshot
> >>>> > barriers would still cleanly segregate the input stream of an
> operator
> >>>> > into tranches that should manifest in only one checkpoint. With
> >>>> > concurrent checkpoints, all that can happen is that we start a
> >>>> > checkpoint before a last one is confirmed completed.
> >>>> >
> >>>> > Unless there is some weirdness in the sources and some sources start
> >>>> > chk1 first and some other ones start chk2 first?
> >>>> >
> >>>> > @Piotrek, do you think this is a problem?
> >>>> >
> >>>> >
> >>>> > For the Committer interface, I am wondering if we should split the
> >>>> > single
> >>>> > commit method into separate "*collect"* and "*commit"* methods? This
> >>>> > way,
> >>>> > it can handle both single and multiple CommT objects.
> >>>> >
> >>>> > I think we can't do this. If the sink only needs a regular Commiter,
> >>>> > we can perform the commits in parallel, possibly on different
> >>>> > machines. Only when the sink needs a GlobalCommitter would we need
> to
> >>>> > ship all commits to a single process and perform the commit there.
> If
> >>>> > both methods were unified in one interface we couldn't make the
> >>>> > decision of were to commit in the framework code.
> >>>> >
> >>>> >
> >>>> > For Iceberg, writers don't need any state. But the GlobalCommitter
> >>>> > needs to
> >>>> > checkpoint StateT. For the committer, CommT is "DataFile". Since a
> >>>> > single
> >>>> > committer can collect thousands (or more) data files in one
> checkpoint
> >>>> > cycle, as an optimization we checkpoint a single "ManifestFile" (for
> >>>> the
> >>>> > collected thousands data files) as StateT. This allows us to absorb
> >>>> > extended commit outages without losing written/uploaded data files,
> as
> >>>> > operator state size is as small as one manifest file per checkpoint
> >>>> > cycle
> >>>> >
> >>>> > You could have a point here. Is the code for this available in
> >>>> > open-source? I was checking out
> >>>> >
> >>>> >
> >>>> >
> >>>>
> https://github.com/apache/iceberg/blob/master/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergFilesCommitter.java
> >>>> >
> >>>> > and didn't find the ManifestFile optimization there.
> >>>> >
> >>>> > Best,
> >>>> > Aljoscha
> >>>> >
> >>>> >
> >>>> >
> >>>>
> >>>
>
Reply | Threaded
Open this post in threaded view
|

Re: [DISCUSS] FLIP-143: Unified Sink API

Steven Wu
Guowei, thanks a lot for the summary. Here are a couple more questions that
need more clarification for the GlobalCommitter case.

* framework provides some sort of unique id per GlobalCommT (e.g. nonce or
some sort of transaction id)
* commit failure handling. Should we roll over to the next cycle? if so, we
may need commit(List<GlobalCommT> )

On Wed, Sep 16, 2020 at 2:11 AM Piotr Nowojski <[hidden email]>
wrote:

> Hey
>
> Thanks Dawid for bringing up my suggestion :)
>
> > I'm not so sure about this, the sinks I'm aware of would not be able to
> > implement this method: Kafka doesn't have this, I didn't see it in the
> > Iceberg interfaces, and HDFS/S3 also don't have it.
>
> Aljoscha, as I wrote, FlinkKafkaProducer is actually one for which we
> could do some magic. At the very least we could use
> `FlinkKafkaProducer#pendingRecords` to make the sink unavailable when some
> threshold is exceeded. Alternatively, maybe we could hook in to the
> KafkaProducer's buffer state [1]:
>
> > The buffer.memory controls the total amount of memory available to the
> producer for buffering.
> > If records are sent faster than they can be transmitted to the server
> then this buffer space will be exhausted.
> > When the buffer space is exhausted additional send calls will block.
>
> As far as I can see, Kafka is exposing the `buffer-available-bytes`
> metric, which we might use instead of `pendingRecords`. Heck, we are
> already hacking KafkaProducer with reflections, we could access
> `org.apache.kafka.clients.producer.KafkaProducer#accumulator` field to
> call  `accumulator.bufferPoolAvailableMemory()` method, if metric would be
> to expensive to check per every record.
>
> Furthermore, I'm pretty sure other sinks (maybe not all) provide similar
> features. If we are desperate, we could always contribute something to
> those systems to make them expose the internal buffer's state.
>
> If we are really desperate, we could provide a generic records handover
> wrapper sink, that would have a buffer of N (5? 10? ) records and would be
> handing over those records to the blocking sink running in another thread.
> If the buffer is full, the sink would be unavailable.
>
> Guowei
> > Does the sink's snapshot return immediately when the sink's status is
> unavailable?
>
> State snapshot call is generally speaking non blocking already, so it
> should not be an issue. If it's blocking and if it will be solving some
> problem, we could later decide in the runtime code to not execute snapshot
> calls if a sink is unavailable. Think about isAvailable more like a hint
> from the operator to the runtime, which we can use to make better
> decisions. Also take a look at the FLIP-27 sources (`SourceReader`), where
> there already is `isAvailable()` method. It would be best if new sinks
> would just duplicate the same contract.
>
> > For me I want to know is what specific sink will benefit from this
> feature
>
> It's not the sinks that would benefit from this, but other parts of the
> system. Currently task thread is blocked on backpressured Sink, it's
> blocking some things from happening (checkpointing, closing, ...). If we
> make sinks non blocking (as is the network stack in the most part and as
> are the FLIP-27 sources), we will be able to snapshot state of the operator
> immediately. For example, change from blocking to non blocking sources was
> speeding up unaligned checkpoints from ~30seconds down to ~5seconds in
> our benchmarks, but the difference can be even more profound (hours instead
> of seconds/minutes as reported by some users).
>
> Piotrek
>
> [1]
> https://kafka.apache.org/10/javadoc/org/apache/kafka/clients/producer/KafkaProducer.html
>
> śr., 16 wrz 2020 o 06:29 Guowei Ma <[hidden email]> napisał(a):
>
>> Hi,all
>>
>> Thanks for all your valuable options and ideas.Currently there are many
>> topics in the mail. I try to summarize what is consensus and what is not.
>> Correct me if I am wrong.
>>
>> ## Consensus
>>
>> 1. The motivation of the unified sink API is to decouple the sink
>> implementation from the different runtime execution mode.
>> 2. The initial scope of the unified sink API only covers the file system
>> type, which supports the real transactions. The FLIP focuses more on the
>> semantics the new sink api should support.
>> 3. We prefer the first alternative API, which could give the framework a
>> greater opportunity to optimize.
>> 4. The `Writer` needs to add a method `prepareCommit`, which would be
>> called from `prepareSnapshotPreBarrier`. And remove the `Flush` method.
>> 5. The FLIP could move the `Snapshot & Drain` section in order to be more
>> focused.
>>
>> ## Not Consensus
>>
>> 1. What should the “Unified Sink API” support/cover? The API can
>> “unified”(decoupe) the commit operation in the term of supporting exactly
>> once semantics. However, even if we narrow down the initial supported
>> system to the file system there would be different topology requirements.
>> These requirements come from performance optimization
>> (IceBergSink/MergeHiveSink) or functionality(i.e. whether a bucket is
>> “finished”).  Should the unified sink API support these requirements?
>> 2. The API does not expose the checkpoint-id because the batch execution
>> mode does not have the normal checkpoint. But there still some
>> implementations depend on this.(IceBergSink uses this to do some dedupe).
>> I think how to support this requirement depends on the first open
>> question.
>> 3. Whether the `Writer` supports async functionality or not. Currently I
>> do
>> not know which sink could benefit from it. Maybe it is just my own
>> problem.
>>
>> Best,
>> Guowei
>>
>>
>> On Wed, Sep 16, 2020 at 12:02 PM Guowei Ma <[hidden email]> wrote:
>>
>> >
>> > Hi, Steven
>> > Thanks you for your thoughtful ideas and concerns.
>> >
>> > >>I still like the concept of grouping data files per checkpoint for
>> > streaming mode. it is cleaner and probably easier to manage and deal
>> with
>> > commit failures. Plus, it >>can reduce dupes for the at least once
>> > >>mode.  I understand checkpoint is not an option for batch execution.
>> We
>> > don't have to expose the checkpointId in API, as >>long as  the internal
>> > bookkeeping groups data files by checkpoints for streaming >>mode.
>> >
>> > I think this problem(How to dedupe the combined committed data) also
>> > depends on where to place the agg/combine logic .
>> >
>> > 1. If the agg/combine takes place in the “commit” maybe we need to
>> figure
>> > out how to give the aggregated committable a unique and auto-increment
>> id
>> > in the committer.
>> > 2. If the agg/combine takes place in a separate operator maybe sink
>> > developer could maintain the id itself by using the state.
>> >
>> > I think this problem is also decided by what the topology pattern the
>> sink
>> > API should support. Actually there are already many other topology
>> > requirements. :)
>> >
>> > Best,
>> > Guowei
>> >
>> >
>> > On Wed, Sep 16, 2020 at 7:46 AM Steven Wu <[hidden email]> wrote:
>> >
>> >> > AFAIK the committer would not see the file-1-2 when ck1 happens in
>> the
>> >> ExactlyOnce mode.
>> >>
>> >> @Guowei Ma <[hidden email]> I think you are right for exactly
>> once
>> >> checkpoint semantics. what about "at least once"? I guess we can argue
>> that
>> >> it is fine to commit file-1-2 for at least once mode.
>> >>
>> >> I still like the concept of grouping data files per checkpoint for
>> >> streaming mode. it is cleaner and probably easier to manage and deal
>> with
>> >> commit failures. Plus, it can reduce dupes for the at least once
>> mode.  I
>> >> understand checkpoint is not an option for batch execution. We don't
>> have
>> >> to expose the checkpointId in API, as long as  the internal bookkeeping
>> >> groups data files by checkpoints for streaming mode.
>> >>
>> >>
>> >> On Tue, Sep 15, 2020 at 6:58 AM Steven Wu <[hidden email]>
>> wrote:
>> >>
>> >>> > images don't make it through to the mailing lists. You would need to
>> >>> host the file somewhere and send a link.
>> >>>
>> >>> Sorry about that. Here is the sample DAG in google drawings.
>> >>>
>> >>>
>> https://docs.google.com/drawings/d/1-P8F2jF9RG9HHTtAfWEBRuU_2uV9aDTdqEt5dLs2JPk/edit?usp=sharing
>> >>>
>> >>>
>> >>> On Tue, Sep 15, 2020 at 4:58 AM Guowei Ma <[hidden email]>
>> wrote:
>> >>>
>> >>>> Hi, Dawid
>> >>>>
>> >>>> >>I still find the merging case the most confusing. I don't
>> necessarily
>> >>>> understand why do you need the "SingleFileCommit" step in this
>> scenario.
>> >>>> The way I
>> >>>> >> understand "commit" operation is that it makes some data/artifacts
>> >>>> visible to the external system, thus it should be immutable from a
>> >>>> point of
>> >>>> view of a single >>process. Having an additional step in the same
>> >>>> process
>> >>>> that works on committed data contradicts with those assumptions. I
>> >>>> might be
>> >>>> missing something though. >> Could you elaborate >why can't it be
>> >>>> something
>> >>>> like FileWriter -> FileMergeWriter -> Committer (either global or
>> >>>> non-global)? Again it might be just me not getting the example.
>> >>>>
>> >>>> I think you are right. The topology
>> >>>> "FileWriter->FileMergeWriter->Committer" could meet the merge
>> >>>> requirement.
>> >>>> The topology "FileWriter-> SingleFileCommitter -> FileMergeWriter ->
>> >>>> GlobalCommitter" reuses some code of the StreamingFileSink(For
>> example
>> >>>> rolling policy) so it has the "SingleFileCommitter" in the topology.
>> In
>> >>>> general I want to use the case to show that there are different
>> >>>> topologies
>> >>>> according to the requirements.
>> >>>>
>> >>>> BTW: IIRC, @Jingsong Lee <[hidden email]> telled me that
>> >>>> the
>> >>>> actual topology of merged supported HiveSink is more complicated than
>> >>>> that.
>> >>>>
>> >>>>
>> >>>> >> I've just briefly skimmed over the proposed interfaces. I would
>> >>>> suggest
>> >>>> one
>> >>>> >> addition to the Writer interface (as I understand this is the
>> runtime
>> >>>> >> interface in this proposal?): add some availability method, to
>> >>>> avoid, if
>> >>>> >> possible, blocking calls on the sink. We already have similar
>> >>>> >> availability methods in the new sources [1] and in various places
>> in
>> >>>> the
>> >>>> >> network stack [2].
>> >>>> >> BTW Let's not forget about Piotr's comment. I think we could add
>> the
>> >>>> isAvailable or similar method to the Writer interface in the FLIP.
>> >>>>
>> >>>> Thanks @Dawid Wysakowicz <[hidden email]>  for your
>> reminder.
>> >>>> There
>> >>>> are two many issues at the same time.
>> >>>>
>> >>>> In addition to what Ajjoscha said : there is very little system
>> support
>> >>>> it.   Another thing I worry about is that: Does the sink's snapshot
>> >>>> return
>> >>>> immediately when the sink's status is unavailable? Maybe we could do
>> it
>> >>>> by
>> >>>> dedupe some element in the state but I think it might be too
>> >>>> complicated.
>> >>>> For me I want to know is what specific sink will benefit from this
>> >>>> feature.  @piotr <[hidden email]>  Please correct me if  I
>> >>>> misunderstand you. thanks.
>> >>>>
>> >>>> Best,
>> >>>> Guowei
>> >>>>
>> >>>>
>> >>>> On Tue, Sep 15, 2020 at 3:55 PM Dawid Wysakowicz <
>> >>>> [hidden email]>
>> >>>> wrote:
>> >>>>
>> >>>> > What I understand is that HiveSink's implementation might need the
>> >>>> local
>> >>>> > committer(FileCommitter) because the file rename is needed.
>> >>>> > But the iceberg only needs to write the manifest file.  Would you
>> >>>> like to
>> >>>> > enlighten me why the Iceberg needs the local committer?
>> >>>> > Thanks
>> >>>> >
>> >>>> > Sorry if I caused a confusion here. I am not saying the Iceberg
>> sink
>> >>>> needs
>> >>>> > a local committer. What I had in mind is that prior to the Iceberg
>> >>>> example
>> >>>> > I did not see a need for a "GlobalCommitter" in the streaming
>> case. I
>> >>>> > thought it is always enough to have the "normal" committer in that
>> >>>> case.
>> >>>> > Now I understand that this differentiation is not really about
>> logical
>> >>>> > separation. It is not really about the granularity with which we
>> >>>> commit,
>> >>>> > i.e. answering the "WHAT" question. It is really about the
>> >>>> performance and
>> >>>> > that in the end we will have a single "transaction", so it is about
>> >>>> > answering the question "HOW".
>> >>>> >
>> >>>> >
>> >>>> >    -
>> >>>> >
>> >>>> >    Commit a directory with merged files(Some user want to merge the
>> >>>> files
>> >>>> >    in a directory before committing the directory to Hive meta
>> store)
>> >>>> >
>> >>>> >
>> >>>> >    1.
>> >>>> >
>> >>>> >    FileWriter -> SingleFileCommit -> FileMergeWriter  ->
>> >>>> GlobalCommitter
>> >>>> >
>> >>>> > I still find the merging case the most confusing. I don't
>> necessarily
>> >>>> > understand why do you need the "SingleFileCommit" step in this
>> >>>> scenario.
>> >>>> > The way I understand "commit" operation is that it makes some
>> >>>> > data/artifacts visible to the external system, thus it should be
>> >>>> immutable
>> >>>> > from a point of view of a single process. Having an additional step
>> >>>> in the
>> >>>> > same process that works on committed data contradicts with those
>> >>>> > assumptions. I might be missing something though. Could you
>> elaborate
>> >>>> why
>> >>>> > can't it be something like FileWriter -> FileMergeWriter ->
>> Committer
>> >>>> > (either global or non-global)? Again it might be just me not
>> getting
>> >>>> the
>> >>>> > example.
>> >>>> >
>> >>>> > I've just briefly skimmed over the proposed interfaces. I would
>> >>>> suggest one
>> >>>> > addition to the Writer interface (as I understand this is the
>> runtime
>> >>>> > interface in this proposal?): add some availability method, to
>> avoid,
>> >>>> if
>> >>>> > possible, blocking calls on the sink. We already have similar
>> >>>> > availability methods in the new sources [1] and in various places
>> in
>> >>>> the
>> >>>> > network stack [2].
>> >>>> >
>> >>>> > BTW Let's not forget about Piotr's comment. I think we could add
>> the
>> >>>> > isAvailable or similar method to the Writer interface in the FLIP.
>> >>>> >
>> >>>> > Best,
>> >>>> >
>> >>>> > Dawid
>> >>>> > On 15/09/2020 08:06, Guowei Ma wrote:
>> >>>> >
>> >>>> > I would think that we only need flush() and the semantics are that
>> it
>> >>>> > prepares for a commit, so on a physical level it would be called
>> from
>> >>>> > "prepareSnapshotPreBarrier". Now that I'm thinking about it more I
>> >>>> > think flush() should be renamed to something like
>> "prepareCommit()".
>> >>>> >
>> >>>> > Generally speaking it is a good point that emitting the
>> committables
>> >>>> > should happen before emitting the checkpoint barrier downstream.
>> >>>> > However, if I remember offline discussions well, the idea behind
>> >>>> > Writer#flush and Writer#snapshotState was to differentiate commit
>> on
>> >>>> > checkpoint vs final checkpoint at the end of the job. Both of these
>> >>>> > methods could emit committables, but the flush should not leave
>> any in
>> >>>> > progress state (e.g. in case of file sink in STREAM mode, in
>> >>>> > snapshotState it could leave some open files that would be
>> committed
>> >>>> in
>> >>>> > a subsequent cycle, however flush should close all files). The
>> >>>> > snapshotState as it is now can not be called in
>> >>>> > prepareSnapshotPreBarrier as it can store some state, which should
>> >>>> > happen in Operator#snapshotState as otherwise it would always be
>> >>>> > synchronous. Therefore I think we would need sth like:
>> >>>> >
>> >>>> > void prepareCommit(boolean flush, WriterOutput<CommT> output);
>> >>>> >
>> >>>> > ver 1:
>> >>>> >
>> >>>> > List<StateT> snapshotState();
>> >>>> >
>> >>>> > ver 2:
>> >>>> >
>> >>>> > void snapshotState(); // not sure if we need that method at all in
>> >>>> option
>> >>>> >
>> >>>> > 2
>> >>>> >
>> >>>> > I second Dawid's proposal. This is a valid scenario. And version2
>> >>>> does not
>> >>>> > need the snapshotState() any more.
>> >>>> >
>> >>>> >
>> >>>> > The Committer is as described in the FLIP, it's basically a
>> function
>> >>>> > "void commit(Committable)". The GobalCommitter would be a function
>> >>>> "void
>> >>>> > commit(List<Committable>)". The former would be used by an S3 sink
>> >>>> where
>> >>>> > we can individually commit files to S3, a committable would be the
>> >>>> list
>> >>>> > of part uploads that will form the final file and the commit
>> operation
>> >>>> > creates the metadata in S3. The latter would be used by something
>> like
>> >>>> > Iceberg where the Committer needs a global view of all the commits
>> to
>> >>>> be
>> >>>> > efficient and not overwhelm the system.
>> >>>> >
>> >>>> > I don't know yet if sinks would only implement on type of commit
>> >>>> > function or potentially both at the same time, and maybe Commit can
>> >>>> > return some CommitResult that gets shipped to the GlobalCommit
>> >>>> function.
>> >>>> > I must admit it I did not get the need for Local/Normal + Global
>> >>>> > committer at first. The Iceberg example helped a lot. I think it
>> >>>> makes a
>> >>>> > lot of sense.
>> >>>> >
>> >>>> > @Dawid
>> >>>> > What I understand is that HiveSink's implementation might need the
>> >>>> local
>> >>>> > committer(FileCommitter) because the file rename is needed.
>> >>>> > But the iceberg only needs to write the manifest file.  Would you
>> >>>> like to
>> >>>> > enlighten me why the Iceberg needs the local committer?
>> >>>> > Thanks
>> >>>> >
>> >>>> > Best,
>> >>>> > Guowei
>> >>>> >
>> >>>> >
>> >>>> > On Mon, Sep 14, 2020 at 11:19 PM Dawid Wysakowicz <
>> >>>> [hidden email]> <[hidden email]>
>> >>>> > wrote:
>> >>>> >
>> >>>> >
>> >>>> > Hi all,
>> >>>> >
>> >>>> >
>> >>>> > I would think that we only need flush() and the semantics are that
>> it
>> >>>> > prepares for a commit, so on a physical level it would be called
>> from
>> >>>> > "prepareSnapshotPreBarrier". Now that I'm thinking about it more I
>> >>>> > think flush() should be renamed to something like
>> "prepareCommit()".
>> >>>> >
>> >>>> > Generally speaking it is a good point that emitting the
>> committables
>> >>>> > should happen before emitting the checkpoint barrier downstream.
>> >>>> > However, if I remember offline discussions well, the idea behind
>> >>>> > Writer#flush and Writer#snapshotState was to differentiate commit
>> on
>> >>>> > checkpoint vs final checkpoint at the end of the job. Both of these
>> >>>> > methods could emit committables, but the flush should not leave
>> any in
>> >>>> > progress state (e.g. in case of file sink in STREAM mode, in
>> >>>> > snapshotState it could leave some open files that would be
>> committed
>> >>>> in
>> >>>> > a subsequent cycle, however flush should close all files). The
>> >>>> > snapshotState as it is now can not be called in
>> >>>> > prepareSnapshotPreBarrier as it can store some state, which should
>> >>>> > happen in Operator#snapshotState as otherwise it would always be
>> >>>> > synchronous. Therefore I think we would need sth like:
>> >>>> >
>> >>>> > void prepareCommit(boolean flush, WriterOutput<CommT> output);
>> >>>> >
>> >>>> > ver 1:
>> >>>> >
>> >>>> > List<StateT> snapshotState();
>> >>>> >
>> >>>> > ver 2:
>> >>>> >
>> >>>> > void snapshotState(); // not sure if we need that method at all in
>> >>>> option 2
>> >>>> >
>> >>>> >
>> >>>> > The Committer is as described in the FLIP, it's basically a
>> function
>> >>>> > "void commit(Committable)". The GobalCommitter would be a function
>> >>>> "void
>> >>>> > commit(List<Committable>)". The former would be used by an S3 sink
>> >>>> where
>> >>>> > we can individually commit files to S3, a committable would be the
>> >>>> list
>> >>>> > of part uploads that will form the final file and the commit
>> operation
>> >>>> > creates the metadata in S3. The latter would be used by something
>> like
>> >>>> > Iceberg where the Committer needs a global view of all the commits
>> to
>> >>>> be
>> >>>> > efficient and not overwhelm the system.
>> >>>> >
>> >>>> > I don't know yet if sinks would only implement on type of commit
>> >>>> > function or potentially both at the same time, and maybe Commit can
>> >>>> > return some CommitResult that gets shipped to the GlobalCommit
>> >>>> function.
>> >>>> >
>> >>>> > I must admit it I did not get the need for Local/Normal + Global
>> >>>> > committer at first. The Iceberg example helped a lot. I think it
>> >>>> makes a
>> >>>> > lot of sense.
>> >>>> >
>> >>>> >
>> >>>> > For Iceberg, writers don't need any state. But the GlobalCommitter
>> >>>> > needs to
>> >>>> > checkpoint StateT. For the committer, CommT is "DataFile". Since a
>> >>>> single
>> >>>> > committer can collect thousands (or more) data files in one
>> checkpoint
>> >>>> > cycle, as an optimization we checkpoint a single "ManifestFile"
>> (for
>> >>>> the
>> >>>> > collected thousands data files) as StateT. This allows us to absorb
>> >>>> > extended commit outages without losing written/uploaded data
>> files, as
>> >>>> > operator state size is as small as one manifest file per checkpoint
>> >>>> cycle
>> >>>> > [2].
>> >>>> > ------------------
>> >>>> > StateT snapshotState(SnapshotContext context) throws Exception;
>> >>>> >
>> >>>> > That means we also need the restoreCommitter API in the Sink
>> interface
>> >>>> > ---------------
>> >>>> > Committer<CommT, StateT> restoreCommitter(InitContext context,
>> StateT
>> >>>> > state);
>> >>>> >
>> >>>> > I think this might be a valid case. Not sure though if I would go
>> >>>> with a
>> >>>> > "state" there. Having a state in a committer would imply we need a
>> >>>> > collect method as well. So far we needed a single method
>> commit(...)
>> >>>> and
>> >>>> > the bookkeeping of the committables could be handled by the
>> >>>> framework. I
>> >>>> > think something like an optional combiner in the GlobalCommitter
>> would
>> >>>> > be enough. What do you think?
>> >>>> >
>> >>>> > GlobalCommitter<CommT, GlobalCommT> {
>> >>>> >
>> >>>> >     void commit(GlobalCommT globalCommittables);
>> >>>> >
>> >>>> >     GlobalCommT combine(List<CommT> committables);
>> >>>> >
>> >>>> > }
>> >>>> >
>> >>>> > A different problem that I see here is how do we handle commit
>> >>>> failures.
>> >>>> > Should the committables (both normal and global be included in the
>> >>>> next
>> >>>> > cycle, shall we retry it, ...) I think it would be worth laying it
>> out
>> >>>> > in the FLIP.
>> >>>> >
>> >>>> > @Aljoscha I think you can find the code Steven was referring in
>> here:
>> >>>> >
>> >>>>
>> https://github.com/Netflix-Skunkworks/nfflink-connector-iceberg/blob/master/nfflink-connector-iceberg/src/main/java/com/netflix/spaas/nfflink/connector/iceberg/sink/IcebergCommitter.java
>> >>>> >
>> >>>> > Best,
>> >>>> >
>> >>>> > Dawid
>> >>>> >
>> >>>> > On 14/09/2020 15:19, Aljoscha Krettek wrote:
>> >>>> >
>> >>>> > On 14.09.20 01:23, Steven Wu wrote:
>> >>>> >
>> >>>> > ## Writer interface
>> >>>> >
>> >>>> > For the Writer interface, should we add "*prepareSnapshot"* before
>> the
>> >>>> > checkpoint barrier emitted downstream?  IcebergWriter would need
>> it.
>> >>>> Or
>> >>>> > would the framework call "*flush*" before the barrier emitted
>> >>>> > downstream?
>> >>>> > that guarantee would achieve the same goal.
>> >>>> >
>> >>>> > I would think that we only need flush() and the semantics are that
>> it
>> >>>> > prepares for a commit, so on a physical level it would be called
>> from
>> >>>> > "prepareSnapshotPreBarrier". Now that I'm thinking about it more I
>> >>>> > think flush() should be renamed to something like
>> "prepareCommit()".
>> >>>> >
>> >>>> > @Guowei, what do you think about this?
>> >>>> >
>> >>>> >
>> >>>> > In [1], we discussed the reason for Writer to emit (checkpointId,
>> >>>> CommT)
>> >>>> > tuple to the committer. The committer needs checkpointId to
>> separate
>> >>>> out
>> >>>> > data files for different checkpoints if concurrent checkpoints are
>> >>>> > enabled.
>> >>>> >
>> >>>> > When can this happen? Even with concurrent checkpoints the snapshot
>> >>>> > barriers would still cleanly segregate the input stream of an
>> operator
>> >>>> > into tranches that should manifest in only one checkpoint. With
>> >>>> > concurrent checkpoints, all that can happen is that we start a
>> >>>> > checkpoint before a last one is confirmed completed.
>> >>>> >
>> >>>> > Unless there is some weirdness in the sources and some sources
>> start
>> >>>> > chk1 first and some other ones start chk2 first?
>> >>>> >
>> >>>> > @Piotrek, do you think this is a problem?
>> >>>> >
>> >>>> >
>> >>>> > For the Committer interface, I am wondering if we should split the
>> >>>> > single
>> >>>> > commit method into separate "*collect"* and "*commit"* methods?
>> This
>> >>>> > way,
>> >>>> > it can handle both single and multiple CommT objects.
>> >>>> >
>> >>>> > I think we can't do this. If the sink only needs a regular
>> Commiter,
>> >>>> > we can perform the commits in parallel, possibly on different
>> >>>> > machines. Only when the sink needs a GlobalCommitter would we need
>> to
>> >>>> > ship all commits to a single process and perform the commit there.
>> If
>> >>>> > both methods were unified in one interface we couldn't make the
>> >>>> > decision of were to commit in the framework code.
>> >>>> >
>> >>>> >
>> >>>> > For Iceberg, writers don't need any state. But the GlobalCommitter
>> >>>> > needs to
>> >>>> > checkpoint StateT. For the committer, CommT is "DataFile". Since a
>> >>>> > single
>> >>>> > committer can collect thousands (or more) data files in one
>> checkpoint
>> >>>> > cycle, as an optimization we checkpoint a single "ManifestFile"
>> (for
>> >>>> the
>> >>>> > collected thousands data files) as StateT. This allows us to absorb
>> >>>> > extended commit outages without losing written/uploaded data
>> files, as
>> >>>> > operator state size is as small as one manifest file per checkpoint
>> >>>> > cycle
>> >>>> >
>> >>>> > You could have a point here. Is the code for this available in
>> >>>> > open-source? I was checking out
>> >>>> >
>> >>>> >
>> >>>> >
>> >>>>
>> https://github.com/apache/iceberg/blob/master/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergFilesCommitter.java
>> >>>> >
>> >>>> > and didn't find the ManifestFile optimization there.
>> >>>> >
>> >>>> > Best,
>> >>>> > Aljoscha
>> >>>> >
>> >>>> >
>> >>>> >
>> >>>>
>> >>>
>>
>
Reply | Threaded
Open this post in threaded view
|

Re: [DISCUSS] FLIP-143: Unified Sink API

Guowei Ma
In reply to this post by Piotr Nowojski-4
Thank @piotr <[hidden email]>  very much for your patient explanation.
I would try to explain what is in my mind.

Considering following case:
FlinkSink E6 -----> Client Buffer Queue |E5|E4|E3|E2| ------> External
System E1

When the FlinkSink can not add the E6 to the Queue maybe the external
client is flushing/writing some element (e.g. E1) but the action is blocked
for some reason.

At that time even if Flink knows the queue is full(`Not Availble`) and
Flink could snapshot the `E5&E4&E3&E2` to the sink’s state it has to handle
the E1 which is handling by the client when the sink snapshot it’s state.
There might be two options in my mind:

1. Flush. The sink has to wait E1 to finish then the sink could finish
doing the snapshot.
2. Dedupe. The sink needs to snapshot the E1 to the state and dedupe the E1
when restoring.

For the option1:  If `isAvailable` is not added, we will wait for one more
record(E6) to complete the snapshot.
For the option2: If `isAvailable` is not added, we will wait for one more
record to complete the snapshot.

In both options, we reduce the processing time of the snapshot by 1
element’s processing time if we add the `isAvailable` interface. If it is
the case I am not sure whether it worth adding this asynchronous interface.
Of course there may be some other benefits I don’t have.  In addition, I
must admit that this case is just an assumption, maybe I am wrong.

For me what is the difference between the source and sink is that we could
not know when the next element would come. We could not always block to
wait the next coming element, which would block the checkpoint even if
there is no back pressure.

Thanks again for your patient and concerns.

Best,
Guowei


On Wed, Sep 16, 2020 at 5:11 PM Piotr Nowojski <[hidden email]>
wrote:

> Hey
>
> Thanks Dawid for bringing up my suggestion :)
>
> > I'm not so sure about this, the sinks I'm aware of would not be able to
> > implement this method: Kafka doesn't have this, I didn't see it in the
> > Iceberg interfaces, and HDFS/S3 also don't have it.
>
> Aljoscha, as I wrote, FlinkKafkaProducer is actually one for which we could
> do some magic. At the very least we could use
> `FlinkKafkaProducer#pendingRecords` to make the sink unavailable when some
> threshold is exceeded. Alternatively, maybe we could hook in to the
> KafkaProducer's buffer state [1]:
>
> > The buffer.memory controls the total amount of memory available to the
> producer for buffering.
> > If records are sent faster than they can be transmitted to the server
> then this buffer space will be exhausted.
> > When the buffer space is exhausted additional send calls will block.
>
> As far as I can see, Kafka is exposing the `buffer-available-bytes` metric,
> which we might use instead of `pendingRecords`. Heck, we are already
> hacking KafkaProducer with reflections, we could access
> `org.apache.kafka.clients.producer.KafkaProducer#accumulator` field to
> call  `accumulator.bufferPoolAvailableMemory()` method, if metric would be
> to expensive to check per every record.
>
> Furthermore, I'm pretty sure other sinks (maybe not all) provide similar
> features. If we are desperate, we could always contribute something to
> those systems to make them expose the internal buffer's state.
>
> If we are really desperate, we could provide a generic records handover
> wrapper sink, that would have a buffer of N (5? 10? ) records and would be
> handing over those records to the blocking sink running in another thread.
> If the buffer is full, the sink would be unavailable.
>
> Guowei
> > Does the sink's snapshot return immediately when the sink's status is
> unavailable?
>
> State snapshot call is generally speaking non blocking already, so it
> should not be an issue. If it's blocking and if it will be solving some
> problem, we could later decide in the runtime code to not execute snapshot
> calls if a sink is unavailable. Think about isAvailable more like a hint
> from the operator to the runtime, which we can use to make better
> decisions. Also take a look at the FLIP-27 sources (`SourceReader`), where
> there already is `isAvailable()` method. It would be best if new sinks
> would just duplicate the same contract.
>
> > For me I want to know is what specific sink will benefit from this
> feature
>
> It's not the sinks that would benefit from this, but other parts of the
> system. Currently task thread is blocked on backpressured Sink, it's
> blocking some things from happening (checkpointing, closing, ...). If we
> make sinks non blocking (as is the network stack in the most part and as
> are the FLIP-27 sources), we will be able to snapshot state of the operator
> immediately. For example, change from blocking to non blocking sources was
> speeding up unaligned checkpoints from ~30seconds down to ~5seconds in
> our benchmarks, but the difference can be even more profound (hours instead
> of seconds/minutes as reported by some users).
>
> Piotrek
>
> [1]
>
> https://kafka.apache.org/10/javadoc/org/apache/kafka/clients/producer/KafkaProducer.html
>
> śr., 16 wrz 2020 o 06:29 Guowei Ma <[hidden email]> napisał(a):
>
> > Hi,all
> >
> > Thanks for all your valuable options and ideas.Currently there are many
> > topics in the mail. I try to summarize what is consensus and what is not.
> > Correct me if I am wrong.
> >
> > ## Consensus
> >
> > 1. The motivation of the unified sink API is to decouple the sink
> > implementation from the different runtime execution mode.
> > 2. The initial scope of the unified sink API only covers the file system
> > type, which supports the real transactions. The FLIP focuses more on the
> > semantics the new sink api should support.
> > 3. We prefer the first alternative API, which could give the framework a
> > greater opportunity to optimize.
> > 4. The `Writer` needs to add a method `prepareCommit`, which would be
> > called from `prepareSnapshotPreBarrier`. And remove the `Flush` method.
> > 5. The FLIP could move the `Snapshot & Drain` section in order to be more
> > focused.
> >
> > ## Not Consensus
> >
> > 1. What should the “Unified Sink API” support/cover? The API can
> > “unified”(decoupe) the commit operation in the term of supporting exactly
> > once semantics. However, even if we narrow down the initial supported
> > system to the file system there would be different topology requirements.
> > These requirements come from performance optimization
> > (IceBergSink/MergeHiveSink) or functionality(i.e. whether a bucket is
> > “finished”).  Should the unified sink API support these requirements?
> > 2. The API does not expose the checkpoint-id because the batch execution
> > mode does not have the normal checkpoint. But there still some
> > implementations depend on this.(IceBergSink uses this to do some dedupe).
> > I think how to support this requirement depends on the first open
> question.
> > 3. Whether the `Writer` supports async functionality or not. Currently I
> do
> > not know which sink could benefit from it. Maybe it is just my own
> problem.
> >
> > Best,
> > Guowei
> >
> >
> > On Wed, Sep 16, 2020 at 12:02 PM Guowei Ma <[hidden email]> wrote:
> >
> > >
> > > Hi, Steven
> > > Thanks you for your thoughtful ideas and concerns.
> > >
> > > >>I still like the concept of grouping data files per checkpoint for
> > > streaming mode. it is cleaner and probably easier to manage and deal
> with
> > > commit failures. Plus, it >>can reduce dupes for the at least once
> > > >>mode.  I understand checkpoint is not an option for batch execution.
> We
> > > don't have to expose the checkpointId in API, as >>long as  the
> internal
> > > bookkeeping groups data files by checkpoints for streaming >>mode.
> > >
> > > I think this problem(How to dedupe the combined committed data) also
> > > depends on where to place the agg/combine logic .
> > >
> > > 1. If the agg/combine takes place in the “commit” maybe we need to
> figure
> > > out how to give the aggregated committable a unique and auto-increment
> id
> > > in the committer.
> > > 2. If the agg/combine takes place in a separate operator maybe sink
> > > developer could maintain the id itself by using the state.
> > >
> > > I think this problem is also decided by what the topology pattern the
> > sink
> > > API should support. Actually there are already many other topology
> > > requirements. :)
> > >
> > > Best,
> > > Guowei
> > >
> > >
> > > On Wed, Sep 16, 2020 at 7:46 AM Steven Wu <[hidden email]>
> wrote:
> > >
> > >> > AFAIK the committer would not see the file-1-2 when ck1 happens in
> the
> > >> ExactlyOnce mode.
> > >>
> > >> @Guowei Ma <[hidden email]> I think you are right for exactly
> > once
> > >> checkpoint semantics. what about "at least once"? I guess we can argue
> > that
> > >> it is fine to commit file-1-2 for at least once mode.
> > >>
> > >> I still like the concept of grouping data files per checkpoint for
> > >> streaming mode. it is cleaner and probably easier to manage and deal
> > with
> > >> commit failures. Plus, it can reduce dupes for the at least once mode.
> > I
> > >> understand checkpoint is not an option for batch execution. We don't
> > have
> > >> to expose the checkpointId in API, as long as  the internal
> bookkeeping
> > >> groups data files by checkpoints for streaming mode.
> > >>
> > >>
> > >> On Tue, Sep 15, 2020 at 6:58 AM Steven Wu <[hidden email]>
> wrote:
> > >>
> > >>> > images don't make it through to the mailing lists. You would need
> to
> > >>> host the file somewhere and send a link.
> > >>>
> > >>> Sorry about that. Here is the sample DAG in google drawings.
> > >>>
> > >>>
> >
> https://docs.google.com/drawings/d/1-P8F2jF9RG9HHTtAfWEBRuU_2uV9aDTdqEt5dLs2JPk/edit?usp=sharing
> > >>>
> > >>>
> > >>> On Tue, Sep 15, 2020 at 4:58 AM Guowei Ma <[hidden email]>
> > wrote:
> > >>>
> > >>>> Hi, Dawid
> > >>>>
> > >>>> >>I still find the merging case the most confusing. I don't
> > necessarily
> > >>>> understand why do you need the "SingleFileCommit" step in this
> > scenario.
> > >>>> The way I
> > >>>> >> understand "commit" operation is that it makes some
> data/artifacts
> > >>>> visible to the external system, thus it should be immutable from a
> > >>>> point of
> > >>>> view of a single >>process. Having an additional step in the same
> > >>>> process
> > >>>> that works on committed data contradicts with those assumptions. I
> > >>>> might be
> > >>>> missing something though. >> Could you elaborate >why can't it be
> > >>>> something
> > >>>> like FileWriter -> FileMergeWriter -> Committer (either global or
> > >>>> non-global)? Again it might be just me not getting the example.
> > >>>>
> > >>>> I think you are right. The topology
> > >>>> "FileWriter->FileMergeWriter->Committer" could meet the merge
> > >>>> requirement.
> > >>>> The topology "FileWriter-> SingleFileCommitter -> FileMergeWriter ->
> > >>>> GlobalCommitter" reuses some code of the StreamingFileSink(For
> example
> > >>>> rolling policy) so it has the "SingleFileCommitter" in the topology.
> > In
> > >>>> general I want to use the case to show that there are different
> > >>>> topologies
> > >>>> according to the requirements.
> > >>>>
> > >>>> BTW: IIRC, @Jingsong Lee <[hidden email]> telled me
> that
> > >>>> the
> > >>>> actual topology of merged supported HiveSink is more complicated
> than
> > >>>> that.
> > >>>>
> > >>>>
> > >>>> >> I've just briefly skimmed over the proposed interfaces. I would
> > >>>> suggest
> > >>>> one
> > >>>> >> addition to the Writer interface (as I understand this is the
> > runtime
> > >>>> >> interface in this proposal?): add some availability method, to
> > >>>> avoid, if
> > >>>> >> possible, blocking calls on the sink. We already have similar
> > >>>> >> availability methods in the new sources [1] and in various places
> > in
> > >>>> the
> > >>>> >> network stack [2].
> > >>>> >> BTW Let's not forget about Piotr's comment. I think we could add
> > the
> > >>>> isAvailable or similar method to the Writer interface in the FLIP.
> > >>>>
> > >>>> Thanks @Dawid Wysakowicz <[hidden email]>  for your
> reminder.
> > >>>> There
> > >>>> are two many issues at the same time.
> > >>>>
> > >>>> In addition to what Ajjoscha said : there is very little system
> > support
> > >>>> it.   Another thing I worry about is that: Does the sink's snapshot
> > >>>> return
> > >>>> immediately when the sink's status is unavailable? Maybe we could do
> > it
> > >>>> by
> > >>>> dedupe some element in the state but I think it might be too
> > >>>> complicated.
> > >>>> For me I want to know is what specific sink will benefit from this
> > >>>> feature.  @piotr <[hidden email]>  Please correct me if  I
> > >>>> misunderstand you. thanks.
> > >>>>
> > >>>> Best,
> > >>>> Guowei
> > >>>>
> > >>>>
> > >>>> On Tue, Sep 15, 2020 at 3:55 PM Dawid Wysakowicz <
> > >>>> [hidden email]>
> > >>>> wrote:
> > >>>>
> > >>>> > What I understand is that HiveSink's implementation might need the
> > >>>> local
> > >>>> > committer(FileCommitter) because the file rename is needed.
> > >>>> > But the iceberg only needs to write the manifest file.  Would you
> > >>>> like to
> > >>>> > enlighten me why the Iceberg needs the local committer?
> > >>>> > Thanks
> > >>>> >
> > >>>> > Sorry if I caused a confusion here. I am not saying the Iceberg
> sink
> > >>>> needs
> > >>>> > a local committer. What I had in mind is that prior to the Iceberg
> > >>>> example
> > >>>> > I did not see a need for a "GlobalCommitter" in the streaming
> case.
> > I
> > >>>> > thought it is always enough to have the "normal" committer in that
> > >>>> case.
> > >>>> > Now I understand that this differentiation is not really about
> > logical
> > >>>> > separation. It is not really about the granularity with which we
> > >>>> commit,
> > >>>> > i.e. answering the "WHAT" question. It is really about the
> > >>>> performance and
> > >>>> > that in the end we will have a single "transaction", so it is
> about
> > >>>> > answering the question "HOW".
> > >>>> >
> > >>>> >
> > >>>> >    -
> > >>>> >
> > >>>> >    Commit a directory with merged files(Some user want to merge
> the
> > >>>> files
> > >>>> >    in a directory before committing the directory to Hive meta
> > store)
> > >>>> >
> > >>>> >
> > >>>> >    1.
> > >>>> >
> > >>>> >    FileWriter -> SingleFileCommit -> FileMergeWriter  ->
> > >>>> GlobalCommitter
> > >>>> >
> > >>>> > I still find the merging case the most confusing. I don't
> > necessarily
> > >>>> > understand why do you need the "SingleFileCommit" step in this
> > >>>> scenario.
> > >>>> > The way I understand "commit" operation is that it makes some
> > >>>> > data/artifacts visible to the external system, thus it should be
> > >>>> immutable
> > >>>> > from a point of view of a single process. Having an additional
> step
> > >>>> in the
> > >>>> > same process that works on committed data contradicts with those
> > >>>> > assumptions. I might be missing something though. Could you
> > elaborate
> > >>>> why
> > >>>> > can't it be something like FileWriter -> FileMergeWriter ->
> > Committer
> > >>>> > (either global or non-global)? Again it might be just me not
> getting
> > >>>> the
> > >>>> > example.
> > >>>> >
> > >>>> > I've just briefly skimmed over the proposed interfaces. I would
> > >>>> suggest one
> > >>>> > addition to the Writer interface (as I understand this is the
> > runtime
> > >>>> > interface in this proposal?): add some availability method, to
> > avoid,
> > >>>> if
> > >>>> > possible, blocking calls on the sink. We already have similar
> > >>>> > availability methods in the new sources [1] and in various places
> in
> > >>>> the
> > >>>> > network stack [2].
> > >>>> >
> > >>>> > BTW Let's not forget about Piotr's comment. I think we could add
> the
> > >>>> > isAvailable or similar method to the Writer interface in the FLIP.
> > >>>> >
> > >>>> > Best,
> > >>>> >
> > >>>> > Dawid
> > >>>> > On 15/09/2020 08:06, Guowei Ma wrote:
> > >>>> >
> > >>>> > I would think that we only need flush() and the semantics are that
> > it
> > >>>> > prepares for a commit, so on a physical level it would be called
> > from
> > >>>> > "prepareSnapshotPreBarrier". Now that I'm thinking about it more I
> > >>>> > think flush() should be renamed to something like
> "prepareCommit()".
> > >>>> >
> > >>>> > Generally speaking it is a good point that emitting the
> committables
> > >>>> > should happen before emitting the checkpoint barrier downstream.
> > >>>> > However, if I remember offline discussions well, the idea behind
> > >>>> > Writer#flush and Writer#snapshotState was to differentiate commit
> on
> > >>>> > checkpoint vs final checkpoint at the end of the job. Both of
> these
> > >>>> > methods could emit committables, but the flush should not leave
> any
> > in
> > >>>> > progress state (e.g. in case of file sink in STREAM mode, in
> > >>>> > snapshotState it could leave some open files that would be
> committed
> > >>>> in
> > >>>> > a subsequent cycle, however flush should close all files). The
> > >>>> > snapshotState as it is now can not be called in
> > >>>> > prepareSnapshotPreBarrier as it can store some state, which should
> > >>>> > happen in Operator#snapshotState as otherwise it would always be
> > >>>> > synchronous. Therefore I think we would need sth like:
> > >>>> >
> > >>>> > void prepareCommit(boolean flush, WriterOutput<CommT> output);
> > >>>> >
> > >>>> > ver 1:
> > >>>> >
> > >>>> > List<StateT> snapshotState();
> > >>>> >
> > >>>> > ver 2:
> > >>>> >
> > >>>> > void snapshotState(); // not sure if we need that method at all in
> > >>>> option
> > >>>> >
> > >>>> > 2
> > >>>> >
> > >>>> > I second Dawid's proposal. This is a valid scenario. And version2
> > >>>> does not
> > >>>> > need the snapshotState() any more.
> > >>>> >
> > >>>> >
> > >>>> > The Committer is as described in the FLIP, it's basically a
> function
> > >>>> > "void commit(Committable)". The GobalCommitter would be a function
> > >>>> "void
> > >>>> > commit(List<Committable>)". The former would be used by an S3 sink
> > >>>> where
> > >>>> > we can individually commit files to S3, a committable would be the
> > >>>> list
> > >>>> > of part uploads that will form the final file and the commit
> > operation
> > >>>> > creates the metadata in S3. The latter would be used by something
> > like
> > >>>> > Iceberg where the Committer needs a global view of all the commits
> > to
> > >>>> be
> > >>>> > efficient and not overwhelm the system.
> > >>>> >
> > >>>> > I don't know yet if sinks would only implement on type of commit
> > >>>> > function or potentially both at the same time, and maybe Commit
> can
> > >>>> > return some CommitResult that gets shipped to the GlobalCommit
> > >>>> function.
> > >>>> > I must admit it I did not get the need for Local/Normal + Global
> > >>>> > committer at first. The Iceberg example helped a lot. I think it
> > >>>> makes a
> > >>>> > lot of sense.
> > >>>> >
> > >>>> > @Dawid
> > >>>> > What I understand is that HiveSink's implementation might need the
> > >>>> local
> > >>>> > committer(FileCommitter) because the file rename is needed.
> > >>>> > But the iceberg only needs to write the manifest file.  Would you
> > >>>> like to
> > >>>> > enlighten me why the Iceberg needs the local committer?
> > >>>> > Thanks
> > >>>> >
> > >>>> > Best,
> > >>>> > Guowei
> > >>>> >
> > >>>> >
> > >>>> > On Mon, Sep 14, 2020 at 11:19 PM Dawid Wysakowicz <
> > >>>> [hidden email]> <[hidden email]>
> > >>>> > wrote:
> > >>>> >
> > >>>> >
> > >>>> > Hi all,
> > >>>> >
> > >>>> >
> > >>>> > I would think that we only need flush() and the semantics are that
> > it
> > >>>> > prepares for a commit, so on a physical level it would be called
> > from
> > >>>> > "prepareSnapshotPreBarrier". Now that I'm thinking about it more I
> > >>>> > think flush() should be renamed to something like
> "prepareCommit()".
> > >>>> >
> > >>>> > Generally speaking it is a good point that emitting the
> committables
> > >>>> > should happen before emitting the checkpoint barrier downstream.
> > >>>> > However, if I remember offline discussions well, the idea behind
> > >>>> > Writer#flush and Writer#snapshotState was to differentiate commit
> on
> > >>>> > checkpoint vs final checkpoint at the end of the job. Both of
> these
> > >>>> > methods could emit committables, but the flush should not leave
> any
> > in
> > >>>> > progress state (e.g. in case of file sink in STREAM mode, in
> > >>>> > snapshotState it could leave some open files that would be
> committed
> > >>>> in
> > >>>> > a subsequent cycle, however flush should close all files). The
> > >>>> > snapshotState as it is now can not be called in
> > >>>> > prepareSnapshotPreBarrier as it can store some state, which should
> > >>>> > happen in Operator#snapshotState as otherwise it would always be
> > >>>> > synchronous. Therefore I think we would need sth like:
> > >>>> >
> > >>>> > void prepareCommit(boolean flush, WriterOutput<CommT> output);
> > >>>> >
> > >>>> > ver 1:
> > >>>> >
> > >>>> > List<StateT> snapshotState();
> > >>>> >
> > >>>> > ver 2:
> > >>>> >
> > >>>> > void snapshotState(); // not sure if we need that method at all in
> > >>>> option 2
> > >>>> >
> > >>>> >
> > >>>> > The Committer is as described in the FLIP, it's basically a
> function
> > >>>> > "void commit(Committable)". The GobalCommitter would be a function
> > >>>> "void
> > >>>> > commit(List<Committable>)". The former would be used by an S3 sink
> > >>>> where
> > >>>> > we can individually commit files to S3, a committable would be the
> > >>>> list
> > >>>> > of part uploads that will form the final file and the commit
> > operation
> > >>>> > creates the metadata in S3. The latter would be used by something
> > like
> > >>>> > Iceberg where the Committer needs a global view of all the commits
> > to
> > >>>> be
> > >>>> > efficient and not overwhelm the system.
> > >>>> >
> > >>>> > I don't know yet if sinks would only implement on type of commit
> > >>>> > function or potentially both at the same time, and maybe Commit
> can
> > >>>> > return some CommitResult that gets shipped to the GlobalCommit
> > >>>> function.
> > >>>> >
> > >>>> > I must admit it I did not get the need for Local/Normal + Global
> > >>>> > committer at first. The Iceberg example helped a lot. I think it
> > >>>> makes a
> > >>>> > lot of sense.
> > >>>> >
> > >>>> >
> > >>>> > For Iceberg, writers don't need any state. But the GlobalCommitter
> > >>>> > needs to
> > >>>> > checkpoint StateT. For the committer, CommT is "DataFile". Since a
> > >>>> single
> > >>>> > committer can collect thousands (or more) data files in one
> > checkpoint
> > >>>> > cycle, as an optimization we checkpoint a single "ManifestFile"
> (for
> > >>>> the
> > >>>> > collected thousands data files) as StateT. This allows us to
> absorb
> > >>>> > extended commit outages without losing written/uploaded data
> files,
> > as
> > >>>> > operator state size is as small as one manifest file per
> checkpoint
> > >>>> cycle
> > >>>> > [2].
> > >>>> > ------------------
> > >>>> > StateT snapshotState(SnapshotContext context) throws Exception;
> > >>>> >
> > >>>> > That means we also need the restoreCommitter API in the Sink
> > interface
> > >>>> > ---------------
> > >>>> > Committer<CommT, StateT> restoreCommitter(InitContext context,
> > StateT
> > >>>> > state);
> > >>>> >
> > >>>> > I think this might be a valid case. Not sure though if I would go
> > >>>> with a
> > >>>> > "state" there. Having a state in a committer would imply we need a
> > >>>> > collect method as well. So far we needed a single method
> commit(...)
> > >>>> and
> > >>>> > the bookkeeping of the committables could be handled by the
> > >>>> framework. I
> > >>>> > think something like an optional combiner in the GlobalCommitter
> > would
> > >>>> > be enough. What do you think?
> > >>>> >
> > >>>> > GlobalCommitter<CommT, GlobalCommT> {
> > >>>> >
> > >>>> >     void commit(GlobalCommT globalCommittables);
> > >>>> >
> > >>>> >     GlobalCommT combine(List<CommT> committables);
> > >>>> >
> > >>>> > }
> > >>>> >
> > >>>> > A different problem that I see here is how do we handle commit
> > >>>> failures.
> > >>>> > Should the committables (both normal and global be included in the
> > >>>> next
> > >>>> > cycle, shall we retry it, ...) I think it would be worth laying it
> > out
> > >>>> > in the FLIP.
> > >>>> >
> > >>>> > @Aljoscha I think you can find the code Steven was referring in
> > here:
> > >>>> >
> > >>>>
> >
> https://github.com/Netflix-Skunkworks/nfflink-connector-iceberg/blob/master/nfflink-connector-iceberg/src/main/java/com/netflix/spaas/nfflink/connector/iceberg/sink/IcebergCommitter.java
> > >>>> >
> > >>>> > Best,
> > >>>> >
> > >>>> > Dawid
> > >>>> >
> > >>>> > On 14/09/2020 15:19, Aljoscha Krettek wrote:
> > >>>> >
> > >>>> > On 14.09.20 01:23, Steven Wu wrote:
> > >>>> >
> > >>>> > ## Writer interface
> > >>>> >
> > >>>> > For the Writer interface, should we add "*prepareSnapshot"* before
> > the
> > >>>> > checkpoint barrier emitted downstream?  IcebergWriter would need
> it.
> > >>>> Or
> > >>>> > would the framework call "*flush*" before the barrier emitted
> > >>>> > downstream?
> > >>>> > that guarantee would achieve the same goal.
> > >>>> >
> > >>>> > I would think that we only need flush() and the semantics are that
> > it
> > >>>> > prepares for a commit, so on a physical level it would be called
> > from
> > >>>> > "prepareSnapshotPreBarrier". Now that I'm thinking about it more I
> > >>>> > think flush() should be renamed to something like
> "prepareCommit()".
> > >>>> >
> > >>>> > @Guowei, what do you think about this?
> > >>>> >
> > >>>> >
> > >>>> > In [1], we discussed the reason for Writer to emit (checkpointId,
> > >>>> CommT)
> > >>>> > tuple to the committer. The committer needs checkpointId to
> separate
> > >>>> out
> > >>>> > data files for different checkpoints if concurrent checkpoints are
> > >>>> > enabled.
> > >>>> >
> > >>>> > When can this happen? Even with concurrent checkpoints the
> snapshot
> > >>>> > barriers would still cleanly segregate the input stream of an
> > operator
> > >>>> > into tranches that should manifest in only one checkpoint. With
> > >>>> > concurrent checkpoints, all that can happen is that we start a
> > >>>> > checkpoint before a last one is confirmed completed.
> > >>>> >
> > >>>> > Unless there is some weirdness in the sources and some sources
> start
> > >>>> > chk1 first and some other ones start chk2 first?
> > >>>> >
> > >>>> > @Piotrek, do you think this is a problem?
> > >>>> >
> > >>>> >
> > >>>> > For the Committer interface, I am wondering if we should split the
> > >>>> > single
> > >>>> > commit method into separate "*collect"* and "*commit"* methods?
> This
> > >>>> > way,
> > >>>> > it can handle both single and multiple CommT objects.
> > >>>> >
> > >>>> > I think we can't do this. If the sink only needs a regular
> Commiter,
> > >>>> > we can perform the commits in parallel, possibly on different
> > >>>> > machines. Only when the sink needs a GlobalCommitter would we need
> > to
> > >>>> > ship all commits to a single process and perform the commit there.
> > If
> > >>>> > both methods were unified in one interface we couldn't make the
> > >>>> > decision of were to commit in the framework code.
> > >>>> >
> > >>>> >
> > >>>> > For Iceberg, writers don't need any state. But the GlobalCommitter
> > >>>> > needs to
> > >>>> > checkpoint StateT. For the committer, CommT is "DataFile". Since a
> > >>>> > single
> > >>>> > committer can collect thousands (or more) data files in one
> > checkpoint
> > >>>> > cycle, as an optimization we checkpoint a single "ManifestFile"
> (for
> > >>>> the
> > >>>> > collected thousands data files) as StateT. This allows us to
> absorb
> > >>>> > extended commit outages without losing written/uploaded data
> files,
> > as
> > >>>> > operator state size is as small as one manifest file per
> checkpoint
> > >>>> > cycle
> > >>>> >
> > >>>> > You could have a point here. Is the code for this available in
> > >>>> > open-source? I was checking out
> > >>>> >
> > >>>> >
> > >>>> >
> > >>>>
> >
> https://github.com/apache/iceberg/blob/master/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergFilesCommitter.java
> > >>>> >
> > >>>> > and didn't find the ManifestFile optimization there.
> > >>>> >
> > >>>> > Best,
> > >>>> > Aljoscha
> > >>>> >
> > >>>> >
> > >>>> >
> > >>>>
> > >>>
> >
>
Reply | Threaded
Open this post in threaded view
|

Re: [DISCUSS] FLIP-143: Unified Sink API

Guowei Ma
In reply to this post by Steven Wu
Hi, Steven

I am not particularly sure whether to provide id in GlobalCommit.

But my understanding is: if the committer function is idempotent, the
framework can guarantee exactly once semantics in batch/stream execution
mode. But I think maybe the idempotence should be guaranteed by the sink
developer, not on the basic API.

We could  provide an id in GlobalCommit. But the following question would
be that: do we need to provide an id for a normal committable? I would like
to say that I prefer to make the committer single responsibility.

I think maybe we could have an answer when the first nonconsensual question
is resolved.

Aboving is just my personal opinion. I think this is still an open question.

Thank you again for your valuable and thoughtful response.

Best,
Guowei


On Thu, Sep 17, 2020 at 10:53 AM Steven Wu <[hidden email]> wrote:

> Guowei, thanks a lot for the summary. Here are a couple more questions that
> need more clarification for the GlobalCommitter case.
>
> * framework provides some sort of unique id per GlobalCommT (e.g. nonce or
> some sort of transaction id)
> * commit failure handling. Should we roll over to the next cycle? if so, we
> may need commit(List<GlobalCommT> )
>
> On Wed, Sep 16, 2020 at 2:11 AM Piotr Nowojski <[hidden email]>
> wrote:
>
> > Hey
> >
> > Thanks Dawid for bringing up my suggestion :)
> >
> > > I'm not so sure about this, the sinks I'm aware of would not be able to
> > > implement this method: Kafka doesn't have this, I didn't see it in the
> > > Iceberg interfaces, and HDFS/S3 also don't have it.
> >
> > Aljoscha, as I wrote, FlinkKafkaProducer is actually one for which we
> > could do some magic. At the very least we could use
> > `FlinkKafkaProducer#pendingRecords` to make the sink unavailable when
> some
> > threshold is exceeded. Alternatively, maybe we could hook in to the
> > KafkaProducer's buffer state [1]:
> >
> > > The buffer.memory controls the total amount of memory available to the
> > producer for buffering.
> > > If records are sent faster than they can be transmitted to the server
> > then this buffer space will be exhausted.
> > > When the buffer space is exhausted additional send calls will block.
> >
> > As far as I can see, Kafka is exposing the `buffer-available-bytes`
> > metric, which we might use instead of `pendingRecords`. Heck, we are
> > already hacking KafkaProducer with reflections, we could access
> > `org.apache.kafka.clients.producer.KafkaProducer#accumulator` field to
> > call  `accumulator.bufferPoolAvailableMemory()` method, if metric would
> be
> > to expensive to check per every record.
> >
> > Furthermore, I'm pretty sure other sinks (maybe not all) provide similar
> > features. If we are desperate, we could always contribute something to
> > those systems to make them expose the internal buffer's state.
> >
> > If we are really desperate, we could provide a generic records handover
> > wrapper sink, that would have a buffer of N (5? 10? ) records and would
> be
> > handing over those records to the blocking sink running in another
> thread.
> > If the buffer is full, the sink would be unavailable.
> >
> > Guowei
> > > Does the sink's snapshot return immediately when the sink's status is
> > unavailable?
> >
> > State snapshot call is generally speaking non blocking already, so it
> > should not be an issue. If it's blocking and if it will be solving some
> > problem, we could later decide in the runtime code to not execute
> snapshot
> > calls if a sink is unavailable. Think about isAvailable more like a hint
> > from the operator to the runtime, which we can use to make better
> > decisions. Also take a look at the FLIP-27 sources (`SourceReader`),
> where
> > there already is `isAvailable()` method. It would be best if new sinks
> > would just duplicate the same contract.
> >
> > > For me I want to know is what specific sink will benefit from this
> > feature
> >
> > It's not the sinks that would benefit from this, but other parts of the
> > system. Currently task thread is blocked on backpressured Sink, it's
> > blocking some things from happening (checkpointing, closing, ...). If we
> > make sinks non blocking (as is the network stack in the most part and as
> > are the FLIP-27 sources), we will be able to snapshot state of the
> operator
> > immediately. For example, change from blocking to non blocking sources
> was
> > speeding up unaligned checkpoints from ~30seconds down to ~5seconds in
> > our benchmarks, but the difference can be even more profound (hours
> instead
> > of seconds/minutes as reported by some users).
> >
> > Piotrek
> >
> > [1]
> >
> https://kafka.apache.org/10/javadoc/org/apache/kafka/clients/producer/KafkaProducer.html
> >
> > śr., 16 wrz 2020 o 06:29 Guowei Ma <[hidden email]> napisał(a):
> >
> >> Hi,all
> >>
> >> Thanks for all your valuable options and ideas.Currently there are many
> >> topics in the mail. I try to summarize what is consensus and what is
> not.
> >> Correct me if I am wrong.
> >>
> >> ## Consensus
> >>
> >> 1. The motivation of the unified sink API is to decouple the sink
> >> implementation from the different runtime execution mode.
> >> 2. The initial scope of the unified sink API only covers the file system
> >> type, which supports the real transactions. The FLIP focuses more on the
> >> semantics the new sink api should support.
> >> 3. We prefer the first alternative API, which could give the framework a
> >> greater opportunity to optimize.
> >> 4. The `Writer` needs to add a method `prepareCommit`, which would be
> >> called from `prepareSnapshotPreBarrier`. And remove the `Flush` method.
> >> 5. The FLIP could move the `Snapshot & Drain` section in order to be
> more
> >> focused.
> >>
> >> ## Not Consensus
> >>
> >> 1. What should the “Unified Sink API” support/cover? The API can
> >> “unified”(decoupe) the commit operation in the term of supporting
> exactly
> >> once semantics. However, even if we narrow down the initial supported
> >> system to the file system there would be different topology
> requirements.
> >> These requirements come from performance optimization
> >> (IceBergSink/MergeHiveSink) or functionality(i.e. whether a bucket is
> >> “finished”).  Should the unified sink API support these requirements?
> >> 2. The API does not expose the checkpoint-id because the batch execution
> >> mode does not have the normal checkpoint. But there still some
> >> implementations depend on this.(IceBergSink uses this to do some
> dedupe).
> >> I think how to support this requirement depends on the first open
> >> question.
> >> 3. Whether the `Writer` supports async functionality or not. Currently I
> >> do
> >> not know which sink could benefit from it. Maybe it is just my own
> >> problem.
> >>
> >> Best,
> >> Guowei
> >>
> >>
> >> On Wed, Sep 16, 2020 at 12:02 PM Guowei Ma <[hidden email]>
> wrote:
> >>
> >> >
> >> > Hi, Steven
> >> > Thanks you for your thoughtful ideas and concerns.
> >> >
> >> > >>I still like the concept of grouping data files per checkpoint for
> >> > streaming mode. it is cleaner and probably easier to manage and deal
> >> with
> >> > commit failures. Plus, it >>can reduce dupes for the at least once
> >> > >>mode.  I understand checkpoint is not an option for batch execution.
> >> We
> >> > don't have to expose the checkpointId in API, as >>long as  the
> internal
> >> > bookkeeping groups data files by checkpoints for streaming >>mode.
> >> >
> >> > I think this problem(How to dedupe the combined committed data) also
> >> > depends on where to place the agg/combine logic .
> >> >
> >> > 1. If the agg/combine takes place in the “commit” maybe we need to
> >> figure
> >> > out how to give the aggregated committable a unique and auto-increment
> >> id
> >> > in the committer.
> >> > 2. If the agg/combine takes place in a separate operator maybe sink
> >> > developer could maintain the id itself by using the state.
> >> >
> >> > I think this problem is also decided by what the topology pattern the
> >> sink
> >> > API should support. Actually there are already many other topology
> >> > requirements. :)
> >> >
> >> > Best,
> >> > Guowei
> >> >
> >> >
> >> > On Wed, Sep 16, 2020 at 7:46 AM Steven Wu <[hidden email]>
> wrote:
> >> >
> >> >> > AFAIK the committer would not see the file-1-2 when ck1 happens in
> >> the
> >> >> ExactlyOnce mode.
> >> >>
> >> >> @Guowei Ma <[hidden email]> I think you are right for exactly
> >> once
> >> >> checkpoint semantics. what about "at least once"? I guess we can
> argue
> >> that
> >> >> it is fine to commit file-1-2 for at least once mode.
> >> >>
> >> >> I still like the concept of grouping data files per checkpoint for
> >> >> streaming mode. it is cleaner and probably easier to manage and deal
> >> with
> >> >> commit failures. Plus, it can reduce dupes for the at least once
> >> mode.  I
> >> >> understand checkpoint is not an option for batch execution. We don't
> >> have
> >> >> to expose the checkpointId in API, as long as  the internal
> bookkeeping
> >> >> groups data files by checkpoints for streaming mode.
> >> >>
> >> >>
> >> >> On Tue, Sep 15, 2020 at 6:58 AM Steven Wu <[hidden email]>
> >> wrote:
> >> >>
> >> >>> > images don't make it through to the mailing lists. You would need
> to
> >> >>> host the file somewhere and send a link.
> >> >>>
> >> >>> Sorry about that. Here is the sample DAG in google drawings.
> >> >>>
> >> >>>
> >>
> https://docs.google.com/drawings/d/1-P8F2jF9RG9HHTtAfWEBRuU_2uV9aDTdqEt5dLs2JPk/edit?usp=sharing
> >> >>>
> >> >>>
> >> >>> On Tue, Sep 15, 2020 at 4:58 AM Guowei Ma <[hidden email]>
> >> wrote:
> >> >>>
> >> >>>> Hi, Dawid
> >> >>>>
> >> >>>> >>I still find the merging case the most confusing. I don't
> >> necessarily
> >> >>>> understand why do you need the "SingleFileCommit" step in this
> >> scenario.
> >> >>>> The way I
> >> >>>> >> understand "commit" operation is that it makes some
> data/artifacts
> >> >>>> visible to the external system, thus it should be immutable from a
> >> >>>> point of
> >> >>>> view of a single >>process. Having an additional step in the same
> >> >>>> process
> >> >>>> that works on committed data contradicts with those assumptions. I
> >> >>>> might be
> >> >>>> missing something though. >> Could you elaborate >why can't it be
> >> >>>> something
> >> >>>> like FileWriter -> FileMergeWriter -> Committer (either global or
> >> >>>> non-global)? Again it might be just me not getting the example.
> >> >>>>
> >> >>>> I think you are right. The topology
> >> >>>> "FileWriter->FileMergeWriter->Committer" could meet the merge
> >> >>>> requirement.
> >> >>>> The topology "FileWriter-> SingleFileCommitter -> FileMergeWriter
> ->
> >> >>>> GlobalCommitter" reuses some code of the StreamingFileSink(For
> >> example
> >> >>>> rolling policy) so it has the "SingleFileCommitter" in the
> topology.
> >> In
> >> >>>> general I want to use the case to show that there are different
> >> >>>> topologies
> >> >>>> according to the requirements.
> >> >>>>
> >> >>>> BTW: IIRC, @Jingsong Lee <[hidden email]> telled me
> that
> >> >>>> the
> >> >>>> actual topology of merged supported HiveSink is more complicated
> than
> >> >>>> that.
> >> >>>>
> >> >>>>
> >> >>>> >> I've just briefly skimmed over the proposed interfaces. I would
> >> >>>> suggest
> >> >>>> one
> >> >>>> >> addition to the Writer interface (as I understand this is the
> >> runtime
> >> >>>> >> interface in this proposal?): add some availability method, to
> >> >>>> avoid, if
> >> >>>> >> possible, blocking calls on the sink. We already have similar
> >> >>>> >> availability methods in the new sources [1] and in various
> places
> >> in
> >> >>>> the
> >> >>>> >> network stack [2].
> >> >>>> >> BTW Let's not forget about Piotr's comment. I think we could add
> >> the
> >> >>>> isAvailable or similar method to the Writer interface in the FLIP.
> >> >>>>
> >> >>>> Thanks @Dawid Wysakowicz <[hidden email]>  for your
> >> reminder.
> >> >>>> There
> >> >>>> are two many issues at the same time.
> >> >>>>
> >> >>>> In addition to what Ajjoscha said : there is very little system
> >> support
> >> >>>> it.   Another thing I worry about is that: Does the sink's snapshot
> >> >>>> return
> >> >>>> immediately when the sink's status is unavailable? Maybe we could
> do
> >> it
> >> >>>> by
> >> >>>> dedupe some element in the state but I think it might be too
> >> >>>> complicated.
> >> >>>> For me I want to know is what specific sink will benefit from this
> >> >>>> feature.  @piotr <[hidden email]>  Please correct me if  I
> >> >>>> misunderstand you. thanks.
> >> >>>>
> >> >>>> Best,
> >> >>>> Guowei
> >> >>>>
> >> >>>>
> >> >>>> On Tue, Sep 15, 2020 at 3:55 PM Dawid Wysakowicz <
> >> >>>> [hidden email]>
> >> >>>> wrote:
> >> >>>>
> >> >>>> > What I understand is that HiveSink's implementation might need
> the
> >> >>>> local
> >> >>>> > committer(FileCommitter) because the file rename is needed.
> >> >>>> > But the iceberg only needs to write the manifest file.  Would you
> >> >>>> like to
> >> >>>> > enlighten me why the Iceberg needs the local committer?
> >> >>>> > Thanks
> >> >>>> >
> >> >>>> > Sorry if I caused a confusion here. I am not saying the Iceberg
> >> sink
> >> >>>> needs
> >> >>>> > a local committer. What I had in mind is that prior to the
> Iceberg
> >> >>>> example
> >> >>>> > I did not see a need for a "GlobalCommitter" in the streaming
> >> case. I
> >> >>>> > thought it is always enough to have the "normal" committer in
> that
> >> >>>> case.
> >> >>>> > Now I understand that this differentiation is not really about
> >> logical
> >> >>>> > separation. It is not really about the granularity with which we
> >> >>>> commit,
> >> >>>> > i.e. answering the "WHAT" question. It is really about the
> >> >>>> performance and
> >> >>>> > that in the end we will have a single "transaction", so it is
> about
> >> >>>> > answering the question "HOW".
> >> >>>> >
> >> >>>> >
> >> >>>> >    -
> >> >>>> >
> >> >>>> >    Commit a directory with merged files(Some user want to merge
> the
> >> >>>> files
> >> >>>> >    in a directory before committing the directory to Hive meta
> >> store)
> >> >>>> >
> >> >>>> >
> >> >>>> >    1.
> >> >>>> >
> >> >>>> >    FileWriter -> SingleFileCommit -> FileMergeWriter  ->
> >> >>>> GlobalCommitter
> >> >>>> >
> >> >>>> > I still find the merging case the most confusing. I don't
> >> necessarily
> >> >>>> > understand why do you need the "SingleFileCommit" step in this
> >> >>>> scenario.
> >> >>>> > The way I understand "commit" operation is that it makes some
> >> >>>> > data/artifacts visible to the external system, thus it should be
> >> >>>> immutable
> >> >>>> > from a point of view of a single process. Having an additional
> step
> >> >>>> in the
> >> >>>> > same process that works on committed data contradicts with those
> >> >>>> > assumptions. I might be missing something though. Could you
> >> elaborate
> >> >>>> why
> >> >>>> > can't it be something like FileWriter -> FileMergeWriter ->
> >> Committer
> >> >>>> > (either global or non-global)? Again it might be just me not
> >> getting
> >> >>>> the
> >> >>>> > example.
> >> >>>> >
> >> >>>> > I've just briefly skimmed over the proposed interfaces. I would
> >> >>>> suggest one
> >> >>>> > addition to the Writer interface (as I understand this is the
> >> runtime
> >> >>>> > interface in this proposal?): add some availability method, to
> >> avoid,
> >> >>>> if
> >> >>>> > possible, blocking calls on the sink. We already have similar
> >> >>>> > availability methods in the new sources [1] and in various places
> >> in
> >> >>>> the
> >> >>>> > network stack [2].
> >> >>>> >
> >> >>>> > BTW Let's not forget about Piotr's comment. I think we could add
> >> the
> >> >>>> > isAvailable or similar method to the Writer interface in the
> FLIP.
> >> >>>> >
> >> >>>> > Best,
> >> >>>> >
> >> >>>> > Dawid
> >> >>>> > On 15/09/2020 08:06, Guowei Ma wrote:
> >> >>>> >
> >> >>>> > I would think that we only need flush() and the semantics are
> that
> >> it
> >> >>>> > prepares for a commit, so on a physical level it would be called
> >> from
> >> >>>> > "prepareSnapshotPreBarrier". Now that I'm thinking about it more
> I
> >> >>>> > think flush() should be renamed to something like
> >> "prepareCommit()".
> >> >>>> >
> >> >>>> > Generally speaking it is a good point that emitting the
> >> committables
> >> >>>> > should happen before emitting the checkpoint barrier downstream.
> >> >>>> > However, if I remember offline discussions well, the idea behind
> >> >>>> > Writer#flush and Writer#snapshotState was to differentiate commit
> >> on
> >> >>>> > checkpoint vs final checkpoint at the end of the job. Both of
> these
> >> >>>> > methods could emit committables, but the flush should not leave
> >> any in
> >> >>>> > progress state (e.g. in case of file sink in STREAM mode, in
> >> >>>> > snapshotState it could leave some open files that would be
> >> committed
> >> >>>> in
> >> >>>> > a subsequent cycle, however flush should close all files). The
> >> >>>> > snapshotState as it is now can not be called in
> >> >>>> > prepareSnapshotPreBarrier as it can store some state, which
> should
> >> >>>> > happen in Operator#snapshotState as otherwise it would always be
> >> >>>> > synchronous. Therefore I think we would need sth like:
> >> >>>> >
> >> >>>> > void prepareCommit(boolean flush, WriterOutput<CommT> output);
> >> >>>> >
> >> >>>> > ver 1:
> >> >>>> >
> >> >>>> > List<StateT> snapshotState();
> >> >>>> >
> >> >>>> > ver 2:
> >> >>>> >
> >> >>>> > void snapshotState(); // not sure if we need that method at all
> in
> >> >>>> option
> >> >>>> >
> >> >>>> > 2
> >> >>>> >
> >> >>>> > I second Dawid's proposal. This is a valid scenario. And version2
> >> >>>> does not
> >> >>>> > need the snapshotState() any more.
> >> >>>> >
> >> >>>> >
> >> >>>> > The Committer is as described in the FLIP, it's basically a
> >> function
> >> >>>> > "void commit(Committable)". The GobalCommitter would be a
> function
> >> >>>> "void
> >> >>>> > commit(List<Committable>)". The former would be used by an S3
> sink
> >> >>>> where
> >> >>>> > we can individually commit files to S3, a committable would be
> the
> >> >>>> list
> >> >>>> > of part uploads that will form the final file and the commit
> >> operation
> >> >>>> > creates the metadata in S3. The latter would be used by something
> >> like
> >> >>>> > Iceberg where the Committer needs a global view of all the
> commits
> >> to
> >> >>>> be
> >> >>>> > efficient and not overwhelm the system.
> >> >>>> >
> >> >>>> > I don't know yet if sinks would only implement on type of commit
> >> >>>> > function or potentially both at the same time, and maybe Commit
> can
> >> >>>> > return some CommitResult that gets shipped to the GlobalCommit
> >> >>>> function.
> >> >>>> > I must admit it I did not get the need for Local/Normal + Global
> >> >>>> > committer at first. The Iceberg example helped a lot. I think it
> >> >>>> makes a
> >> >>>> > lot of sense.
> >> >>>> >
> >> >>>> > @Dawid
> >> >>>> > What I understand is that HiveSink's implementation might need
> the
> >> >>>> local
> >> >>>> > committer(FileCommitter) because the file rename is needed.
> >> >>>> > But the iceberg only needs to write the manifest file.  Would you
> >> >>>> like to
> >> >>>> > enlighten me why the Iceberg needs the local committer?
> >> >>>> > Thanks
> >> >>>> >
> >> >>>> > Best,
> >> >>>> > Guowei
> >> >>>> >
> >> >>>> >
> >> >>>> > On Mon, Sep 14, 2020 at 11:19 PM Dawid Wysakowicz <
> >> >>>> [hidden email]> <[hidden email]>
> >> >>>> > wrote:
> >> >>>> >
> >> >>>> >
> >> >>>> > Hi all,
> >> >>>> >
> >> >>>> >
> >> >>>> > I would think that we only need flush() and the semantics are
> that
> >> it
> >> >>>> > prepares for a commit, so on a physical level it would be called
> >> from
> >> >>>> > "prepareSnapshotPreBarrier". Now that I'm thinking about it more
> I
> >> >>>> > think flush() should be renamed to something like
> >> "prepareCommit()".
> >> >>>> >
> >> >>>> > Generally speaking it is a good point that emitting the
> >> committables
> >> >>>> > should happen before emitting the checkpoint barrier downstream.
> >> >>>> > However, if I remember offline discussions well, the idea behind
> >> >>>> > Writer#flush and Writer#snapshotState was to differentiate commit
> >> on
> >> >>>> > checkpoint vs final checkpoint at the end of the job. Both of
> these
> >> >>>> > methods could emit committables, but the flush should not leave
> >> any in
> >> >>>> > progress state (e.g. in case of file sink in STREAM mode, in
> >> >>>> > snapshotState it could leave some open files that would be
> >> committed
> >> >>>> in
> >> >>>> > a subsequent cycle, however flush should close all files). The
> >> >>>> > snapshotState as it is now can not be called in
> >> >>>> > prepareSnapshotPreBarrier as it can store some state, which
> should
> >> >>>> > happen in Operator#snapshotState as otherwise it would always be
> >> >>>> > synchronous. Therefore I think we would need sth like:
> >> >>>> >
> >> >>>> > void prepareCommit(boolean flush, WriterOutput<CommT> output);
> >> >>>> >
> >> >>>> > ver 1:
> >> >>>> >
> >> >>>> > List<StateT> snapshotState();
> >> >>>> >
> >> >>>> > ver 2:
> >> >>>> >
> >> >>>> > void snapshotState(); // not sure if we need that method at all
> in
> >> >>>> option 2
> >> >>>> >
> >> >>>> >
> >> >>>> > The Committer is as described in the FLIP, it's basically a
> >> function
> >> >>>> > "void commit(Committable)". The GobalCommitter would be a
> function
> >> >>>> "void
> >> >>>> > commit(List<Committable>)". The former would be used by an S3
> sink
> >> >>>> where
> >> >>>> > we can individually commit files to S3, a committable would be
> the
> >> >>>> list
> >> >>>> > of part uploads that will form the final file and the commit
> >> operation
> >> >>>> > creates the metadata in S3. The latter would be used by something
> >> like
> >> >>>> > Iceberg where the Committer needs a global view of all the
> commits
> >> to
> >> >>>> be
> >> >>>> > efficient and not overwhelm the system.
> >> >>>> >
> >> >>>> > I don't know yet if sinks would only implement on type of commit
> >> >>>> > function or potentially both at the same time, and maybe Commit
> can
> >> >>>> > return some CommitResult that gets shipped to the GlobalCommit
> >> >>>> function.
> >> >>>> >
> >> >>>> > I must admit it I did not get the need for Local/Normal + Global
> >> >>>> > committer at first. The Iceberg example helped a lot. I think it
> >> >>>> makes a
> >> >>>> > lot of sense.
> >> >>>> >
> >> >>>> >
> >> >>>> > For Iceberg, writers don't need any state. But the
> GlobalCommitter
> >> >>>> > needs to
> >> >>>> > checkpoint StateT. For the committer, CommT is "DataFile". Since
> a
> >> >>>> single
> >> >>>> > committer can collect thousands (or more) data files in one
> >> checkpoint
> >> >>>> > cycle, as an optimization we checkpoint a single "ManifestFile"
> >> (for
> >> >>>> the
> >> >>>> > collected thousands data files) as StateT. This allows us to
> absorb
> >> >>>> > extended commit outages without losing written/uploaded data
> >> files, as
> >> >>>> > operator state size is as small as one manifest file per
> checkpoint
> >> >>>> cycle
> >> >>>> > [2].
> >> >>>> > ------------------
> >> >>>> > StateT snapshotState(SnapshotContext context) throws Exception;
> >> >>>> >
> >> >>>> > That means we also need the restoreCommitter API in the Sink
> >> interface
> >> >>>> > ---------------
> >> >>>> > Committer<CommT, StateT> restoreCommitter(InitContext context,
> >> StateT
> >> >>>> > state);
> >> >>>> >
> >> >>>> > I think this might be a valid case. Not sure though if I would go
> >> >>>> with a
> >> >>>> > "state" there. Having a state in a committer would imply we need
> a
> >> >>>> > collect method as well. So far we needed a single method
> >> commit(...)
> >> >>>> and
> >> >>>> > the bookkeeping of the committables could be handled by the
> >> >>>> framework. I
> >> >>>> > think something like an optional combiner in the GlobalCommitter
> >> would
> >> >>>> > be enough. What do you think?
> >> >>>> >
> >> >>>> > GlobalCommitter<CommT, GlobalCommT> {
> >> >>>> >
> >> >>>> >     void commit(GlobalCommT globalCommittables);
> >> >>>> >
> >> >>>> >     GlobalCommT combine(List<CommT> committables);
> >> >>>> >
> >> >>>> > }
> >> >>>> >
> >> >>>> > A different problem that I see here is how do we handle commit
> >> >>>> failures.
> >> >>>> > Should the committables (both normal and global be included in
> the
> >> >>>> next
> >> >>>> > cycle, shall we retry it, ...) I think it would be worth laying
> it
> >> out
> >> >>>> > in the FLIP.
> >> >>>> >
> >> >>>> > @Aljoscha I think you can find the code Steven was referring in
> >> here:
> >> >>>> >
> >> >>>>
> >>
> https://github.com/Netflix-Skunkworks/nfflink-connector-iceberg/blob/master/nfflink-connector-iceberg/src/main/java/com/netflix/spaas/nfflink/connector/iceberg/sink/IcebergCommitter.java
> >> >>>> >
> >> >>>> > Best,
> >> >>>> >
> >> >>>> > Dawid
> >> >>>> >
> >> >>>> > On 14/09/2020 15:19, Aljoscha Krettek wrote:
> >> >>>> >
> >> >>>> > On 14.09.20 01:23, Steven Wu wrote:
> >> >>>> >
> >> >>>> > ## Writer interface
> >> >>>> >
> >> >>>> > For the Writer interface, should we add "*prepareSnapshot"*
> before
> >> the
> >> >>>> > checkpoint barrier emitted downstream?  IcebergWriter would need
> >> it.
> >> >>>> Or
> >> >>>> > would the framework call "*flush*" before the barrier emitted
> >> >>>> > downstream?
> >> >>>> > that guarantee would achieve the same goal.
> >> >>>> >
> >> >>>> > I would think that we only need flush() and the semantics are
> that
> >> it
> >> >>>> > prepares for a commit, so on a physical level it would be called
> >> from
> >> >>>> > "prepareSnapshotPreBarrier". Now that I'm thinking about it more
> I
> >> >>>> > think flush() should be renamed to something like
> >> "prepareCommit()".
> >> >>>> >
> >> >>>> > @Guowei, what do you think about this?
> >> >>>> >
> >> >>>> >
> >> >>>> > In [1], we discussed the reason for Writer to emit (checkpointId,
> >> >>>> CommT)
> >> >>>> > tuple to the committer. The committer needs checkpointId to
> >> separate
> >> >>>> out
> >> >>>> > data files for different checkpoints if concurrent checkpoints
> are
> >> >>>> > enabled.
> >> >>>> >
> >> >>>> > When can this happen? Even with concurrent checkpoints the
> snapshot
> >> >>>> > barriers would still cleanly segregate the input stream of an
> >> operator
> >> >>>> > into tranches that should manifest in only one checkpoint. With
> >> >>>> > concurrent checkpoints, all that can happen is that we start a
> >> >>>> > checkpoint before a last one is confirmed completed.
> >> >>>> >
> >> >>>> > Unless there is some weirdness in the sources and some sources
> >> start
> >> >>>> > chk1 first and some other ones start chk2 first?
> >> >>>> >
> >> >>>> > @Piotrek, do you think this is a problem?
> >> >>>> >
> >> >>>> >
> >> >>>> > For the Committer interface, I am wondering if we should split
> the
> >> >>>> > single
> >> >>>> > commit method into separate "*collect"* and "*commit"* methods?
> >> This
> >> >>>> > way,
> >> >>>> > it can handle both single and multiple CommT objects.
> >> >>>> >
> >> >>>> > I think we can't do this. If the sink only needs a regular
> >> Commiter,
> >> >>>> > we can perform the commits in parallel, possibly on different
> >> >>>> > machines. Only when the sink needs a GlobalCommitter would we
> need
> >> to
> >> >>>> > ship all commits to a single process and perform the commit
> there.
> >> If
> >> >>>> > both methods were unified in one interface we couldn't make the
> >> >>>> > decision of were to commit in the framework code.
> >> >>>> >
> >> >>>> >
> >> >>>> > For Iceberg, writers don't need any state. But the
> GlobalCommitter
> >> >>>> > needs to
> >> >>>> > checkpoint StateT. For the committer, CommT is "DataFile". Since
> a
> >> >>>> > single
> >> >>>> > committer can collect thousands (or more) data files in one
> >> checkpoint
> >> >>>> > cycle, as an optimization we checkpoint a single "ManifestFile"
> >> (for
> >> >>>> the
> >> >>>> > collected thousands data files) as StateT. This allows us to
> absorb
> >> >>>> > extended commit outages without losing written/uploaded data
> >> files, as
> >> >>>> > operator state size is as small as one manifest file per
> checkpoint
> >> >>>> > cycle
> >> >>>> >
> >> >>>> > You could have a point here. Is the code for this available in
> >> >>>> > open-source? I was checking out
> >> >>>> >
> >> >>>> >
> >> >>>> >
> >> >>>>
> >>
> https://github.com/apache/iceberg/blob/master/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergFilesCommitter.java
> >> >>>> >
> >> >>>> > and didn't find the ManifestFile optimization there.
> >> >>>> >
> >> >>>> > Best,
> >> >>>> > Aljoscha
> >> >>>> >
> >> >>>> >
> >> >>>> >
> >> >>>>
> >> >>>
> >>
> >
>
Reply | Threaded
Open this post in threaded view
|

Re: [DISCUSS] FLIP-143: Unified Sink API

Piotr Nowojski-5
Hi Guowei,

Thanks for the explanation. Now I get your point.

Basically any action that would make sink unavailable, would also cause it
to block on snapshotting the state (in option 1. with flushing). I agree
that lack of availability is much less of an issue than I have thought
before.

For option 2., the difference would be much larger, in cases when the sink
is heavily back pressured and writing buffered records on to the state is
must faster compared to flushing them. Without availability, we would need
to wait for at least a single record to be written (which can be a long
time), and if sink is batching writes, it could block for the whole batch.
While with the availability, we could snapshot the state immediately.

Currently we don't have option 2. sinks, right? But isn't option 2. the
write ahead log approach?

There are some other minor benefits for availability:
a) metrics/backpressure detection
b) the more non blocking API, the more future proof it is. For example it
could allow us to scale up the number of tasks that we can execute on a
single TM (having a pool of worker threads instead of one thread per task)
c) we could move more critical things into the task thread and extend
single thread/mailbox model

For c), we would probably want to have asynchronous flushing (snapshotting).

I wonder if non blocking WAL sinks and metrics/backpressure detection are
good enough justifications for the availability?


But the whole discussion made me realise that it might be nice to have some
unified way to configure the amount of buffered data for the new sinks.
Either passing via some parameter, or via some config (`ExecutionConfig`?)
an optional request:
"hey sink, user configured that he wants to have at most 100KB of buffered
data. If you can, please respect this configuration".

What do you think?

Piotrek

czw., 17 wrz 2020 o 06:28 Guowei Ma <[hidden email]> napisał(a):

> Hi, Steven
>
> I am not particularly sure whether to provide id in GlobalCommit.
>
> But my understanding is: if the committer function is idempotent, the
> framework can guarantee exactly once semantics in batch/stream execution
> mode. But I think maybe the idempotence should be guaranteed by the sink
> developer, not on the basic API.
>
> We could  provide an id in GlobalCommit. But the following question would
> be that: do we need to provide an id for a normal committable? I would like
> to say that I prefer to make the committer single responsibility.
>
> I think maybe we could have an answer when the first nonconsensual
> question is resolved.
>
> Aboving is just my personal opinion. I think this is still an open
> question.
>
> Thank you again for your valuable and thoughtful response.
>
> Best,
> Guowei
>
>
> On Thu, Sep 17, 2020 at 10:53 AM Steven Wu <[hidden email]> wrote:
>
>> Guowei, thanks a lot for the summary. Here are a couple more questions
>> that
>> need more clarification for the GlobalCommitter case.
>>
>> * framework provides some sort of unique id per GlobalCommT (e.g. nonce or
>> some sort of transaction id)
>> * commit failure handling. Should we roll over to the next cycle? if so,
>> we
>> may need commit(List<GlobalCommT> )
>>
>> On Wed, Sep 16, 2020 at 2:11 AM Piotr Nowojski <[hidden email]>
>> wrote:
>>
>> > Hey
>> >
>> > Thanks Dawid for bringing up my suggestion :)
>> >
>> > > I'm not so sure about this, the sinks I'm aware of would not be able
>> to
>> > > implement this method: Kafka doesn't have this, I didn't see it in the
>> > > Iceberg interfaces, and HDFS/S3 also don't have it.
>> >
>> > Aljoscha, as I wrote, FlinkKafkaProducer is actually one for which we
>> > could do some magic. At the very least we could use
>> > `FlinkKafkaProducer#pendingRecords` to make the sink unavailable when
>> some
>> > threshold is exceeded. Alternatively, maybe we could hook in to the
>> > KafkaProducer's buffer state [1]:
>> >
>> > > The buffer.memory controls the total amount of memory available to the
>> > producer for buffering.
>> > > If records are sent faster than they can be transmitted to the server
>> > then this buffer space will be exhausted.
>> > > When the buffer space is exhausted additional send calls will block.
>> >
>> > As far as I can see, Kafka is exposing the `buffer-available-bytes`
>> > metric, which we might use instead of `pendingRecords`. Heck, we are
>> > already hacking KafkaProducer with reflections, we could access
>> > `org.apache.kafka.clients.producer.KafkaProducer#accumulator` field to
>> > call  `accumulator.bufferPoolAvailableMemory()` method, if metric would
>> be
>> > to expensive to check per every record.
>> >
>> > Furthermore, I'm pretty sure other sinks (maybe not all) provide similar
>> > features. If we are desperate, we could always contribute something to
>> > those systems to make them expose the internal buffer's state.
>> >
>> > If we are really desperate, we could provide a generic records handover
>> > wrapper sink, that would have a buffer of N (5? 10? ) records and would
>> be
>> > handing over those records to the blocking sink running in another
>> thread.
>> > If the buffer is full, the sink would be unavailable.
>> >
>> > Guowei
>> > > Does the sink's snapshot return immediately when the sink's status is
>> > unavailable?
>> >
>> > State snapshot call is generally speaking non blocking already, so it
>> > should not be an issue. If it's blocking and if it will be solving some
>> > problem, we could later decide in the runtime code to not execute
>> snapshot
>> > calls if a sink is unavailable. Think about isAvailable more like a hint
>> > from the operator to the runtime, which we can use to make better
>> > decisions. Also take a look at the FLIP-27 sources (`SourceReader`),
>> where
>> > there already is `isAvailable()` method. It would be best if new sinks
>> > would just duplicate the same contract.
>> >
>> > > For me I want to know is what specific sink will benefit from this
>> > feature
>> >
>> > It's not the sinks that would benefit from this, but other parts of the
>> > system. Currently task thread is blocked on backpressured Sink, it's
>> > blocking some things from happening (checkpointing, closing, ...). If we
>> > make sinks non blocking (as is the network stack in the most part and as
>> > are the FLIP-27 sources), we will be able to snapshot state of the
>> operator
>> > immediately. For example, change from blocking to non blocking sources
>> was
>> > speeding up unaligned checkpoints from ~30seconds down to ~5seconds in
>> > our benchmarks, but the difference can be even more profound (hours
>> instead
>> > of seconds/minutes as reported by some users).
>> >
>> > Piotrek
>> >
>> > [1]
>> >
>> https://kafka.apache.org/10/javadoc/org/apache/kafka/clients/producer/KafkaProducer.html
>> >
>> > śr., 16 wrz 2020 o 06:29 Guowei Ma <[hidden email]> napisał(a):
>> >
>> >> Hi,all
>> >>
>> >> Thanks for all your valuable options and ideas.Currently there are many
>> >> topics in the mail. I try to summarize what is consensus and what is
>> not.
>> >> Correct me if I am wrong.
>> >>
>> >> ## Consensus
>> >>
>> >> 1. The motivation of the unified sink API is to decouple the sink
>> >> implementation from the different runtime execution mode.
>> >> 2. The initial scope of the unified sink API only covers the file
>> system
>> >> type, which supports the real transactions. The FLIP focuses more on
>> the
>> >> semantics the new sink api should support.
>> >> 3. We prefer the first alternative API, which could give the framework
>> a
>> >> greater opportunity to optimize.
>> >> 4. The `Writer` needs to add a method `prepareCommit`, which would be
>> >> called from `prepareSnapshotPreBarrier`. And remove the `Flush` method.
>> >> 5. The FLIP could move the `Snapshot & Drain` section in order to be
>> more
>> >> focused.
>> >>
>> >> ## Not Consensus
>> >>
>> >> 1. What should the “Unified Sink API” support/cover? The API can
>> >> “unified”(decoupe) the commit operation in the term of supporting
>> exactly
>> >> once semantics. However, even if we narrow down the initial supported
>> >> system to the file system there would be different topology
>> requirements.
>> >> These requirements come from performance optimization
>> >> (IceBergSink/MergeHiveSink) or functionality(i.e. whether a bucket is
>> >> “finished”).  Should the unified sink API support these requirements?
>> >> 2. The API does not expose the checkpoint-id because the batch
>> execution
>> >> mode does not have the normal checkpoint. But there still some
>> >> implementations depend on this.(IceBergSink uses this to do some
>> dedupe).
>> >> I think how to support this requirement depends on the first open
>> >> question.
>> >> 3. Whether the `Writer` supports async functionality or not. Currently
>> I
>> >> do
>> >> not know which sink could benefit from it. Maybe it is just my own
>> >> problem.
>> >>
>> >> Best,
>> >> Guowei
>> >>
>> >>
>> >> On Wed, Sep 16, 2020 at 12:02 PM Guowei Ma <[hidden email]>
>> wrote:
>> >>
>> >> >
>> >> > Hi, Steven
>> >> > Thanks you for your thoughtful ideas and concerns.
>> >> >
>> >> > >>I still like the concept of grouping data files per checkpoint for
>> >> > streaming mode. it is cleaner and probably easier to manage and deal
>> >> with
>> >> > commit failures. Plus, it >>can reduce dupes for the at least once
>> >> > >>mode.  I understand checkpoint is not an option for batch
>> execution.
>> >> We
>> >> > don't have to expose the checkpointId in API, as >>long as  the
>> internal
>> >> > bookkeeping groups data files by checkpoints for streaming >>mode.
>> >> >
>> >> > I think this problem(How to dedupe the combined committed data) also
>> >> > depends on where to place the agg/combine logic .
>> >> >
>> >> > 1. If the agg/combine takes place in the “commit” maybe we need to
>> >> figure
>> >> > out how to give the aggregated committable a unique and
>> auto-increment
>> >> id
>> >> > in the committer.
>> >> > 2. If the agg/combine takes place in a separate operator maybe sink
>> >> > developer could maintain the id itself by using the state.
>> >> >
>> >> > I think this problem is also decided by what the topology pattern the
>> >> sink
>> >> > API should support. Actually there are already many other topology
>> >> > requirements. :)
>> >> >
>> >> > Best,
>> >> > Guowei
>> >> >
>> >> >
>> >> > On Wed, Sep 16, 2020 at 7:46 AM Steven Wu <[hidden email]>
>> wrote:
>> >> >
>> >> >> > AFAIK the committer would not see the file-1-2 when ck1 happens in
>> >> the
>> >> >> ExactlyOnce mode.
>> >> >>
>> >> >> @Guowei Ma <[hidden email]> I think you are right for exactly
>> >> once
>> >> >> checkpoint semantics. what about "at least once"? I guess we can
>> argue
>> >> that
>> >> >> it is fine to commit file-1-2 for at least once mode.
>> >> >>
>> >> >> I still like the concept of grouping data files per checkpoint for
>> >> >> streaming mode. it is cleaner and probably easier to manage and deal
>> >> with
>> >> >> commit failures. Plus, it can reduce dupes for the at least once
>> >> mode.  I
>> >> >> understand checkpoint is not an option for batch execution. We don't
>> >> have
>> >> >> to expose the checkpointId in API, as long as  the internal
>> bookkeeping
>> >> >> groups data files by checkpoints for streaming mode.
>> >> >>
>> >> >>
>> >> >> On Tue, Sep 15, 2020 at 6:58 AM Steven Wu <[hidden email]>
>> >> wrote:
>> >> >>
>> >> >>> > images don't make it through to the mailing lists. You would
>> need to
>> >> >>> host the file somewhere and send a link.
>> >> >>>
>> >> >>> Sorry about that. Here is the sample DAG in google drawings.
>> >> >>>
>> >> >>>
>> >>
>> https://docs.google.com/drawings/d/1-P8F2jF9RG9HHTtAfWEBRuU_2uV9aDTdqEt5dLs2JPk/edit?usp=sharing
>> >> >>>
>> >> >>>
>> >> >>> On Tue, Sep 15, 2020 at 4:58 AM Guowei Ma <[hidden email]>
>> >> wrote:
>> >> >>>
>> >> >>>> Hi, Dawid
>> >> >>>>
>> >> >>>> >>I still find the merging case the most confusing. I don't
>> >> necessarily
>> >> >>>> understand why do you need the "SingleFileCommit" step in this
>> >> scenario.
>> >> >>>> The way I
>> >> >>>> >> understand "commit" operation is that it makes some
>> data/artifacts
>> >> >>>> visible to the external system, thus it should be immutable from a
>> >> >>>> point of
>> >> >>>> view of a single >>process. Having an additional step in the same
>> >> >>>> process
>> >> >>>> that works on committed data contradicts with those assumptions. I
>> >> >>>> might be
>> >> >>>> missing something though. >> Could you elaborate >why can't it be
>> >> >>>> something
>> >> >>>> like FileWriter -> FileMergeWriter -> Committer (either global or
>> >> >>>> non-global)? Again it might be just me not getting the example.
>> >> >>>>
>> >> >>>> I think you are right. The topology
>> >> >>>> "FileWriter->FileMergeWriter->Committer" could meet the merge
>> >> >>>> requirement.
>> >> >>>> The topology "FileWriter-> SingleFileCommitter -> FileMergeWriter
>> ->
>> >> >>>> GlobalCommitter" reuses some code of the StreamingFileSink(For
>> >> example
>> >> >>>> rolling policy) so it has the "SingleFileCommitter" in the
>> topology.
>> >> In
>> >> >>>> general I want to use the case to show that there are different
>> >> >>>> topologies
>> >> >>>> according to the requirements.
>> >> >>>>
>> >> >>>> BTW: IIRC, @Jingsong Lee <[hidden email]> telled me
>> that
>> >> >>>> the
>> >> >>>> actual topology of merged supported HiveSink is more complicated
>> than
>> >> >>>> that.
>> >> >>>>
>> >> >>>>
>> >> >>>> >> I've just briefly skimmed over the proposed interfaces. I would
>> >> >>>> suggest
>> >> >>>> one
>> >> >>>> >> addition to the Writer interface (as I understand this is the
>> >> runtime
>> >> >>>> >> interface in this proposal?): add some availability method, to
>> >> >>>> avoid, if
>> >> >>>> >> possible, blocking calls on the sink. We already have similar
>> >> >>>> >> availability methods in the new sources [1] and in various
>> places
>> >> in
>> >> >>>> the
>> >> >>>> >> network stack [2].
>> >> >>>> >> BTW Let's not forget about Piotr's comment. I think we could
>> add
>> >> the
>> >> >>>> isAvailable or similar method to the Writer interface in the FLIP.
>> >> >>>>
>> >> >>>> Thanks @Dawid Wysakowicz <[hidden email]>  for your
>> >> reminder.
>> >> >>>> There
>> >> >>>> are two many issues at the same time.
>> >> >>>>
>> >> >>>> In addition to what Ajjoscha said : there is very little system
>> >> support
>> >> >>>> it.   Another thing I worry about is that: Does the sink's
>> snapshot
>> >> >>>> return
>> >> >>>> immediately when the sink's status is unavailable? Maybe we could
>> do
>> >> it
>> >> >>>> by
>> >> >>>> dedupe some element in the state but I think it might be too
>> >> >>>> complicated.
>> >> >>>> For me I want to know is what specific sink will benefit from this
>> >> >>>> feature.  @piotr <[hidden email]>  Please correct me if  I
>> >> >>>> misunderstand you. thanks.
>> >> >>>>
>> >> >>>> Best,
>> >> >>>> Guowei
>> >> >>>>
>> >> >>>>
>> >> >>>> On Tue, Sep 15, 2020 at 3:55 PM Dawid Wysakowicz <
>> >> >>>> [hidden email]>
>> >> >>>> wrote:
>> >> >>>>
>> >> >>>> > What I understand is that HiveSink's implementation might need
>> the
>> >> >>>> local
>> >> >>>> > committer(FileCommitter) because the file rename is needed.
>> >> >>>> > But the iceberg only needs to write the manifest file.  Would
>> you
>> >> >>>> like to
>> >> >>>> > enlighten me why the Iceberg needs the local committer?
>> >> >>>> > Thanks
>> >> >>>> >
>> >> >>>> > Sorry if I caused a confusion here. I am not saying the Iceberg
>> >> sink
>> >> >>>> needs
>> >> >>>> > a local committer. What I had in mind is that prior to the
>> Iceberg
>> >> >>>> example
>> >> >>>> > I did not see a need for a "GlobalCommitter" in the streaming
>> >> case. I
>> >> >>>> > thought it is always enough to have the "normal" committer in
>> that
>> >> >>>> case.
>> >> >>>> > Now I understand that this differentiation is not really about
>> >> logical
>> >> >>>> > separation. It is not really about the granularity with which we
>> >> >>>> commit,
>> >> >>>> > i.e. answering the "WHAT" question. It is really about the
>> >> >>>> performance and
>> >> >>>> > that in the end we will have a single "transaction", so it is
>> about
>> >> >>>> > answering the question "HOW".
>> >> >>>> >
>> >> >>>> >
>> >> >>>> >    -
>> >> >>>> >
>> >> >>>> >    Commit a directory with merged files(Some user want to merge
>> the
>> >> >>>> files
>> >> >>>> >    in a directory before committing the directory to Hive meta
>> >> store)
>> >> >>>> >
>> >> >>>> >
>> >> >>>> >    1.
>> >> >>>> >
>> >> >>>> >    FileWriter -> SingleFileCommit -> FileMergeWriter  ->
>> >> >>>> GlobalCommitter
>> >> >>>> >
>> >> >>>> > I still find the merging case the most confusing. I don't
>> >> necessarily
>> >> >>>> > understand why do you need the "SingleFileCommit" step in this
>> >> >>>> scenario.
>> >> >>>> > The way I understand "commit" operation is that it makes some
>> >> >>>> > data/artifacts visible to the external system, thus it should be
>> >> >>>> immutable
>> >> >>>> > from a point of view of a single process. Having an additional
>> step
>> >> >>>> in the
>> >> >>>> > same process that works on committed data contradicts with those
>> >> >>>> > assumptions. I might be missing something though. Could you
>> >> elaborate
>> >> >>>> why
>> >> >>>> > can't it be something like FileWriter -> FileMergeWriter ->
>> >> Committer
>> >> >>>> > (either global or non-global)? Again it might be just me not
>> >> getting
>> >> >>>> the
>> >> >>>> > example.
>> >> >>>> >
>> >> >>>> > I've just briefly skimmed over the proposed interfaces. I would
>> >> >>>> suggest one
>> >> >>>> > addition to the Writer interface (as I understand this is the
>> >> runtime
>> >> >>>> > interface in this proposal?): add some availability method, to
>> >> avoid,
>> >> >>>> if
>> >> >>>> > possible, blocking calls on the sink. We already have similar
>> >> >>>> > availability methods in the new sources [1] and in various
>> places
>> >> in
>> >> >>>> the
>> >> >>>> > network stack [2].
>> >> >>>> >
>> >> >>>> > BTW Let's not forget about Piotr's comment. I think we could add
>> >> the
>> >> >>>> > isAvailable or similar method to the Writer interface in the
>> FLIP.
>> >> >>>> >
>> >> >>>> > Best,
>> >> >>>> >
>> >> >>>> > Dawid
>> >> >>>> > On 15/09/2020 08:06, Guowei Ma wrote:
>> >> >>>> >
>> >> >>>> > I would think that we only need flush() and the semantics are
>> that
>> >> it
>> >> >>>> > prepares for a commit, so on a physical level it would be called
>> >> from
>> >> >>>> > "prepareSnapshotPreBarrier". Now that I'm thinking about it
>> more I
>> >> >>>> > think flush() should be renamed to something like
>> >> "prepareCommit()".
>> >> >>>> >
>> >> >>>> > Generally speaking it is a good point that emitting the
>> >> committables
>> >> >>>> > should happen before emitting the checkpoint barrier downstream.
>> >> >>>> > However, if I remember offline discussions well, the idea behind
>> >> >>>> > Writer#flush and Writer#snapshotState was to differentiate
>> commit
>> >> on
>> >> >>>> > checkpoint vs final checkpoint at the end of the job. Both of
>> these
>> >> >>>> > methods could emit committables, but the flush should not leave
>> >> any in
>> >> >>>> > progress state (e.g. in case of file sink in STREAM mode, in
>> >> >>>> > snapshotState it could leave some open files that would be
>> >> committed
>> >> >>>> in
>> >> >>>> > a subsequent cycle, however flush should close all files). The
>> >> >>>> > snapshotState as it is now can not be called in
>> >> >>>> > prepareSnapshotPreBarrier as it can store some state, which
>> should
>> >> >>>> > happen in Operator#snapshotState as otherwise it would always be
>> >> >>>> > synchronous. Therefore I think we would need sth like:
>> >> >>>> >
>> >> >>>> > void prepareCommit(boolean flush, WriterOutput<CommT> output);
>> >> >>>> >
>> >> >>>> > ver 1:
>> >> >>>> >
>> >> >>>> > List<StateT> snapshotState();
>> >> >>>> >
>> >> >>>> > ver 2:
>> >> >>>> >
>> >> >>>> > void snapshotState(); // not sure if we need that method at all
>> in
>> >> >>>> option
>> >> >>>> >
>> >> >>>> > 2
>> >> >>>> >
>> >> >>>> > I second Dawid's proposal. This is a valid scenario. And
>> version2
>> >> >>>> does not
>> >> >>>> > need the snapshotState() any more.
>> >> >>>> >
>> >> >>>> >
>> >> >>>> > The Committer is as described in the FLIP, it's basically a
>> >> function
>> >> >>>> > "void commit(Committable)". The GobalCommitter would be a
>> function
>> >> >>>> "void
>> >> >>>> > commit(List<Committable>)". The former would be used by an S3
>> sink
>> >> >>>> where
>> >> >>>> > we can individually commit files to S3, a committable would be
>> the
>> >> >>>> list
>> >> >>>> > of part uploads that will form the final file and the commit
>> >> operation
>> >> >>>> > creates the metadata in S3. The latter would be used by
>> something
>> >> like
>> >> >>>> > Iceberg where the Committer needs a global view of all the
>> commits
>> >> to
>> >> >>>> be
>> >> >>>> > efficient and not overwhelm the system.
>> >> >>>> >
>> >> >>>> > I don't know yet if sinks would only implement on type of commit
>> >> >>>> > function or potentially both at the same time, and maybe Commit
>> can
>> >> >>>> > return some CommitResult that gets shipped to the GlobalCommit
>> >> >>>> function.
>> >> >>>> > I must admit it I did not get the need for Local/Normal + Global
>> >> >>>> > committer at first. The Iceberg example helped a lot. I think it
>> >> >>>> makes a
>> >> >>>> > lot of sense.
>> >> >>>> >
>> >> >>>> > @Dawid
>> >> >>>> > What I understand is that HiveSink's implementation might need
>> the
>> >> >>>> local
>> >> >>>> > committer(FileCommitter) because the file rename is needed.
>> >> >>>> > But the iceberg only needs to write the manifest file.  Would
>> you
>> >> >>>> like to
>> >> >>>> > enlighten me why the Iceberg needs the local committer?
>> >> >>>> > Thanks
>> >> >>>> >
>> >> >>>> > Best,
>> >> >>>> > Guowei
>> >> >>>> >
>> >> >>>> >
>> >> >>>> > On Mon, Sep 14, 2020 at 11:19 PM Dawid Wysakowicz <
>> >> >>>> [hidden email]> <[hidden email]>
>> >> >>>> > wrote:
>> >> >>>> >
>> >> >>>> >
>> >> >>>> > Hi all,
>> >> >>>> >
>> >> >>>> >
>> >> >>>> > I would think that we only need flush() and the semantics are
>> that
>> >> it
>> >> >>>> > prepares for a commit, so on a physical level it would be called
>> >> from
>> >> >>>> > "prepareSnapshotPreBarrier". Now that I'm thinking about it
>> more I
>> >> >>>> > think flush() should be renamed to something like
>> >> "prepareCommit()".
>> >> >>>> >
>> >> >>>> > Generally speaking it is a good point that emitting the
>> >> committables
>> >> >>>> > should happen before emitting the checkpoint barrier downstream.
>> >> >>>> > However, if I remember offline discussions well, the idea behind
>> >> >>>> > Writer#flush and Writer#snapshotState was to differentiate
>> commit
>> >> on
>> >> >>>> > checkpoint vs final checkpoint at the end of the job. Both of
>> these
>> >> >>>> > methods could emit committables, but the flush should not leave
>> >> any in
>> >> >>>> > progress state (e.g. in case of file sink in STREAM mode, in
>> >> >>>> > snapshotState it could leave some open files that would be
>> >> committed
>> >> >>>> in
>> >> >>>> > a subsequent cycle, however flush should close all files). The
>> >> >>>> > snapshotState as it is now can not be called in
>> >> >>>> > prepareSnapshotPreBarrier as it can store some state, which
>> should
>> >> >>>> > happen in Operator#snapshotState as otherwise it would always be
>> >> >>>> > synchronous. Therefore I think we would need sth like:
>> >> >>>> >
>> >> >>>> > void prepareCommit(boolean flush, WriterOutput<CommT> output);
>> >> >>>> >
>> >> >>>> > ver 1:
>> >> >>>> >
>> >> >>>> > List<StateT> snapshotState();
>> >> >>>> >
>> >> >>>> > ver 2:
>> >> >>>> >
>> >> >>>> > void snapshotState(); // not sure if we need that method at all
>> in
>> >> >>>> option 2
>> >> >>>> >
>> >> >>>> >
>> >> >>>> > The Committer is as described in the FLIP, it's basically a
>> >> function
>> >> >>>> > "void commit(Committable)". The GobalCommitter would be a
>> function
>> >> >>>> "void
>> >> >>>> > commit(List<Committable>)". The former would be used by an S3
>> sink
>> >> >>>> where
>> >> >>>> > we can individually commit files to S3, a committable would be
>> the
>> >> >>>> list
>> >> >>>> > of part uploads that will form the final file and the commit
>> >> operation
>> >> >>>> > creates the metadata in S3. The latter would be used by
>> something
>> >> like
>> >> >>>> > Iceberg where the Committer needs a global view of all the
>> commits
>> >> to
>> >> >>>> be
>> >> >>>> > efficient and not overwhelm the system.
>> >> >>>> >
>> >> >>>> > I don't know yet if sinks would only implement on type of commit
>> >> >>>> > function or potentially both at the same time, and maybe Commit
>> can
>> >> >>>> > return some CommitResult that gets shipped to the GlobalCommit
>> >> >>>> function.
>> >> >>>> >
>> >> >>>> > I must admit it I did not get the need for Local/Normal + Global
>> >> >>>> > committer at first. The Iceberg example helped a lot. I think it
>> >> >>>> makes a
>> >> >>>> > lot of sense.
>> >> >>>> >
>> >> >>>> >
>> >> >>>> > For Iceberg, writers don't need any state. But the
>> GlobalCommitter
>> >> >>>> > needs to
>> >> >>>> > checkpoint StateT. For the committer, CommT is "DataFile".
>> Since a
>> >> >>>> single
>> >> >>>> > committer can collect thousands (or more) data files in one
>> >> checkpoint
>> >> >>>> > cycle, as an optimization we checkpoint a single "ManifestFile"
>> >> (for
>> >> >>>> the
>> >> >>>> > collected thousands data files) as StateT. This allows us to
>> absorb
>> >> >>>> > extended commit outages without losing written/uploaded data
>> >> files, as
>> >> >>>> > operator state size is as small as one manifest file per
>> checkpoint
>> >> >>>> cycle
>> >> >>>> > [2].
>> >> >>>> > ------------------
>> >> >>>> > StateT snapshotState(SnapshotContext context) throws Exception;
>> >> >>>> >
>> >> >>>> > That means we also need the restoreCommitter API in the Sink
>> >> interface
>> >> >>>> > ---------------
>> >> >>>> > Committer<CommT, StateT> restoreCommitter(InitContext context,
>> >> StateT
>> >> >>>> > state);
>> >> >>>> >
>> >> >>>> > I think this might be a valid case. Not sure though if I would
>> go
>> >> >>>> with a
>> >> >>>> > "state" there. Having a state in a committer would imply we
>> need a
>> >> >>>> > collect method as well. So far we needed a single method
>> >> commit(...)
>> >> >>>> and
>> >> >>>> > the bookkeeping of the committables could be handled by the
>> >> >>>> framework. I
>> >> >>>> > think something like an optional combiner in the GlobalCommitter
>> >> would
>> >> >>>> > be enough. What do you think?
>> >> >>>> >
>> >> >>>> > GlobalCommitter<CommT, GlobalCommT> {
>> >> >>>> >
>> >> >>>> >     void commit(GlobalCommT globalCommittables);
>> >> >>>> >
>> >> >>>> >     GlobalCommT combine(List<CommT> committables);
>> >> >>>> >
>> >> >>>> > }
>> >> >>>> >
>> >> >>>> > A different problem that I see here is how do we handle commit
>> >> >>>> failures.
>> >> >>>> > Should the committables (both normal and global be included in
>> the
>> >> >>>> next
>> >> >>>> > cycle, shall we retry it, ...) I think it would be worth laying
>> it
>> >> out
>> >> >>>> > in the FLIP.
>> >> >>>> >
>> >> >>>> > @Aljoscha I think you can find the code Steven was referring in
>> >> here:
>> >> >>>> >
>> >> >>>>
>> >>
>> https://github.com/Netflix-Skunkworks/nfflink-connector-iceberg/blob/master/nfflink-connector-iceberg/src/main/java/com/netflix/spaas/nfflink/connector/iceberg/sink/IcebergCommitter.java
>> >> >>>> >
>> >> >>>> > Best,
>> >> >>>> >
>> >> >>>> > Dawid
>> >> >>>> >
>> >> >>>> > On 14/09/2020 15:19, Aljoscha Krettek wrote:
>> >> >>>> >
>> >> >>>> > On 14.09.20 01:23, Steven Wu wrote:
>> >> >>>> >
>> >> >>>> > ## Writer interface
>> >> >>>> >
>> >> >>>> > For the Writer interface, should we add "*prepareSnapshot"*
>> before
>> >> the
>> >> >>>> > checkpoint barrier emitted downstream?  IcebergWriter would need
>> >> it.
>> >> >>>> Or
>> >> >>>> > would the framework call "*flush*" before the barrier emitted
>> >> >>>> > downstream?
>> >> >>>> > that guarantee would achieve the same goal.
>> >> >>>> >
>> >> >>>> > I would think that we only need flush() and the semantics are
>> that
>> >> it
>> >> >>>> > prepares for a commit, so on a physical level it would be called
>> >> from
>> >> >>>> > "prepareSnapshotPreBarrier". Now that I'm thinking about it
>> more I
>> >> >>>> > think flush() should be renamed to something like
>> >> "prepareCommit()".
>> >> >>>> >
>> >> >>>> > @Guowei, what do you think about this?
>> >> >>>> >
>> >> >>>> >
>> >> >>>> > In [1], we discussed the reason for Writer to emit
>> (checkpointId,
>> >> >>>> CommT)
>> >> >>>> > tuple to the committer. The committer needs checkpointId to
>> >> separate
>> >> >>>> out
>> >> >>>> > data files for different checkpoints if concurrent checkpoints
>> are
>> >> >>>> > enabled.
>> >> >>>> >
>> >> >>>> > When can this happen? Even with concurrent checkpoints the
>> snapshot
>> >> >>>> > barriers would still cleanly segregate the input stream of an
>> >> operator
>> >> >>>> > into tranches that should manifest in only one checkpoint. With
>> >> >>>> > concurrent checkpoints, all that can happen is that we start a
>> >> >>>> > checkpoint before a last one is confirmed completed.
>> >> >>>> >
>> >> >>>> > Unless there is some weirdness in the sources and some sources
>> >> start
>> >> >>>> > chk1 first and some other ones start chk2 first?
>> >> >>>> >
>> >> >>>> > @Piotrek, do you think this is a problem?
>> >> >>>> >
>> >> >>>> >
>> >> >>>> > For the Committer interface, I am wondering if we should split
>> the
>> >> >>>> > single
>> >> >>>> > commit method into separate "*collect"* and "*commit"* methods?
>> >> This
>> >> >>>> > way,
>> >> >>>> > it can handle both single and multiple CommT objects.
>> >> >>>> >
>> >> >>>> > I think we can't do this. If the sink only needs a regular
>> >> Commiter,
>> >> >>>> > we can perform the commits in parallel, possibly on different
>> >> >>>> > machines. Only when the sink needs a GlobalCommitter would we
>> need
>> >> to
>> >> >>>> > ship all commits to a single process and perform the commit
>> there.
>> >> If
>> >> >>>> > both methods were unified in one interface we couldn't make the
>> >> >>>> > decision of were to commit in the framework code.
>> >> >>>> >
>> >> >>>> >
>> >> >>>> > For Iceberg, writers don't need any state. But the
>> GlobalCommitter
>> >> >>>> > needs to
>> >> >>>> > checkpoint StateT. For the committer, CommT is "DataFile".
>> Since a
>> >> >>>> > single
>> >> >>>> > committer can collect thousands (or more) data files in one
>> >> checkpoint
>> >> >>>> > cycle, as an optimization we checkpoint a single "ManifestFile"
>> >> (for
>> >> >>>> the
>> >> >>>> > collected thousands data files) as StateT. This allows us to
>> absorb
>> >> >>>> > extended commit outages without losing written/uploaded data
>> >> files, as
>> >> >>>> > operator state size is as small as one manifest file per
>> checkpoint
>> >> >>>> > cycle
>> >> >>>> >
>> >> >>>> > You could have a point here. Is the code for this available in
>> >> >>>> > open-source? I was checking out
>> >> >>>> >
>> >> >>>> >
>> >> >>>> >
>> >> >>>>
>> >>
>> https://github.com/apache/iceberg/blob/master/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergFilesCommitter.java
>> >> >>>> >
>> >> >>>> > and didn't find the ManifestFile optimization there.
>> >> >>>> >
>> >> >>>> > Best,
>> >> >>>> > Aljoscha
>> >> >>>> >
>> >> >>>> >
>> >> >>>> >
>> >> >>>>
>> >> >>>
>> >>
>> >
>>
>
Reply | Threaded
Open this post in threaded view
|

Re: [DISCUSS] FLIP-143: Unified Sink API

Aljoscha Krettek-2
In reply to this post by Guowei Ma
Thanks for the summary!

On 16.09.20 06:29, Guowei Ma wrote:

> ## Consensus
>
> 1. The motivation of the unified sink API is to decouple the sink
> implementation from the different runtime execution mode.
> 2. The initial scope of the unified sink API only covers the file system
> type, which supports the real transactions. The FLIP focuses more on the
> semantics the new sink api should support.
> 3. We prefer the first alternative API, which could give the framework a
> greater opportunity to optimize.
> 4. The `Writer` needs to add a method `prepareCommit`, which would be
> called from `prepareSnapshotPreBarrier`. And remove the `Flush` method.
> 5. The FLIP could move the `Snapshot & Drain` section in order to be more
> focused.

Agreed!

> ## Not Consensus
>
> 1. What should the “Unified Sink API” support/cover? The API can
> “unified”(decoupe) the commit operation in the term of supporting exactly
> once semantics. However, even if we narrow down the initial supported
> system to the file system there would be different topology requirements.
> These requirements come from performance optimization
> (IceBergSink/MergeHiveSink) or functionality(i.e. whether a bucket is
> “finished”).  Should the unified sink API support these requirements?

Yes, this is still tricky. What is the current state, would the
introduction of a "LocalCommit" and a "GlobalCommit" already solve both
the Iceberg and Hive cases? I believe Hive is the most tricky one here,
but if we introduce the "combine" method on GlobalCommit, that could
serve the same purpose as the "aggregation operation" on the individual
files, and we could even execute that "combine" in a distributed way.

To answer the more general question, I think we will offer a couple of
different commit strategies and sinks can implement 0 to n of them. What
is unified about the sink is that the same sink implementation will work
for both STREAMING and BATCH execution mode.

> 2. The API does not expose the checkpoint-id because the batch execution
> mode does not have the normal checkpoint. But there still some
> implementations depend on this.(IceBergSink uses this to do some dedupe).
> I think how to support this requirement depends on the first open question.

I think this can be solved by introducing a nonce, see more thorough
explanation below.

> 3. Whether the `Writer` supports async functionality or not. Currently I do
> not know which sink could benefit from it. Maybe it is just my own problem.

Here, I don't really know. We can introduce an "isAvailable()" method
and mostly ignore it for now and sinks can just always return true. Or,
as an alternative, we don't add the method now but can add it later with
a default implementation. Either way, we will probably not take
advantage of the "isAvailable()" now because that would require more
runtime changes.

On 17.09.20 06:28, Guowei Ma wrote:
> But my understanding is: if the committer function is idempotent, the
> framework can guarantee exactly once semantics in batch/stream execution
> mode. But I think maybe the idempotence should be guaranteed by the sink
> developer, not on the basic API.

I believe the problem here is that some sinks (including Iceberg) can
only be idempotent with a little help from the framework.

The process would be like this:

1. collect all committables, generate unique ID (nonce), store
committables and ID in fault tolerant storage

2. call commitGlobal(committables, nonce)

3. Iceberg checks if there is already a commit with the given nonce, if
not it will append a commit of the committables along with the nonce to
the log structure/meta store

The problem is that Iceberg cannot decide without some extra data
whether a set of committables has already been committed because the
commit basically just appends some information to the end of a log. And
we just just keep appending the same data if we didn't check the nonce.

We would have this same problem if we wanted to implement a
write-ahead-log Kafka sink where the "commit" would just take some
records from a file and append it to Kafka. Without looking at Kafka and
checking if you already committed the same records you don't know if you
already committed.




Reply | Threaded
Open this post in threaded view
|

Re: [DISCUSS] FLIP-143: Unified Sink API

Steven Wu
Guowei

Just to add to what Aljoscha said regarding the unique id. Iceberg sink
checkpoints the unique id into state during snapshot. It also inserts the
unique id into the Iceberg snapshot metadata during commit. When a job
restores the state after failure, it needs to know if the restored
transactions/commits were successful or not. It basically iterates through
the list of table snapshots from Iceberg and matches the unique ids with
what is stored in Iceberg snapshot metadata.

Thanks,
Steven


On Thu, Sep 17, 2020 at 7:40 AM Aljoscha Krettek <[hidden email]>
wrote:

> Thanks for the summary!
>
> On 16.09.20 06:29, Guowei Ma wrote:
> > ## Consensus
> >
> > 1. The motivation of the unified sink API is to decouple the sink
> > implementation from the different runtime execution mode.
> > 2. The initial scope of the unified sink API only covers the file system
> > type, which supports the real transactions. The FLIP focuses more on the
> > semantics the new sink api should support.
> > 3. We prefer the first alternative API, which could give the framework a
> > greater opportunity to optimize.
> > 4. The `Writer` needs to add a method `prepareCommit`, which would be
> > called from `prepareSnapshotPreBarrier`. And remove the `Flush` method.
> > 5. The FLIP could move the `Snapshot & Drain` section in order to be more
> > focused.
>
> Agreed!
>
> > ## Not Consensus
> >
> > 1. What should the “Unified Sink API” support/cover? The API can
> > “unified”(decoupe) the commit operation in the term of supporting exactly
> > once semantics. However, even if we narrow down the initial supported
> > system to the file system there would be different topology requirements.
> > These requirements come from performance optimization
> > (IceBergSink/MergeHiveSink) or functionality(i.e. whether a bucket is
> > “finished”).  Should the unified sink API support these requirements?
>
> Yes, this is still tricky. What is the current state, would the
> introduction of a "LocalCommit" and a "GlobalCommit" already solve both
> the Iceberg and Hive cases? I believe Hive is the most tricky one here,
> but if we introduce the "combine" method on GlobalCommit, that could
> serve the same purpose as the "aggregation operation" on the individual
> files, and we could even execute that "combine" in a distributed way.
>
> To answer the more general question, I think we will offer a couple of
> different commit strategies and sinks can implement 0 to n of them. What
> is unified about the sink is that the same sink implementation will work
> for both STREAMING and BATCH execution mode.
>
> > 2. The API does not expose the checkpoint-id because the batch execution
> > mode does not have the normal checkpoint. But there still some
> > implementations depend on this.(IceBergSink uses this to do some dedupe).
> > I think how to support this requirement depends on the first open
> question.
>
> I think this can be solved by introducing a nonce, see more thorough
> explanation below.
>
> > 3. Whether the `Writer` supports async functionality or not. Currently I
> do
> > not know which sink could benefit from it. Maybe it is just my own
> problem.
>
> Here, I don't really know. We can introduce an "isAvailable()" method
> and mostly ignore it for now and sinks can just always return true. Or,
> as an alternative, we don't add the method now but can add it later with
> a default implementation. Either way, we will probably not take
> advantage of the "isAvailable()" now because that would require more
> runtime changes.
>
> On 17.09.20 06:28, Guowei Ma wrote:
> > But my understanding is: if the committer function is idempotent, the
> > framework can guarantee exactly once semantics in batch/stream execution
> > mode. But I think maybe the idempotence should be guaranteed by the sink
> > developer, not on the basic API.
>
> I believe the problem here is that some sinks (including Iceberg) can
> only be idempotent with a little help from the framework.
>
> The process would be like this:
>
> 1. collect all committables, generate unique ID (nonce), store
> committables and ID in fault tolerant storage
>
> 2. call commitGlobal(committables, nonce)
>
> 3. Iceberg checks if there is already a commit with the given nonce, if
> not it will append a commit of the committables along with the nonce to
> the log structure/meta store
>
> The problem is that Iceberg cannot decide without some extra data
> whether a set of committables has already been committed because the
> commit basically just appends some information to the end of a log. And
> we just just keep appending the same data if we didn't check the nonce.
>
> We would have this same problem if we wanted to implement a
> write-ahead-log Kafka sink where the "commit" would just take some
> records from a file and append it to Kafka. Without looking at Kafka and
> checking if you already committed the same records you don't know if you
> already committed.
>
>
>
>
>
Reply | Threaded
Open this post in threaded view
|

Re: [DISCUSS] FLIP-143: Unified Sink API

Aljoscha Krettek-2
Steven,

we were also wondering if it is a strict requirement that "later"
updates to Iceberg subsume earlier updates. In the current version, you
only check whether checkpoint X made it to Iceberg and then discard all
committable state from Flink state for checkpoints smaller X.

If we go with a (somewhat random) nonce, this would not work. Instead
the sink would have to check for each set of committables seperately if
they had already been committed. Do you think this is feasible? During
normal operation this set would be very small, it would usually only be
the committables for the last checkpoint. Only when there is an outage
would multiple sets of committables pile up.

We were thinking to extend the GlobalCommitter interface to allow it to
report success or failure and then let the framework retry. I think this
is something that you would need for the Iceberg case. The signature
could be like this:

CommitStatus commitGlobally(List<Committable>, Nonce)

where CommitStatus could be an enum of SUCCESS, TERMINAL_FAILURE, and
RETRY.

Best,
Aljoscha
Reply | Threaded
Open this post in threaded view
|

Re: [DISCUSS] FLIP-143: Unified Sink API

Steven Wu
Aljoscha,

> Instead the sink would have to check for each set of committables
seperately if they had already been committed. Do you think this is
feasible?

Yes, that is how it works in our internal implementation [1]. We don't use
checkpointId. We generate a manifest file (GlobalCommT) to bundle all the
data files that the committer received in one checkpoint cycle. Then we
generate a unique manifest id for by hashing the location of the manifest
file. The manifest ids are stored in Iceberg snapshot metadata. Upon
restore, we check each of the restored manifest files against Iceberg table
snapshot metadata to determine if we should discard or keep the restored
manifest files. If a commit has multiple manifest files (e.g. accumulated
from previous failed commits), we store the comma-separated manifest ids in
Iceberg snapshot metadata.

> During normal operation this set would be very small, it would usually
only be the committables for the last checkpoint. Only when there is an
outage would multiple sets of committables pile up.

You are absolutely right here. Even if there are multiple sets of
committables, it is usually the last a few or dozen of snapshots we need to
check. Even with our current inefficient implementation of traversing all
table snapshots (in the scale of thousands) from oldest to latest, it only
took avg 60 ms and max 800 ms. so it is really not a concern for Iceberg.

> CommitStatus commitGlobally(List<Committable>, Nonce)

Just to clarify on the terminology here. Assuming here the Committable
meant the `GlobalCommT` (like ManifestFile in Iceberg) in
previous discussions, right? `CommT` means the Iceberg DataFile from writer
to committer.

This can work assuming we *don't have concurrent executions
of commitGlobally* even with concurrent checkpoints. Here is the scenario
regarding failure recovery I want to avoid.

Assuming checkpoints 1, 2, 3 all completed. Each checkpoint generates a
manifest file, manifest-1, 2, 3.
timeline
------------------------------------------------------------------------->
now
commitGlobally(manifest-1, nonce-1) started
         commitGlobally(manifest-2, nonce-2) started
                    commitGlobally(manifest-2, nonce-2) failed
                            commitGlobally(manifest-2 and manifest-3,
nonce-3) started
                                    commitGlobally(manifest-1, nonce-1)
failed
                                            commitGlobally(manifest-2 and
manifest-3, nonce-3) succeeded

Now the job failed and was restored from checkpoint 3, which contains
manifest file 1,2,3. We found nonce-3 was committed when checking Iceberg
table snapshot metadata. But in this case we won't be able to correctly
determine which manifest files were committed or not.

If it is possible to have concurrent executions of  commitGlobally, the
alternative is to generate the unique id/nonce per GlobalCommT. Then we can
check each individual GlobalCommT (ManifestFile) with Iceberg snapshot
metadata.

Thanks,
Steven

[1]
https://github.com/Netflix-Skunkworks/nfflink-connector-iceberg/blob/master/nfflink-connector-iceberg/src/main/java/com/netflix/spaas/nfflink/connector/iceberg/sink/IcebergCommitter.java#L569

On Fri, Sep 18, 2020 at 2:44 AM Aljoscha Krettek <[hidden email]>
wrote:

> Steven,
>
> we were also wondering if it is a strict requirement that "later"
> updates to Iceberg subsume earlier updates. In the current version, you
> only check whether checkpoint X made it to Iceberg and then discard all
> committable state from Flink state for checkpoints smaller X.
>
> If we go with a (somewhat random) nonce, this would not work. Instead
> the sink would have to check for each set of committables seperately if
> they had already been committed. Do you think this is feasible? During
> normal operation this set would be very small, it would usually only be
> the committables for the last checkpoint. Only when there is an outage
> would multiple sets of committables pile up.
>
> We were thinking to extend the GlobalCommitter interface to allow it to
> report success or failure and then let the framework retry. I think this
> is something that you would need for the Iceberg case. The signature
> could be like this:
>
> CommitStatus commitGlobally(List<Committable>, Nonce)
>
> where CommitStatus could be an enum of SUCCESS, TERMINAL_FAILURE, and
> RETRY.
>
> Best,
> Aljoscha
>
Reply | Threaded
Open this post in threaded view
|

Re: [DISCUSS] FLIP-143: Unified Sink API

Guowei Ma
Hi, all

>>Just to add to what Aljoscha said regarding the unique id. Iceberg sink
>>checkpoints the unique id into state during snapshot. It also inserts the
>>unique id into the Iceberg snapshot metadata during commit. When a job
>>restores the state after failure, it needs to know if the restored
>>transactions/commits were successful or not. It basically iterates through
>>the list of table snapshots from Iceberg and matches the unique ids with
>>what is stored in Iceberg snapshot metadata.

Thanks Steven for these detailed explanations. It makes me know the IceBerg
better. However, I prefer to let the developer produce id to dedupe. I
think this gives the developer more opportunity to optimize. You could see
the following for more details. Please correct me if I misunderstand you.

>> 3. Whether the `Writer` supports async functionality or not. Currently I
do
>> not know which sink could benefit from it. Maybe it is just my own
problem.

>> Here, I don't really know. We can introduce an "isAvailable()" method
>> and mostly ignore it for now and sinks can just always return true. Or,
>> as an alternative, we don't add the method now but can add it later with
>> a default implementation. Either way, we will probably not take
>> advantage of the "isAvailable()" now because that would require more
>> runtime changes.

From the @Pitor's explanation I could see the other benefit that might be
gained in the future. For example decoupling the task number and the thread
number. But I have to admit that introducing `isAvailable` might introduce
some complications in the runtime. You could see my alternative API option
in the following. I believe that we could support such an async sink writer
very easily in the future. What do you think?

>> Yes, this is still tricky. What is the current state, would the
>> introduction of a "LocalCommit" and a "GlobalCommit" already solve both
>> the Iceberg and Hive cases? I believe Hive is the most tricky one here,
>> but if we introduce the "combine" method on GlobalCommit, that could
>> serve the same purpose as the "aggregation operation" on the individual
>> files, and we could even execute that "combine" in a distributed way.
>>We assume that GlobalCommit is a Agg/Combiner?

I would share what possible problems that I am seeing currently and the
alternative options.

## IceBerg Sink

### Concern about generating nonce by framework.

If let the `GlobalCommitter` provide a random nonce for the `IceBergSink` I
think that it might not be efficient.  Because even if there are a very
small number of committables in the state you still need to iterate all the
iceberg snapshot files to check whether the committable is committed
already. Even if it is efficient for the IceBergSink it might not be the
case for other sinks.

If the framework generates auto-increment nonce instead, it might still not
be optimal for users. For example, users might want to use some business id
so that after failover they could query whether the commit is successful
after failover.

I think users could generate more efficient nonce such as an auto-increment
one. Therefore, it seems to provide more optimization chances if we let
users to generate the nonce.


### Alternative Option

public interface GlobalCommit<CommT, GlobalCommT> {
        // provide some runtime context such as attempt-id,job-id,task-id.
        void open(InitContext context);

        // This GlobalCommit would aggregate the committable to a
GlobalCommit before doing the commit operation.
        GlobalCommT combine(List<Committable> commitables)

        // This method would be called after committing all the
GlobalCommit producing in the previous session.
        void recoveredGlobalCommittables(List<GlobalCommit> globalCommits)

        // developer would guarantee the idempotency by himself
        void commit(GlobalCommit globalCommit);
}

User could guarantee the idenpointecy himself in a more efficient or
application specific way. If the user wants the `GlobalCommit` to be
executed in a distributed way, the user could use the runtime information
to generate the partial order id himself.(We could ignore the clean up
first)

Currently the sink might be looks like following:

Sink<IN, LC, LCO, GC> {
        Writer<IN, LC> createWriter();
        Optional<Committer<LC, LCO>> createCommitter();
        Optional<GlobalCommitter<LCO, GC>> createGlobalCommitter();
}

## Hive

The HiveSink needs to compute whether a directory is finished or not. But
HiveSink can not use the above `combine` method to decide whether a
directory is finished or not.

For example we assume that whether the directory is finished or not is
decided by the event time. There might be a topology that the source and
sink are forward. The event time might be different in different instances
of the `writer`. So the GlobalCommit’s combine can not produce a
GlobalCommT when the snapshot happens.

In addition to the above case we should also consider the unaligned
checkpoint. Because the watermark does not skip. So there might be the same
problem in the unaligned checkpoint.

### Option1:

public interface GlobalCommit<CommT, GlobalCommT, StateT, ShareT> {
        // provide some runtime context such as attempt-id,job-id,task-id,
maybe the event time;provide the restore state
        void open(InitContext context, StateT state);

        // This is for the HiveSink. When all the writer say that the the
bucket is finished it would return a GlobalCommitT
        Optional<GlobalCommT> combine(Committable commitables)

        // This is for IcebergSink. Producing a GlobalCommitT every
checkpoint.
        Optional<GlobalCommT> preCommit();

        // Maybe we need the shareState? After we decide the directory we
make more detailed consideration then. The id could be remembered here.
        StateT snapshotState();

        // developer would guarantee the idempotency by himself
        void commit(GlobalCommit globalCommit);
}

### Option2

Actually the `GlobalCommit` in the option1 mixes the `Writer` and
`Committer` together. So it is intuitive to decouple the two functions. For
support the hive we could prove a sink look like following

Sink<In, LC, LCO, LCG> {
        Writer<In, LC> createWriter();
        Optional<Committer<LC, LCO>> createCommitter(); // we need this to
change name.
        Optional<Writer<LCO, LCG>> createGlobalAgg();
        Optional<Committer<LCG, void>> createGlobalCommitter();
}

The pro of this method is that we use two basic concepts: `Committer` and
`Writer` to build a HiveSink.

### CompactHiveSink / MergeHiveSink

There are still other complicated cases, which are not satisfied by the
above option. Users often complain about writing out many small files,
which will affect file reading efficiency and the performance and stability
of the distributed file system. CompactHiveSink/MergeHiveSink hopes to
merge all files generated by this job in a single Checkpoint.

The CompactHiveSink/MergeHiveSink topology can simply describe this
topology as follows:

CompactSubTopology -> GlobalAgg -> GobalCommitter.

The CompactSubTopology would look like following:

TmpFileWriter -> CompactCoodinator -> CompactorFileWriter

Maybe the topology could be simpler but please keep in mind I just want to
show that there might be very complicated topology requirements for users.


A possible alternative option would be let the user build the topology
himself. But considering we have two execution modes we could only use
`Writer` and `Committer` to build the sink topology.

### Build Topology Option

Sink<IN, OUT> {
        Sink<In, Out> addWriter(Writer<In, Out> Writer); // Maybe a
WriterBuidler
        Sink<In, Out> addCommitter(Committer<In, Out> committer); // Maybe
we could make this return Void if we do not consider code reuse and
introduce the cleaner
}

## Summary
The requirements of sink might be different, maybe we could use two basic
bricks(Writer/Committer) to let the user build their own sink topology.
What do you guys think?

I know the name stuff might be trikky for now but I want to discuss these
things after we get the consus on the direction first.

Best,
Guowei


On Sat, Sep 19, 2020 at 12:15 AM Steven Wu <[hidden email]> wrote:

> Aljoscha,
>
> > Instead the sink would have to check for each set of committables
> seperately if they had already been committed. Do you think this is
> feasible?
>
> Yes, that is how it works in our internal implementation [1]. We don't use
> checkpointId. We generate a manifest file (GlobalCommT) to bundle all the
> data files that the committer received in one checkpoint cycle. Then we
> generate a unique manifest id for by hashing the location of the manifest
> file. The manifest ids are stored in Iceberg snapshot metadata. Upon
> restore, we check each of the restored manifest files against Iceberg table
> snapshot metadata to determine if we should discard or keep the restored
> manifest files. If a commit has multiple manifest files (e.g. accumulated
> from previous failed commits), we store the comma-separated manifest ids in
> Iceberg snapshot metadata.
>
> > During normal operation this set would be very small, it would usually
> only be the committables for the last checkpoint. Only when there is an
> outage would multiple sets of committables pile up.
>
> You are absolutely right here. Even if there are multiple sets of
> committables, it is usually the last a few or dozen of snapshots we need to
> check. Even with our current inefficient implementation of traversing all
> table snapshots (in the scale of thousands) from oldest to latest, it only
> took avg 60 ms and max 800 ms. so it is really not a concern for Iceberg.
>
> > CommitStatus commitGlobally(List<Committable>, Nonce)
>
> Just to clarify on the terminology here. Assuming here the Committable
> meant the `GlobalCommT` (like ManifestFile in Iceberg) in
> previous discussions, right? `CommT` means the Iceberg DataFile from writer
> to committer.
>
> This can work assuming we *don't have concurrent executions
> of commitGlobally* even with concurrent checkpoints. Here is the scenario
> regarding failure recovery I want to avoid.
>
> Assuming checkpoints 1, 2, 3 all completed. Each checkpoint generates a
> manifest file, manifest-1, 2, 3.
> timeline
> ------------------------------------------------------------------------->
> now
> commitGlobally(manifest-1, nonce-1) started
>          commitGlobally(manifest-2, nonce-2) started
>                     commitGlobally(manifest-2, nonce-2) failed
>                             commitGlobally(manifest-2 and manifest-3,
> nonce-3) started
>                                     commitGlobally(manifest-1, nonce-1)
> failed
>                                             commitGlobally(manifest-2 and
> manifest-3, nonce-3) succeeded
>
> Now the job failed and was restored from checkpoint 3, which contains
> manifest file 1,2,3. We found nonce-3 was committed when checking Iceberg
> table snapshot metadata. But in this case we won't be able to correctly
> determine which manifest files were committed or not.
>
> If it is possible to have concurrent executions of  commitGlobally, the
> alternative is to generate the unique id/nonce per GlobalCommT. Then we can
> check each individual GlobalCommT (ManifestFile) with Iceberg snapshot
> metadata.
>
> Thanks,
> Steven
>
> [1]
>
> https://github.com/Netflix-Skunkworks/nfflink-connector-iceberg/blob/master/nfflink-connector-iceberg/src/main/java/com/netflix/spaas/nfflink/connector/iceberg/sink/IcebergCommitter.java#L569
>
> On Fri, Sep 18, 2020 at 2:44 AM Aljoscha Krettek <[hidden email]>
> wrote:
>
> > Steven,
> >
> > we were also wondering if it is a strict requirement that "later"
> > updates to Iceberg subsume earlier updates. In the current version, you
> > only check whether checkpoint X made it to Iceberg and then discard all
> > committable state from Flink state for checkpoints smaller X.
> >
> > If we go with a (somewhat random) nonce, this would not work. Instead
> > the sink would have to check for each set of committables seperately if
> > they had already been committed. Do you think this is feasible? During
> > normal operation this set would be very small, it would usually only be
> > the committables for the last checkpoint. Only when there is an outage
> > would multiple sets of committables pile up.
> >
> > We were thinking to extend the GlobalCommitter interface to allow it to
> > report success or failure and then let the framework retry. I think this
> > is something that you would need for the Iceberg case. The signature
> > could be like this:
> >
> > CommitStatus commitGlobally(List<Committable>, Nonce)
> >
> > where CommitStatus could be an enum of SUCCESS, TERMINAL_FAILURE, and
> > RETRY.
> >
> > Best,
> > Aljoscha
> >
>
Reply | Threaded
Open this post in threaded view
|

Re: [DISCUSS] FLIP-143: Unified Sink API

Steven Wu
> I prefer to let the developer produce id to dedupe. I think this gives
the developer more opportunity to optimize.

Thinking about it again, I totally agree with Guowei on this. We don't
really need the framework to generate the unique id for Iceberg sink.
De-dup logic is totally internal to Iceberg sink and should be isolated
inside. My earlier question regarding "commitGlobally(List<GlobalCommT>)
can be concurrent or not" also becomes irrelevant, as long as the framework
handles the GlobalCommT list properly (even with concurrent calls).

Here are the things where framework can help

   1. run global committer in jobmanager (e.g. like sink coordinator)
   2. help with checkpointing, bookkeeping, commit failure handling,
   recovery


@Guowei Ma <[hidden email]> regarding the GlobalCommitter interface,
I have some clarifying questions.

> void recoveredGlobalCommittables(List<GlobalCommit> globalCommits)

   1. You meant GlobalCommit -> GlobalCommT, right?
   2. Is this called when restored from checkpoint/savepoint?
   3.  Iceberg sink needs to do a dup check here on which GlobalCommT were
   committed and which weren't. Should it return the filtered/de-duped list of
   GlobalCommT?
   4. Sink implementation can decide if it wants to commit immediately or
   just leave

> void commit(GlobalCommit globalCommit);

should this be "commit(List<GlobalCommT>)"?

Thanks,
Steven


On Sat, Sep 19, 2020 at 1:56 AM Guowei Ma <[hidden email]> wrote:

> Hi, all
>
> >>Just to add to what Aljoscha said regarding the unique id. Iceberg sink
> >>checkpoints the unique id into state during snapshot. It also inserts the
> >>unique id into the Iceberg snapshot metadata during commit. When a job
> >>restores the state after failure, it needs to know if the restored
> >>transactions/commits were successful or not. It basically iterates
> through
> >>the list of table snapshots from Iceberg and matches the unique ids with
> >>what is stored in Iceberg snapshot metadata.
>
> Thanks Steven for these detailed explanations. It makes me know the IceBerg
> better. However, I prefer to let the developer produce id to dedupe. I
> think this gives the developer more opportunity to optimize. You could see
> the following for more details. Please correct me if I misunderstand you.
>
> >> 3. Whether the `Writer` supports async functionality or not. Currently I
> do
> >> not know which sink could benefit from it. Maybe it is just my own
> problem.
>
> >> Here, I don't really know. We can introduce an "isAvailable()" method
> >> and mostly ignore it for now and sinks can just always return true. Or,
> >> as an alternative, we don't add the method now but can add it later with
> >> a default implementation. Either way, we will probably not take
> >> advantage of the "isAvailable()" now because that would require more
> >> runtime changes.
>
> From the @Pitor's explanation I could see the other benefit that might be
> gained in the future. For example decoupling the task number and the thread
> number. But I have to admit that introducing `isAvailable` might introduce
> some complications in the runtime. You could see my alternative API option
> in the following. I believe that we could support such an async sink writer
> very easily in the future. What do you think?
>
> >> Yes, this is still tricky. What is the current state, would the
> >> introduction of a "LocalCommit" and a "GlobalCommit" already solve both
> >> the Iceberg and Hive cases? I believe Hive is the most tricky one here,
> >> but if we introduce the "combine" method on GlobalCommit, that could
> >> serve the same purpose as the "aggregation operation" on the individual
> >> files, and we could even execute that "combine" in a distributed way.
> >>We assume that GlobalCommit is a Agg/Combiner?
>
> I would share what possible problems that I am seeing currently and the
> alternative options.
>
> ## IceBerg Sink
>
> ### Concern about generating nonce by framework.
>
> If let the `GlobalCommitter` provide a random nonce for the `IceBergSink` I
> think that it might not be efficient.  Because even if there are a very
> small number of committables in the state you still need to iterate all the
> iceberg snapshot files to check whether the committable is committed
> already. Even if it is efficient for the IceBergSink it might not be the
> case for other sinks.
>
> If the framework generates auto-increment nonce instead, it might still not
> be optimal for users. For example, users might want to use some business id
> so that after failover they could query whether the commit is successful
> after failover.
>
> I think users could generate more efficient nonce such as an auto-increment
> one. Therefore, it seems to provide more optimization chances if we let
> users to generate the nonce.
>
>
> ### Alternative Option
>
> public interface GlobalCommit<CommT, GlobalCommT> {
>         // provide some runtime context such as attempt-id,job-id,task-id.
>         void open(InitContext context);
>
>         // This GlobalCommit would aggregate the committable to a
> GlobalCommit before doing the commit operation.
>         GlobalCommT combine(List<Committable> commitables)
>
>         // This method would be called after committing all the
> GlobalCommit producing in the previous session.
>         void recoveredGlobalCommittables(List<GlobalCommit> globalCommits)
>
>         // developer would guarantee the idempotency by himself
>         void commit(GlobalCommit globalCommit);
> }
>
> User could guarantee the idenpointecy himself in a more efficient or
> application specific way. If the user wants the `GlobalCommit` to be
> executed in a distributed way, the user could use the runtime information
> to generate the partial order id himself.(We could ignore the clean up
> first)
>
> Currently the sink might be looks like following:
>
> Sink<IN, LC, LCO, GC> {
>         Writer<IN, LC> createWriter();
>         Optional<Committer<LC, LCO>> createCommitter();
>         Optional<GlobalCommitter<LCO, GC>> createGlobalCommitter();
> }
>
> ## Hive
>
> The HiveSink needs to compute whether a directory is finished or not. But
> HiveSink can not use the above `combine` method to decide whether a
> directory is finished or not.
>
> For example we assume that whether the directory is finished or not is
> decided by the event time. There might be a topology that the source and
> sink are forward. The event time might be different in different instances
> of the `writer`. So the GlobalCommit’s combine can not produce a
> GlobalCommT when the snapshot happens.
>
> In addition to the above case we should also consider the unaligned
> checkpoint. Because the watermark does not skip. So there might be the same
> problem in the unaligned checkpoint.
>
> ### Option1:
>
> public interface GlobalCommit<CommT, GlobalCommT, StateT, ShareT> {
>         // provide some runtime context such as attempt-id,job-id,task-id,
> maybe the event time;provide the restore state
>         void open(InitContext context, StateT state);
>
>         // This is for the HiveSink. When all the writer say that the the
> bucket is finished it would return a GlobalCommitT
>         Optional<GlobalCommT> combine(Committable commitables)
>
>         // This is for IcebergSink. Producing a GlobalCommitT every
> checkpoint.
>         Optional<GlobalCommT> preCommit();
>
>         // Maybe we need the shareState? After we decide the directory we
> make more detailed consideration then. The id could be remembered here.
>         StateT snapshotState();
>
>         // developer would guarantee the idempotency by himself
>         void commit(GlobalCommit globalCommit);
> }
>
> ### Option2
>
> Actually the `GlobalCommit` in the option1 mixes the `Writer` and
> `Committer` together. So it is intuitive to decouple the two functions. For
> support the hive we could prove a sink look like following
>
> Sink<In, LC, LCO, LCG> {
>         Writer<In, LC> createWriter();
>         Optional<Committer<LC, LCO>> createCommitter(); // we need this to
> change name.
>         Optional<Writer<LCO, LCG>> createGlobalAgg();
>         Optional<Committer<LCG, void>> createGlobalCommitter();
> }
>
> The pro of this method is that we use two basic concepts: `Committer` and
> `Writer` to build a HiveSink.
>
> ### CompactHiveSink / MergeHiveSink
>
> There are still other complicated cases, which are not satisfied by the
> above option. Users often complain about writing out many small files,
> which will affect file reading efficiency and the performance and stability
> of the distributed file system. CompactHiveSink/MergeHiveSink hopes to
> merge all files generated by this job in a single Checkpoint.
>
> The CompactHiveSink/MergeHiveSink topology can simply describe this
> topology as follows:
>
> CompactSubTopology -> GlobalAgg -> GobalCommitter.
>
> The CompactSubTopology would look like following:
>
> TmpFileWriter -> CompactCoodinator -> CompactorFileWriter
>
> Maybe the topology could be simpler but please keep in mind I just want to
> show that there might be very complicated topology requirements for users.
>
>
> A possible alternative option would be let the user build the topology
> himself. But considering we have two execution modes we could only use
> `Writer` and `Committer` to build the sink topology.
>
> ### Build Topology Option
>
> Sink<IN, OUT> {
>         Sink<In, Out> addWriter(Writer<In, Out> Writer); // Maybe a
> WriterBuidler
>         Sink<In, Out> addCommitter(Committer<In, Out> committer); // Maybe
> we could make this return Void if we do not consider code reuse and
> introduce the cleaner
> }
>
> ## Summary
> The requirements of sink might be different, maybe we could use two basic
> bricks(Writer/Committer) to let the user build their own sink topology.
> What do you guys think?
>
> I know the name stuff might be trikky for now but I want to discuss these
> things after we get the consus on the direction first.
>
> Best,
> Guowei
>
>
> On Sat, Sep 19, 2020 at 12:15 AM Steven Wu <[hidden email]> wrote:
>
> > Aljoscha,
> >
> > > Instead the sink would have to check for each set of committables
> > seperately if they had already been committed. Do you think this is
> > feasible?
> >
> > Yes, that is how it works in our internal implementation [1]. We don't
> use
> > checkpointId. We generate a manifest file (GlobalCommT) to bundle all the
> > data files that the committer received in one checkpoint cycle. Then we
> > generate a unique manifest id for by hashing the location of the manifest
> > file. The manifest ids are stored in Iceberg snapshot metadata. Upon
> > restore, we check each of the restored manifest files against Iceberg
> table
> > snapshot metadata to determine if we should discard or keep the restored
> > manifest files. If a commit has multiple manifest files (e.g. accumulated
> > from previous failed commits), we store the comma-separated manifest ids
> in
> > Iceberg snapshot metadata.
> >
> > > During normal operation this set would be very small, it would usually
> > only be the committables for the last checkpoint. Only when there is an
> > outage would multiple sets of committables pile up.
> >
> > You are absolutely right here. Even if there are multiple sets of
> > committables, it is usually the last a few or dozen of snapshots we need
> to
> > check. Even with our current inefficient implementation of traversing all
> > table snapshots (in the scale of thousands) from oldest to latest, it
> only
> > took avg 60 ms and max 800 ms. so it is really not a concern for Iceberg.
> >
> > > CommitStatus commitGlobally(List<Committable>, Nonce)
> >
> > Just to clarify on the terminology here. Assuming here the Committable
> > meant the `GlobalCommT` (like ManifestFile in Iceberg) in
> > previous discussions, right? `CommT` means the Iceberg DataFile from
> writer
> > to committer.
> >
> > This can work assuming we *don't have concurrent executions
> > of commitGlobally* even with concurrent checkpoints. Here is the scenario
> > regarding failure recovery I want to avoid.
> >
> > Assuming checkpoints 1, 2, 3 all completed. Each checkpoint generates a
> > manifest file, manifest-1, 2, 3.
> > timeline
> >
> ------------------------------------------------------------------------->
> > now
> > commitGlobally(manifest-1, nonce-1) started
> >          commitGlobally(manifest-2, nonce-2) started
> >                     commitGlobally(manifest-2, nonce-2) failed
> >                             commitGlobally(manifest-2 and manifest-3,
> > nonce-3) started
> >                                     commitGlobally(manifest-1, nonce-1)
> > failed
> >                                             commitGlobally(manifest-2 and
> > manifest-3, nonce-3) succeeded
> >
> > Now the job failed and was restored from checkpoint 3, which contains
> > manifest file 1,2,3. We found nonce-3 was committed when checking Iceberg
> > table snapshot metadata. But in this case we won't be able to correctly
> > determine which manifest files were committed or not.
> >
> > If it is possible to have concurrent executions of  commitGlobally, the
> > alternative is to generate the unique id/nonce per GlobalCommT. Then we
> can
> > check each individual GlobalCommT (ManifestFile) with Iceberg snapshot
> > metadata.
> >
> > Thanks,
> > Steven
> >
> > [1]
> >
> >
> https://github.com/Netflix-Skunkworks/nfflink-connector-iceberg/blob/master/nfflink-connector-iceberg/src/main/java/com/netflix/spaas/nfflink/connector/iceberg/sink/IcebergCommitter.java#L569
> >
> > On Fri, Sep 18, 2020 at 2:44 AM Aljoscha Krettek <[hidden email]>
> > wrote:
> >
> > > Steven,
> > >
> > > we were also wondering if it is a strict requirement that "later"
> > > updates to Iceberg subsume earlier updates. In the current version, you
> > > only check whether checkpoint X made it to Iceberg and then discard all
> > > committable state from Flink state for checkpoints smaller X.
> > >
> > > If we go with a (somewhat random) nonce, this would not work. Instead
> > > the sink would have to check for each set of committables seperately if
> > > they had already been committed. Do you think this is feasible? During
> > > normal operation this set would be very small, it would usually only be
> > > the committables for the last checkpoint. Only when there is an outage
> > > would multiple sets of committables pile up.
> > >
> > > We were thinking to extend the GlobalCommitter interface to allow it to
> > > report success or failure and then let the framework retry. I think
> this
> > > is something that you would need for the Iceberg case. The signature
> > > could be like this:
> > >
> > > CommitStatus commitGlobally(List<Committable>, Nonce)
> > >
> > > where CommitStatus could be an enum of SUCCESS, TERMINAL_FAILURE, and
> > > RETRY.
> > >
> > > Best,
> > > Aljoscha
> > >
> >
>
Reply | Threaded
Open this post in threaded view
|

Re: [DISCUSS] FLIP-143: Unified Sink API

Guowei Ma
Hi, Stevn
I want to make a clarification first, the following reply only considers
the Iceberge sink, but does not consider other sinks.  Before make decision
we should consider all the sink.I would try to summary all the sink
requirments in the next mail


>>  run global committer in jobmanager (e.g. like sink coordinator)

I think it could be.


>> You meant GlobalCommit -> GlobalCommT, right?

Yes. Thanks :)


>> Is this called when restored from checkpoint/savepoint?

Yes.


>>Iceberg sink needs to do a dup check here on which GlobalCommT were
committed and which weren't. Should it return the filtered/de-duped list of
GlobalCommT?


I think Iceberg sink needs to do the dedup in the `commit` call. The
`recoveredGlobalCommittables` is just for restoring the ids.


>> Sink implementation can decide if it wants to commit immediately or just
leave

I think only the frame knows *when* call the commit function.


>>should this be "commit(List<GlobalCommT>)"?

It could be. thanks.


Best,
Guowei


On Sun, Sep 20, 2020 at 12:11 AM Steven Wu <[hidden email]> wrote:

> > I prefer to let the developer produce id to dedupe. I think this gives
> the developer more opportunity to optimize.
>
> Thinking about it again, I totally agree with Guowei on this. We don't
> really need the framework to generate the unique id for Iceberg sink.
> De-dup logic is totally internal to Iceberg sink and should be isolated
> inside. My earlier question regarding "commitGlobally(List<GlobalCommT>)
> can be concurrent or not" also becomes irrelevant, as long as the framework
> handles the GlobalCommT list properly (even with concurrent calls).
>
> Here are the things where framework can help
>
>    1. run global committer in jobmanager (e.g. like sink coordinator)
>    2. help with checkpointing, bookkeeping, commit failure handling,
>    recovery
>
>
> @Guowei Ma <[hidden email]> regarding the GlobalCommitter
> interface, I have some clarifying questions.
>
> > void recoveredGlobalCommittables(List<GlobalCommit> globalCommits)
>
>    1. You meant GlobalCommit -> GlobalCommT, right?
>    2. Is this called when restored from checkpoint/savepoint?
>    3.  Iceberg sink needs to do a dup check here on which GlobalCommT
>    were committed and which weren't. Should it return the filtered/de-duped
>    list of GlobalCommT?
>    4. Sink implementation can decide if it wants to commit immediately or
>    just leave
>
> > void commit(GlobalCommit globalCommit);
>
> should this be "commit(List<GlobalCommT>)"?
>
> Thanks,
> Steven
>
>
> On Sat, Sep 19, 2020 at 1:56 AM Guowei Ma <[hidden email]> wrote:
>
>> Hi, all
>>
>> >>Just to add to what Aljoscha said regarding the unique id. Iceberg sink
>> >>checkpoints the unique id into state during snapshot. It also inserts
>> the
>> >>unique id into the Iceberg snapshot metadata during commit. When a job
>> >>restores the state after failure, it needs to know if the restored
>> >>transactions/commits were successful or not. It basically iterates
>> through
>> >>the list of table snapshots from Iceberg and matches the unique ids with
>> >>what is stored in Iceberg snapshot metadata.
>>
>> Thanks Steven for these detailed explanations. It makes me know the
>> IceBerg
>> better. However, I prefer to let the developer produce id to dedupe. I
>> think this gives the developer more opportunity to optimize. You could see
>> the following for more details. Please correct me if I misunderstand you.
>>
>> >> 3. Whether the `Writer` supports async functionality or not. Currently
>> I
>> do
>> >> not know which sink could benefit from it. Maybe it is just my own
>> problem.
>>
>> >> Here, I don't really know. We can introduce an "isAvailable()" method
>> >> and mostly ignore it for now and sinks can just always return true. Or,
>> >> as an alternative, we don't add the method now but can add it later
>> with
>> >> a default implementation. Either way, we will probably not take
>> >> advantage of the "isAvailable()" now because that would require more
>> >> runtime changes.
>>
>> From the @Pitor's explanation I could see the other benefit that might be
>> gained in the future. For example decoupling the task number and the
>> thread
>> number. But I have to admit that introducing `isAvailable` might introduce
>> some complications in the runtime. You could see my alternative API option
>> in the following. I believe that we could support such an async sink
>> writer
>> very easily in the future. What do you think?
>>
>> >> Yes, this is still tricky. What is the current state, would the
>> >> introduction of a "LocalCommit" and a "GlobalCommit" already solve both
>> >> the Iceberg and Hive cases? I believe Hive is the most tricky one here,
>> >> but if we introduce the "combine" method on GlobalCommit, that could
>> >> serve the same purpose as the "aggregation operation" on the individual
>> >> files, and we could even execute that "combine" in a distributed way.
>> >>We assume that GlobalCommit is a Agg/Combiner?
>>
>> I would share what possible problems that I am seeing currently and the
>> alternative options.
>>
>> ## IceBerg Sink
>>
>> ### Concern about generating nonce by framework.
>>
>> If let the `GlobalCommitter` provide a random nonce for the `IceBergSink`
>> I
>> think that it might not be efficient.  Because even if there are a very
>> small number of committables in the state you still need to iterate all
>> the
>> iceberg snapshot files to check whether the committable is committed
>> already. Even if it is efficient for the IceBergSink it might not be the
>> case for other sinks.
>>
>> If the framework generates auto-increment nonce instead, it might still
>> not
>> be optimal for users. For example, users might want to use some business
>> id
>> so that after failover they could query whether the commit is successful
>> after failover.
>>
>> I think users could generate more efficient nonce such as an
>> auto-increment
>> one. Therefore, it seems to provide more optimization chances if we let
>> users to generate the nonce.
>>
>>
>> ### Alternative Option
>>
>> public interface GlobalCommit<CommT, GlobalCommT> {
>>         // provide some runtime context such as attempt-id,job-id,task-id.
>>         void open(InitContext context);
>>
>>         // This GlobalCommit would aggregate the committable to a
>> GlobalCommit before doing the commit operation.
>>         GlobalCommT combine(List<Committable> commitables)
>>
>>         // This method would be called after committing all the
>> GlobalCommit producing in the previous session.
>>         void recoveredGlobalCommittables(List<GlobalCommit> globalCommits)
>>
>>         // developer would guarantee the idempotency by himself
>>         void commit(GlobalCommit globalCommit);
>> }
>>
>> User could guarantee the idenpointecy himself in a more efficient or
>> application specific way. If the user wants the `GlobalCommit` to be
>> executed in a distributed way, the user could use the runtime information
>> to generate the partial order id himself.(We could ignore the clean up
>> first)
>>
>> Currently the sink might be looks like following:
>>
>> Sink<IN, LC, LCO, GC> {
>>         Writer<IN, LC> createWriter();
>>         Optional<Committer<LC, LCO>> createCommitter();
>>         Optional<GlobalCommitter<LCO, GC>> createGlobalCommitter();
>> }
>>
>> ## Hive
>>
>> The HiveSink needs to compute whether a directory is finished or not. But
>> HiveSink can not use the above `combine` method to decide whether a
>> directory is finished or not.
>>
>> For example we assume that whether the directory is finished or not is
>> decided by the event time. There might be a topology that the source and
>> sink are forward. The event time might be different in different instances
>> of the `writer`. So the GlobalCommit’s combine can not produce a
>> GlobalCommT when the snapshot happens.
>>
>> In addition to the above case we should also consider the unaligned
>> checkpoint. Because the watermark does not skip. So there might be the
>> same
>> problem in the unaligned checkpoint.
>>
>> ### Option1:
>>
>> public interface GlobalCommit<CommT, GlobalCommT, StateT, ShareT> {
>>         // provide some runtime context such as attempt-id,job-id,task-id,
>> maybe the event time;provide the restore state
>>         void open(InitContext context, StateT state);
>>
>>         // This is for the HiveSink. When all the writer say that the the
>> bucket is finished it would return a GlobalCommitT
>>         Optional<GlobalCommT> combine(Committable commitables)
>>
>>         // This is for IcebergSink. Producing a GlobalCommitT every
>> checkpoint.
>>         Optional<GlobalCommT> preCommit();
>>
>>         // Maybe we need the shareState? After we decide the directory we
>> make more detailed consideration then. The id could be remembered here.
>>         StateT snapshotState();
>>
>>         // developer would guarantee the idempotency by himself
>>         void commit(GlobalCommit globalCommit);
>> }
>>
>> ### Option2
>>
>> Actually the `GlobalCommit` in the option1 mixes the `Writer` and
>> `Committer` together. So it is intuitive to decouple the two functions.
>> For
>> support the hive we could prove a sink look like following
>>
>> Sink<In, LC, LCO, LCG> {
>>         Writer<In, LC> createWriter();
>>         Optional<Committer<LC, LCO>> createCommitter(); // we need this to
>> change name.
>>         Optional<Writer<LCO, LCG>> createGlobalAgg();
>>         Optional<Committer<LCG, void>> createGlobalCommitter();
>> }
>>
>> The pro of this method is that we use two basic concepts: `Committer` and
>> `Writer` to build a HiveSink.
>>
>> ### CompactHiveSink / MergeHiveSink
>>
>> There are still other complicated cases, which are not satisfied by the
>> above option. Users often complain about writing out many small files,
>> which will affect file reading efficiency and the performance and
>> stability
>> of the distributed file system. CompactHiveSink/MergeHiveSink hopes to
>> merge all files generated by this job in a single Checkpoint.
>>
>> The CompactHiveSink/MergeHiveSink topology can simply describe this
>> topology as follows:
>>
>> CompactSubTopology -> GlobalAgg -> GobalCommitter.
>>
>> The CompactSubTopology would look like following:
>>
>> TmpFileWriter -> CompactCoodinator -> CompactorFileWriter
>>
>> Maybe the topology could be simpler but please keep in mind I just want to
>> show that there might be very complicated topology requirements for users.
>>
>>
>> A possible alternative option would be let the user build the topology
>> himself. But considering we have two execution modes we could only use
>> `Writer` and `Committer` to build the sink topology.
>>
>> ### Build Topology Option
>>
>> Sink<IN, OUT> {
>>         Sink<In, Out> addWriter(Writer<In, Out> Writer); // Maybe a
>> WriterBuidler
>>         Sink<In, Out> addCommitter(Committer<In, Out> committer); // Maybe
>> we could make this return Void if we do not consider code reuse and
>> introduce the cleaner
>> }
>>
>> ## Summary
>> The requirements of sink might be different, maybe we could use two basic
>> bricks(Writer/Committer) to let the user build their own sink topology.
>> What do you guys think?
>>
>> I know the name stuff might be trikky for now but I want to discuss these
>> things after we get the consus on the direction first.
>>
>> Best,
>> Guowei
>>
>>
>> On Sat, Sep 19, 2020 at 12:15 AM Steven Wu <[hidden email]> wrote:
>>
>> > Aljoscha,
>> >
>> > > Instead the sink would have to check for each set of committables
>> > seperately if they had already been committed. Do you think this is
>> > feasible?
>> >
>> > Yes, that is how it works in our internal implementation [1]. We don't
>> use
>> > checkpointId. We generate a manifest file (GlobalCommT) to bundle all
>> the
>> > data files that the committer received in one checkpoint cycle. Then we
>> > generate a unique manifest id for by hashing the location of the
>> manifest
>> > file. The manifest ids are stored in Iceberg snapshot metadata. Upon
>> > restore, we check each of the restored manifest files against Iceberg
>> table
>> > snapshot metadata to determine if we should discard or keep the restored
>> > manifest files. If a commit has multiple manifest files (e.g.
>> accumulated
>> > from previous failed commits), we store the comma-separated manifest
>> ids in
>> > Iceberg snapshot metadata.
>> >
>> > > During normal operation this set would be very small, it would usually
>> > only be the committables for the last checkpoint. Only when there is an
>> > outage would multiple sets of committables pile up.
>> >
>> > You are absolutely right here. Even if there are multiple sets of
>> > committables, it is usually the last a few or dozen of snapshots we
>> need to
>> > check. Even with our current inefficient implementation of traversing
>> all
>> > table snapshots (in the scale of thousands) from oldest to latest, it
>> only
>> > took avg 60 ms and max 800 ms. so it is really not a concern for
>> Iceberg.
>> >
>> > > CommitStatus commitGlobally(List<Committable>, Nonce)
>> >
>> > Just to clarify on the terminology here. Assuming here the Committable
>> > meant the `GlobalCommT` (like ManifestFile in Iceberg) in
>> > previous discussions, right? `CommT` means the Iceberg DataFile from
>> writer
>> > to committer.
>> >
>> > This can work assuming we *don't have concurrent executions
>> > of commitGlobally* even with concurrent checkpoints. Here is the
>> scenario
>> > regarding failure recovery I want to avoid.
>> >
>> > Assuming checkpoints 1, 2, 3 all completed. Each checkpoint generates a
>> > manifest file, manifest-1, 2, 3.
>> > timeline
>> >
>> ------------------------------------------------------------------------->
>> > now
>> > commitGlobally(manifest-1, nonce-1) started
>> >          commitGlobally(manifest-2, nonce-2) started
>> >                     commitGlobally(manifest-2, nonce-2) failed
>> >                             commitGlobally(manifest-2 and manifest-3,
>> > nonce-3) started
>> >                                     commitGlobally(manifest-1, nonce-1)
>> > failed
>> >                                             commitGlobally(manifest-2
>> and
>> > manifest-3, nonce-3) succeeded
>> >
>> > Now the job failed and was restored from checkpoint 3, which contains
>> > manifest file 1,2,3. We found nonce-3 was committed when checking
>> Iceberg
>> > table snapshot metadata. But in this case we won't be able to correctly
>> > determine which manifest files were committed or not.
>> >
>> > If it is possible to have concurrent executions of  commitGlobally, the
>> > alternative is to generate the unique id/nonce per GlobalCommT. Then we
>> can
>> > check each individual GlobalCommT (ManifestFile) with Iceberg snapshot
>> > metadata.
>> >
>> > Thanks,
>> > Steven
>> >
>> > [1]
>> >
>> >
>> https://github.com/Netflix-Skunkworks/nfflink-connector-iceberg/blob/master/nfflink-connector-iceberg/src/main/java/com/netflix/spaas/nfflink/connector/iceberg/sink/IcebergCommitter.java#L569
>> >
>> > On Fri, Sep 18, 2020 at 2:44 AM Aljoscha Krettek <[hidden email]>
>> > wrote:
>> >
>> > > Steven,
>> > >
>> > > we were also wondering if it is a strict requirement that "later"
>> > > updates to Iceberg subsume earlier updates. In the current version,
>> you
>> > > only check whether checkpoint X made it to Iceberg and then discard
>> all
>> > > committable state from Flink state for checkpoints smaller X.
>> > >
>> > > If we go with a (somewhat random) nonce, this would not work. Instead
>> > > the sink would have to check for each set of committables seperately
>> if
>> > > they had already been committed. Do you think this is feasible? During
>> > > normal operation this set would be very small, it would usually only
>> be
>> > > the committables for the last checkpoint. Only when there is an outage
>> > > would multiple sets of committables pile up.
>> > >
>> > > We were thinking to extend the GlobalCommitter interface to allow it
>> to
>> > > report success or failure and then let the framework retry. I think
>> this
>> > > is something that you would need for the Iceberg case. The signature
>> > > could be like this:
>> > >
>> > > CommitStatus commitGlobally(List<Committable>, Nonce)
>> > >
>> > > where CommitStatus could be an enum of SUCCESS, TERMINAL_FAILURE, and
>> > > RETRY.
>> > >
>> > > Best,
>> > > Aljoscha
>> > >
>> >
>>
>
Reply | Threaded
Open this post in threaded view
|

Re: [DISCUSS] FLIP-143: Unified Sink API

Guowei Ma
I would like to summarize the file type sink in the thread and their
possible topologies.  I also try to give pros and cons of every topology
option. Correct me if I am wrong.

### FileSink

Topology Option: TmpFileWriter + Committer.

### IceBerg Sink

#### Topology Option1: `DataFileWriter` + `GlobalCommitterV0`.
Pro:
1. Same group has some id.
Cons:
1. May limit users’ optimization space;
2. The topology does not meet the Hive’s requirements.

#### Topology Option 2: `DataFileWriter` + `GlobalCommitterV1`
Pro:
1. User has the opportunity to optimize the implementation of idempotence
Cons:
2. Make the GlobalCommit more complicated.
3. The topology does not meets the Hive’s requirements

### Topology Option3: DataFileWriter + AggWriter + Committer

Pros:
1. Use two basic `Writer` & `Commiter` to meet the IceBerge’s requirements.
2. Opportunity to optimize the implementation of idempotence
3. The topology meets the Hive’s requirements.(See flowing)
Con:
1. It introduce a relative complex topologies

## HiveSink

### Topology Option1: `TmpFileWriter` + `Committer` + `GlobalCommitterV2`.
Pro:
1. Could skip the cleanup problem at first.
Con:
1. This style topology does not meet the CompactHiveSink requirements.

### Topology Option2: `TmpFileWriter` + `Committer` + `AggWriter` +
`Committer`
Pros
1. Could skip the clean up problem at first.
2. Decouple the GlobalCommitterV2 to `AggWriter` + `Committer`
Cons
1. This style topology does not meet the CompactHiveSink requirements.
2. There are two general `Committers` in the topology. For Hive’s case
there might be no problem. But there might be a problem in 1.12. For
example where to execute the sub-topology following the `Committer` in
batch execution mode for the general case. Because the topology is built
from `Writer` and `Committer` we might put all the sub-topology in the
OperatorCoordinator. But if the topology is too complicated it might be
very complicated. See following.

### Topology Option3 `FileWriter` + `AggWriter` + `Committer`
Pro
1. There is only one general committer.
Cons
1. It has to consider the cleanup problem. (In theory both the Option1 and
Option2 need to cleanup)
2. This style topology does not meet the CompactHiveSink requirements.
3. Have to figure out how to make the current version compatible.

### CompactHiveSink/MergeHiveSink

#### Topology Option1 `TmpFileWriter` + `Committer` + `MergerCoordinator` +
`MergeWriter` + `GlobalCommiterV2`
Pro
1. Could skip the clean up problem at first.
Cons
2. Where to execute the sub-topology following the `Committer`.

#### Topology Option2 `TmpFileWriter` + `Committer` + `MergerCoordinator` +
`MergeWriter` + AggWriter + Committer
Pros
1. Could skip the clean up problem at first
2. Decouple the GlobalCommitterV2 to `AggWriter` + `Committer`
Con
1. Where to execute the sub-topology following the `Committer`.

### Option3 FileWriter + MergeCoordinator + MergeFileWriter + Writer(Agg) +
Committer
Pro
1. There is only one committer. It is very easy to support in the batch
execution mode.
Con
2. It has to consider the cleanup problem. (In theory both the Option1 and
Option2 need to cleanup)


### Summary

From above we could divide the sink topology into two parts:
1. Write topology.
2. And One committer

So we could provide a unified sink API looks like the following:

public interface Sink<CommT> {
        List<Writer<?, ?>> getWriters();
        Committer<CommT> createCommitter()
}

In the long run maybe we could give the user more powerful ability like
this (Currently some transformation still belongs to runtime):
Sink<CommT> {
        Transformation<CommT> createWriteTopology();
         CommitFunction<CommT> createCommitter();
}

Best,
Guowei


On Sun, Sep 20, 2020 at 6:09 PM Guowei Ma <[hidden email]> wrote:

> Hi, Stevn
> I want to make a clarification first, the following reply only considers
> the Iceberge sink, but does not consider other sinks.  Before make decision
> we should consider all the sink.I would try to summary all the sink
> requirments in the next mail
>
>
> >>  run global committer in jobmanager (e.g. like sink coordinator)
>
> I think it could be.
>
>
> >> You meant GlobalCommit -> GlobalCommT, right?
>
> Yes. Thanks :)
>
>
> >> Is this called when restored from checkpoint/savepoint?
>
> Yes.
>
>
> >>Iceberg sink needs to do a dup check here on which GlobalCommT were
> committed and which weren't. Should it return the filtered/de-duped list of
> GlobalCommT?
>
>
> I think Iceberg sink needs to do the dedup in the `commit` call. The
> `recoveredGlobalCommittables` is just for restoring the ids.
>
>
> >> Sink implementation can decide if it wants to commit immediately or
> just leave
>
> I think only the frame knows *when* call the commit function.
>
>
> >>should this be "commit(List<GlobalCommT>)"?
>
> It could be. thanks.
>
>
> Best,
> Guowei
>
>
> On Sun, Sep 20, 2020 at 12:11 AM Steven Wu <[hidden email]> wrote:
>
>> > I prefer to let the developer produce id to dedupe. I think this gives
>> the developer more opportunity to optimize.
>>
>> Thinking about it again, I totally agree with Guowei on this. We don't
>> really need the framework to generate the unique id for Iceberg sink.
>> De-dup logic is totally internal to Iceberg sink and should be isolated
>> inside. My earlier question regarding "commitGlobally(List<GlobalCommT>)
>> can be concurrent or not" also becomes irrelevant, as long as the framework
>> handles the GlobalCommT list properly (even with concurrent calls).
>>
>> Here are the things where framework can help
>>
>>    1. run global committer in jobmanager (e.g. like sink coordinator)
>>    2. help with checkpointing, bookkeeping, commit failure handling,
>>    recovery
>>
>>
>> @Guowei Ma <[hidden email]> regarding the GlobalCommitter
>> interface, I have some clarifying questions.
>>
>> > void recoveredGlobalCommittables(List<GlobalCommit> globalCommits)
>>
>>    1. You meant GlobalCommit -> GlobalCommT, right?
>>    2. Is this called when restored from checkpoint/savepoint?
>>    3.  Iceberg sink needs to do a dup check here on which GlobalCommT
>>    were committed and which weren't. Should it return the filtered/de-duped
>>    list of GlobalCommT?
>>    4. Sink implementation can decide if it wants to commit immediately
>>    or just leave
>>
>> > void commit(GlobalCommit globalCommit);
>>
>> should this be "commit(List<GlobalCommT>)"?
>>
>> Thanks,
>> Steven
>>
>>
>> On Sat, Sep 19, 2020 at 1:56 AM Guowei Ma <[hidden email]> wrote:
>>
>>> Hi, all
>>>
>>> >>Just to add to what Aljoscha said regarding the unique id. Iceberg sink
>>> >>checkpoints the unique id into state during snapshot. It also inserts
>>> the
>>> >>unique id into the Iceberg snapshot metadata during commit. When a job
>>> >>restores the state after failure, it needs to know if the restored
>>> >>transactions/commits were successful or not. It basically iterates
>>> through
>>> >>the list of table snapshots from Iceberg and matches the unique ids
>>> with
>>> >>what is stored in Iceberg snapshot metadata.
>>>
>>> Thanks Steven for these detailed explanations. It makes me know the
>>> IceBerg
>>> better. However, I prefer to let the developer produce id to dedupe. I
>>> think this gives the developer more opportunity to optimize. You could
>>> see
>>> the following for more details. Please correct me if I misunderstand you.
>>>
>>> >> 3. Whether the `Writer` supports async functionality or not.
>>> Currently I
>>> do
>>> >> not know which sink could benefit from it. Maybe it is just my own
>>> problem.
>>>
>>> >> Here, I don't really know. We can introduce an "isAvailable()" method
>>> >> and mostly ignore it for now and sinks can just always return true.
>>> Or,
>>> >> as an alternative, we don't add the method now but can add it later
>>> with
>>> >> a default implementation. Either way, we will probably not take
>>> >> advantage of the "isAvailable()" now because that would require more
>>> >> runtime changes.
>>>
>>> From the @Pitor's explanation I could see the other benefit that might be
>>> gained in the future. For example decoupling the task number and the
>>> thread
>>> number. But I have to admit that introducing `isAvailable` might
>>> introduce
>>> some complications in the runtime. You could see my alternative API
>>> option
>>> in the following. I believe that we could support such an async sink
>>> writer
>>> very easily in the future. What do you think?
>>>
>>> >> Yes, this is still tricky. What is the current state, would the
>>> >> introduction of a "LocalCommit" and a "GlobalCommit" already solve
>>> both
>>> >> the Iceberg and Hive cases? I believe Hive is the most tricky one
>>> here,
>>> >> but if we introduce the "combine" method on GlobalCommit, that could
>>> >> serve the same purpose as the "aggregation operation" on the
>>> individual
>>> >> files, and we could even execute that "combine" in a distributed way.
>>> >>We assume that GlobalCommit is a Agg/Combiner?
>>>
>>> I would share what possible problems that I am seeing currently and the
>>> alternative options.
>>>
>>> ## IceBerg Sink
>>>
>>> ### Concern about generating nonce by framework.
>>>
>>> If let the `GlobalCommitter` provide a random nonce for the
>>> `IceBergSink` I
>>> think that it might not be efficient.  Because even if there are a very
>>> small number of committables in the state you still need to iterate all
>>> the
>>> iceberg snapshot files to check whether the committable is committed
>>> already. Even if it is efficient for the IceBergSink it might not be the
>>> case for other sinks.
>>>
>>> If the framework generates auto-increment nonce instead, it might still
>>> not
>>> be optimal for users. For example, users might want to use some business
>>> id
>>> so that after failover they could query whether the commit is successful
>>> after failover.
>>>
>>> I think users could generate more efficient nonce such as an
>>> auto-increment
>>> one. Therefore, it seems to provide more optimization chances if we let
>>> users to generate the nonce.
>>>
>>>
>>> ### Alternative Option
>>>
>>> public interface GlobalCommit<CommT, GlobalCommT> {
>>>         // provide some runtime context such as
>>> attempt-id,job-id,task-id.
>>>         void open(InitContext context);
>>>
>>>         // This GlobalCommit would aggregate the committable to a
>>> GlobalCommit before doing the commit operation.
>>>         GlobalCommT combine(List<Committable> commitables)
>>>
>>>         // This method would be called after committing all the
>>> GlobalCommit producing in the previous session.
>>>         void recoveredGlobalCommittables(List<GlobalCommit>
>>> globalCommits)
>>>
>>>         // developer would guarantee the idempotency by himself
>>>         void commit(GlobalCommit globalCommit);
>>> }
>>>
>>> User could guarantee the idenpointecy himself in a more efficient or
>>> application specific way. If the user wants the `GlobalCommit` to be
>>> executed in a distributed way, the user could use the runtime information
>>> to generate the partial order id himself.(We could ignore the clean up
>>> first)
>>>
>>> Currently the sink might be looks like following:
>>>
>>> Sink<IN, LC, LCO, GC> {
>>>         Writer<IN, LC> createWriter();
>>>         Optional<Committer<LC, LCO>> createCommitter();
>>>         Optional<GlobalCommitter<LCO, GC>> createGlobalCommitter();
>>> }
>>>
>>> ## Hive
>>>
>>> The HiveSink needs to compute whether a directory is finished or not. But
>>> HiveSink can not use the above `combine` method to decide whether a
>>> directory is finished or not.
>>>
>>> For example we assume that whether the directory is finished or not is
>>> decided by the event time. There might be a topology that the source and
>>> sink are forward. The event time might be different in different
>>> instances
>>> of the `writer`. So the GlobalCommit’s combine can not produce a
>>> GlobalCommT when the snapshot happens.
>>>
>>> In addition to the above case we should also consider the unaligned
>>> checkpoint. Because the watermark does not skip. So there might be the
>>> same
>>> problem in the unaligned checkpoint.
>>>
>>> ### Option1:
>>>
>>> public interface GlobalCommit<CommT, GlobalCommT, StateT, ShareT> {
>>>         // provide some runtime context such as
>>> attempt-id,job-id,task-id,
>>> maybe the event time;provide the restore state
>>>         void open(InitContext context, StateT state);
>>>
>>>         // This is for the HiveSink. When all the writer say that the the
>>> bucket is finished it would return a GlobalCommitT
>>>         Optional<GlobalCommT> combine(Committable commitables)
>>>
>>>         // This is for IcebergSink. Producing a GlobalCommitT every
>>> checkpoint.
>>>         Optional<GlobalCommT> preCommit();
>>>
>>>         // Maybe we need the shareState? After we decide the directory we
>>> make more detailed consideration then. The id could be remembered here.
>>>         StateT snapshotState();
>>>
>>>         // developer would guarantee the idempotency by himself
>>>         void commit(GlobalCommit globalCommit);
>>> }
>>>
>>> ### Option2
>>>
>>> Actually the `GlobalCommit` in the option1 mixes the `Writer` and
>>> `Committer` together. So it is intuitive to decouple the two functions.
>>> For
>>> support the hive we could prove a sink look like following
>>>
>>> Sink<In, LC, LCO, LCG> {
>>>         Writer<In, LC> createWriter();
>>>         Optional<Committer<LC, LCO>> createCommitter(); // we need this
>>> to
>>> change name.
>>>         Optional<Writer<LCO, LCG>> createGlobalAgg();
>>>         Optional<Committer<LCG, void>> createGlobalCommitter();
>>> }
>>>
>>> The pro of this method is that we use two basic concepts: `Committer` and
>>> `Writer` to build a HiveSink.
>>>
>>> ### CompactHiveSink / MergeHiveSink
>>>
>>> There are still other complicated cases, which are not satisfied by the
>>> above option. Users often complain about writing out many small files,
>>> which will affect file reading efficiency and the performance and
>>> stability
>>> of the distributed file system. CompactHiveSink/MergeHiveSink hopes to
>>> merge all files generated by this job in a single Checkpoint.
>>>
>>> The CompactHiveSink/MergeHiveSink topology can simply describe this
>>> topology as follows:
>>>
>>> CompactSubTopology -> GlobalAgg -> GobalCommitter.
>>>
>>> The CompactSubTopology would look like following:
>>>
>>> TmpFileWriter -> CompactCoodinator -> CompactorFileWriter
>>>
>>> Maybe the topology could be simpler but please keep in mind I just want
>>> to
>>> show that there might be very complicated topology requirements for
>>> users.
>>>
>>>
>>> A possible alternative option would be let the user build the topology
>>> himself. But considering we have two execution modes we could only use
>>> `Writer` and `Committer` to build the sink topology.
>>>
>>> ### Build Topology Option
>>>
>>> Sink<IN, OUT> {
>>>         Sink<In, Out> addWriter(Writer<In, Out> Writer); // Maybe a
>>> WriterBuidler
>>>         Sink<In, Out> addCommitter(Committer<In, Out> committer); //
>>> Maybe
>>> we could make this return Void if we do not consider code reuse and
>>> introduce the cleaner
>>> }
>>>
>>> ## Summary
>>> The requirements of sink might be different, maybe we could use two basic
>>> bricks(Writer/Committer) to let the user build their own sink topology.
>>> What do you guys think?
>>>
>>> I know the name stuff might be trikky for now but I want to discuss these
>>> things after we get the consus on the direction first.
>>>
>>> Best,
>>> Guowei
>>>
>>>
>>> On Sat, Sep 19, 2020 at 12:15 AM Steven Wu <[hidden email]> wrote:
>>>
>>> > Aljoscha,
>>> >
>>> > > Instead the sink would have to check for each set of committables
>>> > seperately if they had already been committed. Do you think this is
>>> > feasible?
>>> >
>>> > Yes, that is how it works in our internal implementation [1]. We don't
>>> use
>>> > checkpointId. We generate a manifest file (GlobalCommT) to bundle all
>>> the
>>> > data files that the committer received in one checkpoint cycle. Then we
>>> > generate a unique manifest id for by hashing the location of the
>>> manifest
>>> > file. The manifest ids are stored in Iceberg snapshot metadata. Upon
>>> > restore, we check each of the restored manifest files against Iceberg
>>> table
>>> > snapshot metadata to determine if we should discard or keep the
>>> restored
>>> > manifest files. If a commit has multiple manifest files (e.g.
>>> accumulated
>>> > from previous failed commits), we store the comma-separated manifest
>>> ids in
>>> > Iceberg snapshot metadata.
>>> >
>>> > > During normal operation this set would be very small, it would
>>> usually
>>> > only be the committables for the last checkpoint. Only when there is an
>>> > outage would multiple sets of committables pile up.
>>> >
>>> > You are absolutely right here. Even if there are multiple sets of
>>> > committables, it is usually the last a few or dozen of snapshots we
>>> need to
>>> > check. Even with our current inefficient implementation of traversing
>>> all
>>> > table snapshots (in the scale of thousands) from oldest to latest, it
>>> only
>>> > took avg 60 ms and max 800 ms. so it is really not a concern for
>>> Iceberg.
>>> >
>>> > > CommitStatus commitGlobally(List<Committable>, Nonce)
>>> >
>>> > Just to clarify on the terminology here. Assuming here the Committable
>>> > meant the `GlobalCommT` (like ManifestFile in Iceberg) in
>>> > previous discussions, right? `CommT` means the Iceberg DataFile from
>>> writer
>>> > to committer.
>>> >
>>> > This can work assuming we *don't have concurrent executions
>>> > of commitGlobally* even with concurrent checkpoints. Here is the
>>> scenario
>>> > regarding failure recovery I want to avoid.
>>> >
>>> > Assuming checkpoints 1, 2, 3 all completed. Each checkpoint generates a
>>> > manifest file, manifest-1, 2, 3.
>>> > timeline
>>> >
>>> ------------------------------------------------------------------------->
>>> > now
>>> > commitGlobally(manifest-1, nonce-1) started
>>> >          commitGlobally(manifest-2, nonce-2) started
>>> >                     commitGlobally(manifest-2, nonce-2) failed
>>> >                             commitGlobally(manifest-2 and manifest-3,
>>> > nonce-3) started
>>> >                                     commitGlobally(manifest-1, nonce-1)
>>> > failed
>>> >                                             commitGlobally(manifest-2
>>> and
>>> > manifest-3, nonce-3) succeeded
>>> >
>>> > Now the job failed and was restored from checkpoint 3, which contains
>>> > manifest file 1,2,3. We found nonce-3 was committed when checking
>>> Iceberg
>>> > table snapshot metadata. But in this case we won't be able to correctly
>>> > determine which manifest files were committed or not.
>>> >
>>> > If it is possible to have concurrent executions of  commitGlobally, the
>>> > alternative is to generate the unique id/nonce per GlobalCommT. Then
>>> we can
>>> > check each individual GlobalCommT (ManifestFile) with Iceberg snapshot
>>> > metadata.
>>> >
>>> > Thanks,
>>> > Steven
>>> >
>>> > [1]
>>> >
>>> >
>>> https://github.com/Netflix-Skunkworks/nfflink-connector-iceberg/blob/master/nfflink-connector-iceberg/src/main/java/com/netflix/spaas/nfflink/connector/iceberg/sink/IcebergCommitter.java#L569
>>> >
>>> > On Fri, Sep 18, 2020 at 2:44 AM Aljoscha Krettek <[hidden email]>
>>> > wrote:
>>> >
>>> > > Steven,
>>> > >
>>> > > we were also wondering if it is a strict requirement that "later"
>>> > > updates to Iceberg subsume earlier updates. In the current version,
>>> you
>>> > > only check whether checkpoint X made it to Iceberg and then discard
>>> all
>>> > > committable state from Flink state for checkpoints smaller X.
>>> > >
>>> > > If we go with a (somewhat random) nonce, this would not work. Instead
>>> > > the sink would have to check for each set of committables seperately
>>> if
>>> > > they had already been committed. Do you think this is feasible?
>>> During
>>> > > normal operation this set would be very small, it would usually only
>>> be
>>> > > the committables for the last checkpoint. Only when there is an
>>> outage
>>> > > would multiple sets of committables pile up.
>>> > >
>>> > > We were thinking to extend the GlobalCommitter interface to allow it
>>> to
>>> > > report success or failure and then let the framework retry. I think
>>> this
>>> > > is something that you would need for the Iceberg case. The signature
>>> > > could be like this:
>>> > >
>>> > > CommitStatus commitGlobally(List<Committable>, Nonce)
>>> > >
>>> > > where CommitStatus could be an enum of SUCCESS, TERMINAL_FAILURE, and
>>> > > RETRY.
>>> > >
>>> > > Best,
>>> > > Aljoscha
>>> > >
>>> >
>>>
>>
Reply | Threaded
Open this post in threaded view
|

Re: [DISCUSS] FLIP-143: Unified Sink API

Steven Wu
> I think Iceberg sink needs to do the dedup in the `commit` call. The
`recoveredGlobalCommittables` is just for restoring the ids.


@Guowei Ma <[hidden email]>  It is undesirable to do the dedup check
in the `commit` call, because it happens for each checkpoint cycle. We only
need to do the de-dup check one time when restoring GlobalCommT list from
the checkpoint.


Can you clarify the purpose of `recoveredGlobalCommittables`? If it is to
let sink implementations know the recovered GlobalCommT list, it is
probably not a sufficient API. For the Iceberg sink, we can try to
implement the de-dup check  inside the `recoveredGlobalCommittables` method
and commit any uncommitted GlobalCommT items. But how do we handle the
commit failed?


One alternative is to allow sink implementations to override "Li
st<GlobalCommT> recoverGlobalCommittables()". Framework handles the
checkpoint/state, and sink implementations can further customize the
restored list with de-dup check and filtering. Recovered uncommitted
GlobalCommT list will be committed in the next cycle. It is the same
rollover strategy for commit failure handling that we have been discussing.


## topologies


Regarding the topology options, if we agree that there is no one size fit
for all, we can let sink implementations choose the best topology. Maybe
the framework can provide 2-3 pre-defined topology implementations to help
the sinks.




On Sun, Sep 20, 2020 at 3:27 AM Guowei Ma <[hidden email]> wrote:

> I would like to summarize the file type sink in the thread and their
> possible topologies.  I also try to give pros and cons of every topology
> option. Correct me if I am wrong.
>
> ### FileSink
>
> Topology Option: TmpFileWriter + Committer.
>
> ### IceBerg Sink
>
> #### Topology Option1: `DataFileWriter` + `GlobalCommitterV0`.
> Pro:
> 1. Same group has some id.
> Cons:
> 1. May limit users’ optimization space;
> 2. The topology does not meet the Hive’s requirements.
>
> #### Topology Option 2: `DataFileWriter` + `GlobalCommitterV1`
> Pro:
> 1. User has the opportunity to optimize the implementation of idempotence
> Cons:
> 2. Make the GlobalCommit more complicated.
> 3. The topology does not meets the Hive’s requirements
>
> ### Topology Option3: DataFileWriter + AggWriter + Committer
>
> Pros:
> 1. Use two basic `Writer` & `Commiter` to meet the IceBerge’s requirements.
> 2. Opportunity to optimize the implementation of idempotence
> 3. The topology meets the Hive’s requirements.(See flowing)
> Con:
> 1. It introduce a relative complex topologies
>
> ## HiveSink
>
> ### Topology Option1: `TmpFileWriter` + `Committer` + `GlobalCommitterV2`.
> Pro:
> 1. Could skip the cleanup problem at first.
> Con:
> 1. This style topology does not meet the CompactHiveSink requirements.
>
> ### Topology Option2: `TmpFileWriter` + `Committer` + `AggWriter` +
> `Committer`
> Pros
> 1. Could skip the clean up problem at first.
> 2. Decouple the GlobalCommitterV2 to `AggWriter` + `Committer`
> Cons
> 1. This style topology does not meet the CompactHiveSink requirements.
> 2. There are two general `Committers` in the topology. For Hive’s case
> there might be no problem. But there might be a problem in 1.12. For
> example where to execute the sub-topology following the `Committer` in
> batch execution mode for the general case. Because the topology is built
> from `Writer` and `Committer` we might put all the sub-topology in the
> OperatorCoordinator. But if the topology is too complicated it might be
> very complicated. See following.
>
> ### Topology Option3 `FileWriter` + `AggWriter` + `Committer`
> Pro
> 1. There is only one general committer.
> Cons
> 1. It has to consider the cleanup problem. (In theory both the Option1 and
> Option2 need to cleanup)
> 2. This style topology does not meet the CompactHiveSink requirements.
> 3. Have to figure out how to make the current version compatible.
>
> ### CompactHiveSink/MergeHiveSink
>
> #### Topology Option1 `TmpFileWriter` + `Committer` + `MergerCoordinator`
> + `MergeWriter` + `GlobalCommiterV2`
> Pro
> 1. Could skip the clean up problem at first.
> Cons
> 2. Where to execute the sub-topology following the `Committer`.
>
> #### Topology Option2 `TmpFileWriter` + `Committer` + `MergerCoordinator`
> + `MergeWriter` + AggWriter + Committer
> Pros
> 1. Could skip the clean up problem at first
> 2. Decouple the GlobalCommitterV2 to `AggWriter` + `Committer`
> Con
> 1. Where to execute the sub-topology following the `Committer`.
>
> ### Option3 FileWriter + MergeCoordinator + MergeFileWriter + Writer(Agg)
> + Committer
> Pro
> 1. There is only one committer. It is very easy to support in the batch
> execution mode.
> Con
> 2. It has to consider the cleanup problem. (In theory both the Option1 and
> Option2 need to cleanup)
>
>
> ### Summary
>
> From above we could divide the sink topology into two parts:
> 1. Write topology.
> 2. And One committer
>
> So we could provide a unified sink API looks like the following:
>
> public interface Sink<CommT> {
>         List<Writer<?, ?>> getWriters();
>         Committer<CommT> createCommitter()
> }
>
> In the long run maybe we could give the user more powerful ability like
> this (Currently some transformation still belongs to runtime):
> Sink<CommT> {
>         Transformation<CommT> createWriteTopology();
>          CommitFunction<CommT> createCommitter();
> }
>
> Best,
> Guowei
>
>
> On Sun, Sep 20, 2020 at 6:09 PM Guowei Ma <[hidden email]> wrote:
>
>> Hi, Stevn
>> I want to make a clarification first, the following reply only considers
>> the Iceberge sink, but does not consider other sinks.  Before make decision
>> we should consider all the sink.I would try to summary all the sink
>> requirments in the next mail
>>
>>
>> >>  run global committer in jobmanager (e.g. like sink coordinator)
>>
>> I think it could be.
>>
>>
>> >> You meant GlobalCommit -> GlobalCommT, right?
>>
>> Yes. Thanks :)
>>
>>
>> >> Is this called when restored from checkpoint/savepoint?
>>
>> Yes.
>>
>>
>> >>Iceberg sink needs to do a dup check here on which GlobalCommT were
>> committed and which weren't. Should it return the filtered/de-duped list of
>> GlobalCommT?
>>
>>
>> I think Iceberg sink needs to do the dedup in the `commit` call. The
>> `recoveredGlobalCommittables` is just for restoring the ids.
>>
>>
>> >> Sink implementation can decide if it wants to commit immediately or
>> just leave
>>
>> I think only the frame knows *when* call the commit function.
>>
>>
>> >>should this be "commit(List<GlobalCommT>)"?
>>
>> It could be. thanks.
>>
>>
>> Best,
>> Guowei
>>
>>
>> On Sun, Sep 20, 2020 at 12:11 AM Steven Wu <[hidden email]> wrote:
>>
>>> > I prefer to let the developer produce id to dedupe. I think this gives
>>> the developer more opportunity to optimize.
>>>
>>> Thinking about it again, I totally agree with Guowei on this. We don't
>>> really need the framework to generate the unique id for Iceberg sink.
>>> De-dup logic is totally internal to Iceberg sink and should be isolated
>>> inside. My earlier question regarding "commitGlobally(List<GlobalCommT>)
>>> can be concurrent or not" also becomes irrelevant, as long as the framework
>>> handles the GlobalCommT list properly (even with concurrent calls).
>>>
>>> Here are the things where framework can help
>>>
>>>    1. run global committer in jobmanager (e.g. like sink coordinator)
>>>    2. help with checkpointing, bookkeeping, commit failure handling,
>>>    recovery
>>>
>>>
>>> @Guowei Ma <[hidden email]> regarding the GlobalCommitter
>>> interface, I have some clarifying questions.
>>>
>>> > void recoveredGlobalCommittables(List<GlobalCommit> globalCommits)
>>>
>>>    1. You meant GlobalCommit -> GlobalCommT, right?
>>>    2. Is this called when restored from checkpoint/savepoint?
>>>    3.  Iceberg sink needs to do a dup check here on which GlobalCommT
>>>    were committed and which weren't. Should it return the filtered/de-duped
>>>    list of GlobalCommT?
>>>    4. Sink implementation can decide if it wants to commit immediately
>>>    or just leave
>>>
>>> > void commit(GlobalCommit globalCommit);
>>>
>>> should this be "commit(List<GlobalCommT>)"?
>>>
>>> Thanks,
>>> Steven
>>>
>>>
>>> On Sat, Sep 19, 2020 at 1:56 AM Guowei Ma <[hidden email]> wrote:
>>>
>>>> Hi, all
>>>>
>>>> >>Just to add to what Aljoscha said regarding the unique id. Iceberg
>>>> sink
>>>> >>checkpoints the unique id into state during snapshot. It also inserts
>>>> the
>>>> >>unique id into the Iceberg snapshot metadata during commit. When a job
>>>> >>restores the state after failure, it needs to know if the restored
>>>> >>transactions/commits were successful or not. It basically iterates
>>>> through
>>>> >>the list of table snapshots from Iceberg and matches the unique ids
>>>> with
>>>> >>what is stored in Iceberg snapshot metadata.
>>>>
>>>> Thanks Steven for these detailed explanations. It makes me know the
>>>> IceBerg
>>>> better. However, I prefer to let the developer produce id to dedupe. I
>>>> think this gives the developer more opportunity to optimize. You could
>>>> see
>>>> the following for more details. Please correct me if I misunderstand
>>>> you.
>>>>
>>>> >> 3. Whether the `Writer` supports async functionality or not.
>>>> Currently I
>>>> do
>>>> >> not know which sink could benefit from it. Maybe it is just my own
>>>> problem.
>>>>
>>>> >> Here, I don't really know. We can introduce an "isAvailable()" method
>>>> >> and mostly ignore it for now and sinks can just always return true.
>>>> Or,
>>>> >> as an alternative, we don't add the method now but can add it later
>>>> with
>>>> >> a default implementation. Either way, we will probably not take
>>>> >> advantage of the "isAvailable()" now because that would require more
>>>> >> runtime changes.
>>>>
>>>> From the @Pitor's explanation I could see the other benefit that might
>>>> be
>>>> gained in the future. For example decoupling the task number and the
>>>> thread
>>>> number. But I have to admit that introducing `isAvailable` might
>>>> introduce
>>>> some complications in the runtime. You could see my alternative API
>>>> option
>>>> in the following. I believe that we could support such an async sink
>>>> writer
>>>> very easily in the future. What do you think?
>>>>
>>>> >> Yes, this is still tricky. What is the current state, would the
>>>> >> introduction of a "LocalCommit" and a "GlobalCommit" already solve
>>>> both
>>>> >> the Iceberg and Hive cases? I believe Hive is the most tricky one
>>>> here,
>>>> >> but if we introduce the "combine" method on GlobalCommit, that could
>>>> >> serve the same purpose as the "aggregation operation" on the
>>>> individual
>>>> >> files, and we could even execute that "combine" in a distributed way.
>>>> >>We assume that GlobalCommit is a Agg/Combiner?
>>>>
>>>> I would share what possible problems that I am seeing currently and the
>>>> alternative options.
>>>>
>>>> ## IceBerg Sink
>>>>
>>>> ### Concern about generating nonce by framework.
>>>>
>>>> If let the `GlobalCommitter` provide a random nonce for the
>>>> `IceBergSink` I
>>>> think that it might not be efficient.  Because even if there are a very
>>>> small number of committables in the state you still need to iterate all
>>>> the
>>>> iceberg snapshot files to check whether the committable is committed
>>>> already. Even if it is efficient for the IceBergSink it might not be the
>>>> case for other sinks.
>>>>
>>>> If the framework generates auto-increment nonce instead, it might still
>>>> not
>>>> be optimal for users. For example, users might want to use some
>>>> business id
>>>> so that after failover they could query whether the commit is successful
>>>> after failover.
>>>>
>>>> I think users could generate more efficient nonce such as an
>>>> auto-increment
>>>> one. Therefore, it seems to provide more optimization chances if we let
>>>> users to generate the nonce.
>>>>
>>>>
>>>> ### Alternative Option
>>>>
>>>> public interface GlobalCommit<CommT, GlobalCommT> {
>>>>         // provide some runtime context such as
>>>> attempt-id,job-id,task-id.
>>>>         void open(InitContext context);
>>>>
>>>>         // This GlobalCommit would aggregate the committable to a
>>>> GlobalCommit before doing the commit operation.
>>>>         GlobalCommT combine(List<Committable> commitables)
>>>>
>>>>         // This method would be called after committing all the
>>>> GlobalCommit producing in the previous session.
>>>>         void recoveredGlobalCommittables(List<GlobalCommit>
>>>> globalCommits)
>>>>
>>>>         // developer would guarantee the idempotency by himself
>>>>         void commit(GlobalCommit globalCommit);
>>>> }
>>>>
>>>> User could guarantee the idenpointecy himself in a more efficient or
>>>> application specific way. If the user wants the `GlobalCommit` to be
>>>> executed in a distributed way, the user could use the runtime
>>>> information
>>>> to generate the partial order id himself.(We could ignore the clean up
>>>> first)
>>>>
>>>> Currently the sink might be looks like following:
>>>>
>>>> Sink<IN, LC, LCO, GC> {
>>>>         Writer<IN, LC> createWriter();
>>>>         Optional<Committer<LC, LCO>> createCommitter();
>>>>         Optional<GlobalCommitter<LCO, GC>> createGlobalCommitter();
>>>> }
>>>>
>>>> ## Hive
>>>>
>>>> The HiveSink needs to compute whether a directory is finished or not.
>>>> But
>>>> HiveSink can not use the above `combine` method to decide whether a
>>>> directory is finished or not.
>>>>
>>>> For example we assume that whether the directory is finished or not is
>>>> decided by the event time. There might be a topology that the source and
>>>> sink are forward. The event time might be different in different
>>>> instances
>>>> of the `writer`. So the GlobalCommit’s combine can not produce a
>>>> GlobalCommT when the snapshot happens.
>>>>
>>>> In addition to the above case we should also consider the unaligned
>>>> checkpoint. Because the watermark does not skip. So there might be the
>>>> same
>>>> problem in the unaligned checkpoint.
>>>>
>>>> ### Option1:
>>>>
>>>> public interface GlobalCommit<CommT, GlobalCommT, StateT, ShareT> {
>>>>         // provide some runtime context such as
>>>> attempt-id,job-id,task-id,
>>>> maybe the event time;provide the restore state
>>>>         void open(InitContext context, StateT state);
>>>>
>>>>         // This is for the HiveSink. When all the writer say that the
>>>> the
>>>> bucket is finished it would return a GlobalCommitT
>>>>         Optional<GlobalCommT> combine(Committable commitables)
>>>>
>>>>         // This is for IcebergSink. Producing a GlobalCommitT every
>>>> checkpoint.
>>>>         Optional<GlobalCommT> preCommit();
>>>>
>>>>         // Maybe we need the shareState? After we decide the directory
>>>> we
>>>> make more detailed consideration then. The id could be remembered here.
>>>>         StateT snapshotState();
>>>>
>>>>         // developer would guarantee the idempotency by himself
>>>>         void commit(GlobalCommit globalCommit);
>>>> }
>>>>
>>>> ### Option2
>>>>
>>>> Actually the `GlobalCommit` in the option1 mixes the `Writer` and
>>>> `Committer` together. So it is intuitive to decouple the two functions.
>>>> For
>>>> support the hive we could prove a sink look like following
>>>>
>>>> Sink<In, LC, LCO, LCG> {
>>>>         Writer<In, LC> createWriter();
>>>>         Optional<Committer<LC, LCO>> createCommitter(); // we need this
>>>> to
>>>> change name.
>>>>         Optional<Writer<LCO, LCG>> createGlobalAgg();
>>>>         Optional<Committer<LCG, void>> createGlobalCommitter();
>>>> }
>>>>
>>>> The pro of this method is that we use two basic concepts: `Committer`
>>>> and
>>>> `Writer` to build a HiveSink.
>>>>
>>>> ### CompactHiveSink / MergeHiveSink
>>>>
>>>> There are still other complicated cases, which are not satisfied by the
>>>> above option. Users often complain about writing out many small files,
>>>> which will affect file reading efficiency and the performance and
>>>> stability
>>>> of the distributed file system. CompactHiveSink/MergeHiveSink hopes to
>>>> merge all files generated by this job in a single Checkpoint.
>>>>
>>>> The CompactHiveSink/MergeHiveSink topology can simply describe this
>>>> topology as follows:
>>>>
>>>> CompactSubTopology -> GlobalAgg -> GobalCommitter.
>>>>
>>>> The CompactSubTopology would look like following:
>>>>
>>>> TmpFileWriter -> CompactCoodinator -> CompactorFileWriter
>>>>
>>>> Maybe the topology could be simpler but please keep in mind I just want
>>>> to
>>>> show that there might be very complicated topology requirements for
>>>> users.
>>>>
>>>>
>>>> A possible alternative option would be let the user build the topology
>>>> himself. But considering we have two execution modes we could only use
>>>> `Writer` and `Committer` to build the sink topology.
>>>>
>>>> ### Build Topology Option
>>>>
>>>> Sink<IN, OUT> {
>>>>         Sink<In, Out> addWriter(Writer<In, Out> Writer); // Maybe a
>>>> WriterBuidler
>>>>         Sink<In, Out> addCommitter(Committer<In, Out> committer); //
>>>> Maybe
>>>> we could make this return Void if we do not consider code reuse and
>>>> introduce the cleaner
>>>> }
>>>>
>>>> ## Summary
>>>> The requirements of sink might be different, maybe we could use two
>>>> basic
>>>> bricks(Writer/Committer) to let the user build their own sink topology.
>>>> What do you guys think?
>>>>
>>>> I know the name stuff might be trikky for now but I want to discuss
>>>> these
>>>> things after we get the consus on the direction first.
>>>>
>>>> Best,
>>>> Guowei
>>>>
>>>>
>>>> On Sat, Sep 19, 2020 at 12:15 AM Steven Wu <[hidden email]>
>>>> wrote:
>>>>
>>>> > Aljoscha,
>>>> >
>>>> > > Instead the sink would have to check for each set of committables
>>>> > seperately if they had already been committed. Do you think this is
>>>> > feasible?
>>>> >
>>>> > Yes, that is how it works in our internal implementation [1]. We
>>>> don't use
>>>> > checkpointId. We generate a manifest file (GlobalCommT) to bundle all
>>>> the
>>>> > data files that the committer received in one checkpoint cycle. Then
>>>> we
>>>> > generate a unique manifest id for by hashing the location of the
>>>> manifest
>>>> > file. The manifest ids are stored in Iceberg snapshot metadata. Upon
>>>> > restore, we check each of the restored manifest files against Iceberg
>>>> table
>>>> > snapshot metadata to determine if we should discard or keep the
>>>> restored
>>>> > manifest files. If a commit has multiple manifest files (e.g.
>>>> accumulated
>>>> > from previous failed commits), we store the comma-separated manifest
>>>> ids in
>>>> > Iceberg snapshot metadata.
>>>> >
>>>> > > During normal operation this set would be very small, it would
>>>> usually
>>>> > only be the committables for the last checkpoint. Only when there is
>>>> an
>>>> > outage would multiple sets of committables pile up.
>>>> >
>>>> > You are absolutely right here. Even if there are multiple sets of
>>>> > committables, it is usually the last a few or dozen of snapshots we
>>>> need to
>>>> > check. Even with our current inefficient implementation of traversing
>>>> all
>>>> > table snapshots (in the scale of thousands) from oldest to latest, it
>>>> only
>>>> > took avg 60 ms and max 800 ms. so it is really not a concern for
>>>> Iceberg.
>>>> >
>>>> > > CommitStatus commitGlobally(List<Committable>, Nonce)
>>>> >
>>>> > Just to clarify on the terminology here. Assuming here the Committable
>>>> > meant the `GlobalCommT` (like ManifestFile in Iceberg) in
>>>> > previous discussions, right? `CommT` means the Iceberg DataFile from
>>>> writer
>>>> > to committer.
>>>> >
>>>> > This can work assuming we *don't have concurrent executions
>>>> > of commitGlobally* even with concurrent checkpoints. Here is the
>>>> scenario
>>>> > regarding failure recovery I want to avoid.
>>>> >
>>>> > Assuming checkpoints 1, 2, 3 all completed. Each checkpoint generates
>>>> a
>>>> > manifest file, manifest-1, 2, 3.
>>>> > timeline
>>>> >
>>>> ------------------------------------------------------------------------->
>>>> > now
>>>> > commitGlobally(manifest-1, nonce-1) started
>>>> >          commitGlobally(manifest-2, nonce-2) started
>>>> >                     commitGlobally(manifest-2, nonce-2) failed
>>>> >                             commitGlobally(manifest-2 and manifest-3,
>>>> > nonce-3) started
>>>> >                                     commitGlobally(manifest-1,
>>>> nonce-1)
>>>> > failed
>>>> >                                             commitGlobally(manifest-2
>>>> and
>>>> > manifest-3, nonce-3) succeeded
>>>> >
>>>> > Now the job failed and was restored from checkpoint 3, which contains
>>>> > manifest file 1,2,3. We found nonce-3 was committed when checking
>>>> Iceberg
>>>> > table snapshot metadata. But in this case we won't be able to
>>>> correctly
>>>> > determine which manifest files were committed or not.
>>>> >
>>>> > If it is possible to have concurrent executions of  commitGlobally,
>>>> the
>>>> > alternative is to generate the unique id/nonce per GlobalCommT. Then
>>>> we can
>>>> > check each individual GlobalCommT (ManifestFile) with Iceberg snapshot
>>>> > metadata.
>>>> >
>>>> > Thanks,
>>>> > Steven
>>>> >
>>>> > [1]
>>>> >
>>>> >
>>>> https://github.com/Netflix-Skunkworks/nfflink-connector-iceberg/blob/master/nfflink-connector-iceberg/src/main/java/com/netflix/spaas/nfflink/connector/iceberg/sink/IcebergCommitter.java#L569
>>>> >
>>>> > On Fri, Sep 18, 2020 at 2:44 AM Aljoscha Krettek <[hidden email]
>>>> >
>>>> > wrote:
>>>> >
>>>> > > Steven,
>>>> > >
>>>> > > we were also wondering if it is a strict requirement that "later"
>>>> > > updates to Iceberg subsume earlier updates. In the current version,
>>>> you
>>>> > > only check whether checkpoint X made it to Iceberg and then discard
>>>> all
>>>> > > committable state from Flink state for checkpoints smaller X.
>>>> > >
>>>> > > If we go with a (somewhat random) nonce, this would not work.
>>>> Instead
>>>> > > the sink would have to check for each set of committables
>>>> seperately if
>>>> > > they had already been committed. Do you think this is feasible?
>>>> During
>>>> > > normal operation this set would be very small, it would usually
>>>> only be
>>>> > > the committables for the last checkpoint. Only when there is an
>>>> outage
>>>> > > would multiple sets of committables pile up.
>>>> > >
>>>> > > We were thinking to extend the GlobalCommitter interface to allow
>>>> it to
>>>> > > report success or failure and then let the framework retry. I think
>>>> this
>>>> > > is something that you would need for the Iceberg case. The signature
>>>> > > could be like this:
>>>> > >
>>>> > > CommitStatus commitGlobally(List<Committable>, Nonce)
>>>> > >
>>>> > > where CommitStatus could be an enum of SUCCESS, TERMINAL_FAILURE,
>>>> and
>>>> > > RETRY.
>>>> > >
>>>> > > Best,
>>>> > > Aljoscha
>>>> > >
>>>> >
>>>>
>>>
Reply | Threaded
Open this post in threaded view
|

Re: [DISCUSS] FLIP-143: Unified Sink API

Piotr Nowojski-5
Hi Guowei,

> I believe that we could support such an async sink writer
> very easily in the future. What do you think?

How would you see the expansion in the future? Do you mean just adding
`isAvailable()` method with a default implementation later on?

Piotrek

pon., 21 wrz 2020 o 02:39 Steven Wu <[hidden email]> napisał(a):

> > I think Iceberg sink needs to do the dedup in the `commit` call. The
> `recoveredGlobalCommittables` is just for restoring the ids.
>
>
> @Guowei Ma <[hidden email]>  It is undesirable to do the dedup check
> in the `commit` call, because it happens for each checkpoint cycle. We only
> need to do the de-dup check one time when restoring GlobalCommT list from
> the checkpoint.
>
>
> Can you clarify the purpose of `recoveredGlobalCommittables`? If it is to
> let sink implementations know the recovered GlobalCommT list, it is
> probably not a sufficient API. For the Iceberg sink, we can try to
> implement the de-dup check  inside the `recoveredGlobalCommittables` method
> and commit any uncommitted GlobalCommT items. But how do we handle the
> commit failed?
>
>
> One alternative is to allow sink implementations to override "Li
> st<GlobalCommT> recoverGlobalCommittables()". Framework handles the
> checkpoint/state, and sink implementations can further customize the
> restored list with de-dup check and filtering. Recovered uncommitted
> GlobalCommT list will be committed in the next cycle. It is the same
> rollover strategy for commit failure handling that we have been discussing.
>
>
> ## topologies
>
>
> Regarding the topology options, if we agree that there is no one size fit
> for all, we can let sink implementations choose the best topology. Maybe
> the framework can provide 2-3 pre-defined topology implementations to help
> the sinks.
>
>
>
>
> On Sun, Sep 20, 2020 at 3:27 AM Guowei Ma <[hidden email]> wrote:
>
> > I would like to summarize the file type sink in the thread and their
> > possible topologies.  I also try to give pros and cons of every topology
> > option. Correct me if I am wrong.
> >
> > ### FileSink
> >
> > Topology Option: TmpFileWriter + Committer.
> >
> > ### IceBerg Sink
> >
> > #### Topology Option1: `DataFileWriter` + `GlobalCommitterV0`.
> > Pro:
> > 1. Same group has some id.
> > Cons:
> > 1. May limit users’ optimization space;
> > 2. The topology does not meet the Hive’s requirements.
> >
> > #### Topology Option 2: `DataFileWriter` + `GlobalCommitterV1`
> > Pro:
> > 1. User has the opportunity to optimize the implementation of idempotence
> > Cons:
> > 2. Make the GlobalCommit more complicated.
> > 3. The topology does not meets the Hive’s requirements
> >
> > ### Topology Option3: DataFileWriter + AggWriter + Committer
> >
> > Pros:
> > 1. Use two basic `Writer` & `Commiter` to meet the IceBerge’s
> requirements.
> > 2. Opportunity to optimize the implementation of idempotence
> > 3. The topology meets the Hive’s requirements.(See flowing)
> > Con:
> > 1. It introduce a relative complex topologies
> >
> > ## HiveSink
> >
> > ### Topology Option1: `TmpFileWriter` + `Committer` +
> `GlobalCommitterV2`.
> > Pro:
> > 1. Could skip the cleanup problem at first.
> > Con:
> > 1. This style topology does not meet the CompactHiveSink requirements.
> >
> > ### Topology Option2: `TmpFileWriter` + `Committer` + `AggWriter` +
> > `Committer`
> > Pros
> > 1. Could skip the clean up problem at first.
> > 2. Decouple the GlobalCommitterV2 to `AggWriter` + `Committer`
> > Cons
> > 1. This style topology does not meet the CompactHiveSink requirements.
> > 2. There are two general `Committers` in the topology. For Hive’s case
> > there might be no problem. But there might be a problem in 1.12. For
> > example where to execute the sub-topology following the `Committer` in
> > batch execution mode for the general case. Because the topology is built
> > from `Writer` and `Committer` we might put all the sub-topology in the
> > OperatorCoordinator. But if the topology is too complicated it might be
> > very complicated. See following.
> >
> > ### Topology Option3 `FileWriter` + `AggWriter` + `Committer`
> > Pro
> > 1. There is only one general committer.
> > Cons
> > 1. It has to consider the cleanup problem. (In theory both the Option1
> and
> > Option2 need to cleanup)
> > 2. This style topology does not meet the CompactHiveSink requirements.
> > 3. Have to figure out how to make the current version compatible.
> >
> > ### CompactHiveSink/MergeHiveSink
> >
> > #### Topology Option1 `TmpFileWriter` + `Committer` + `MergerCoordinator`
> > + `MergeWriter` + `GlobalCommiterV2`
> > Pro
> > 1. Could skip the clean up problem at first.
> > Cons
> > 2. Where to execute the sub-topology following the `Committer`.
> >
> > #### Topology Option2 `TmpFileWriter` + `Committer` + `MergerCoordinator`
> > + `MergeWriter` + AggWriter + Committer
> > Pros
> > 1. Could skip the clean up problem at first
> > 2. Decouple the GlobalCommitterV2 to `AggWriter` + `Committer`
> > Con
> > 1. Where to execute the sub-topology following the `Committer`.
> >
> > ### Option3 FileWriter + MergeCoordinator + MergeFileWriter + Writer(Agg)
> > + Committer
> > Pro
> > 1. There is only one committer. It is very easy to support in the batch
> > execution mode.
> > Con
> > 2. It has to consider the cleanup problem. (In theory both the Option1
> and
> > Option2 need to cleanup)
> >
> >
> > ### Summary
> >
> > From above we could divide the sink topology into two parts:
> > 1. Write topology.
> > 2. And One committer
> >
> > So we could provide a unified sink API looks like the following:
> >
> > public interface Sink<CommT> {
> >         List<Writer<?, ?>> getWriters();
> >         Committer<CommT> createCommitter()
> > }
> >
> > In the long run maybe we could give the user more powerful ability like
> > this (Currently some transformation still belongs to runtime):
> > Sink<CommT> {
> >         Transformation<CommT> createWriteTopology();
> >          CommitFunction<CommT> createCommitter();
> > }
> >
> > Best,
> > Guowei
> >
> >
> > On Sun, Sep 20, 2020 at 6:09 PM Guowei Ma <[hidden email]> wrote:
> >
> >> Hi, Stevn
> >> I want to make a clarification first, the following reply only considers
> >> the Iceberge sink, but does not consider other sinks.  Before make
> decision
> >> we should consider all the sink.I would try to summary all the sink
> >> requirments in the next mail
> >>
> >>
> >> >>  run global committer in jobmanager (e.g. like sink coordinator)
> >>
> >> I think it could be.
> >>
> >>
> >> >> You meant GlobalCommit -> GlobalCommT, right?
> >>
> >> Yes. Thanks :)
> >>
> >>
> >> >> Is this called when restored from checkpoint/savepoint?
> >>
> >> Yes.
> >>
> >>
> >> >>Iceberg sink needs to do a dup check here on which GlobalCommT were
> >> committed and which weren't. Should it return the filtered/de-duped
> list of
> >> GlobalCommT?
> >>
> >>
> >> I think Iceberg sink needs to do the dedup in the `commit` call. The
> >> `recoveredGlobalCommittables` is just for restoring the ids.
> >>
> >>
> >> >> Sink implementation can decide if it wants to commit immediately or
> >> just leave
> >>
> >> I think only the frame knows *when* call the commit function.
> >>
> >>
> >> >>should this be "commit(List<GlobalCommT>)"?
> >>
> >> It could be. thanks.
> >>
> >>
> >> Best,
> >> Guowei
> >>
> >>
> >> On Sun, Sep 20, 2020 at 12:11 AM Steven Wu <[hidden email]>
> wrote:
> >>
> >>> > I prefer to let the developer produce id to dedupe. I think this
> gives
> >>> the developer more opportunity to optimize.
> >>>
> >>> Thinking about it again, I totally agree with Guowei on this. We don't
> >>> really need the framework to generate the unique id for Iceberg sink.
> >>> De-dup logic is totally internal to Iceberg sink and should be isolated
> >>> inside. My earlier question regarding
> "commitGlobally(List<GlobalCommT>)
> >>> can be concurrent or not" also becomes irrelevant, as long as the
> framework
> >>> handles the GlobalCommT list properly (even with concurrent calls).
> >>>
> >>> Here are the things where framework can help
> >>>
> >>>    1. run global committer in jobmanager (e.g. like sink coordinator)
> >>>    2. help with checkpointing, bookkeeping, commit failure handling,
> >>>    recovery
> >>>
> >>>
> >>> @Guowei Ma <[hidden email]> regarding the GlobalCommitter
> >>> interface, I have some clarifying questions.
> >>>
> >>> > void recoveredGlobalCommittables(List<GlobalCommit> globalCommits)
> >>>
> >>>    1. You meant GlobalCommit -> GlobalCommT, right?
> >>>    2. Is this called when restored from checkpoint/savepoint?
> >>>    3.  Iceberg sink needs to do a dup check here on which GlobalCommT
> >>>    were committed and which weren't. Should it return the
> filtered/de-duped
> >>>    list of GlobalCommT?
> >>>    4. Sink implementation can decide if it wants to commit immediately
> >>>    or just leave
> >>>
> >>> > void commit(GlobalCommit globalCommit);
> >>>
> >>> should this be "commit(List<GlobalCommT>)"?
> >>>
> >>> Thanks,
> >>> Steven
> >>>
> >>>
> >>> On Sat, Sep 19, 2020 at 1:56 AM Guowei Ma <[hidden email]>
> wrote:
> >>>
> >>>> Hi, all
> >>>>
> >>>> >>Just to add to what Aljoscha said regarding the unique id. Iceberg
> >>>> sink
> >>>> >>checkpoints the unique id into state during snapshot. It also
> inserts
> >>>> the
> >>>> >>unique id into the Iceberg snapshot metadata during commit. When a
> job
> >>>> >>restores the state after failure, it needs to know if the restored
> >>>> >>transactions/commits were successful or not. It basically iterates
> >>>> through
> >>>> >>the list of table snapshots from Iceberg and matches the unique ids
> >>>> with
> >>>> >>what is stored in Iceberg snapshot metadata.
> >>>>
> >>>> Thanks Steven for these detailed explanations. It makes me know the
> >>>> IceBerg
> >>>> better. However, I prefer to let the developer produce id to dedupe. I
> >>>> think this gives the developer more opportunity to optimize. You could
> >>>> see
> >>>> the following for more details. Please correct me if I misunderstand
> >>>> you.
> >>>>
> >>>> >> 3. Whether the `Writer` supports async functionality or not.
> >>>> Currently I
> >>>> do
> >>>> >> not know which sink could benefit from it. Maybe it is just my own
> >>>> problem.
> >>>>
> >>>> >> Here, I don't really know. We can introduce an "isAvailable()"
> method
> >>>> >> and mostly ignore it for now and sinks can just always return true.
> >>>> Or,
> >>>> >> as an alternative, we don't add the method now but can add it later
> >>>> with
> >>>> >> a default implementation. Either way, we will probably not take
> >>>> >> advantage of the "isAvailable()" now because that would require
> more
> >>>> >> runtime changes.
> >>>>
> >>>> From the @Pitor's explanation I could see the other benefit that might
> >>>> be
> >>>> gained in the future. For example decoupling the task number and the
> >>>> thread
> >>>> number. But I have to admit that introducing `isAvailable` might
> >>>> introduce
> >>>> some complications in the runtime. You could see my alternative API
> >>>> option
> >>>> in the following. I believe that we could support such an async sink
> >>>> writer
> >>>> very easily in the future. What do you think?
> >>>>
> >>>> >> Yes, this is still tricky. What is the current state, would the
> >>>> >> introduction of a "LocalCommit" and a "GlobalCommit" already solve
> >>>> both
> >>>> >> the Iceberg and Hive cases? I believe Hive is the most tricky one
> >>>> here,
> >>>> >> but if we introduce the "combine" method on GlobalCommit, that
> could
> >>>> >> serve the same purpose as the "aggregation operation" on the
> >>>> individual
> >>>> >> files, and we could even execute that "combine" in a distributed
> way.
> >>>> >>We assume that GlobalCommit is a Agg/Combiner?
> >>>>
> >>>> I would share what possible problems that I am seeing currently and
> the
> >>>> alternative options.
> >>>>
> >>>> ## IceBerg Sink
> >>>>
> >>>> ### Concern about generating nonce by framework.
> >>>>
> >>>> If let the `GlobalCommitter` provide a random nonce for the
> >>>> `IceBergSink` I
> >>>> think that it might not be efficient.  Because even if there are a
> very
> >>>> small number of committables in the state you still need to iterate
> all
> >>>> the
> >>>> iceberg snapshot files to check whether the committable is committed
> >>>> already. Even if it is efficient for the IceBergSink it might not be
> the
> >>>> case for other sinks.
> >>>>
> >>>> If the framework generates auto-increment nonce instead, it might
> still
> >>>> not
> >>>> be optimal for users. For example, users might want to use some
> >>>> business id
> >>>> so that after failover they could query whether the commit is
> successful
> >>>> after failover.
> >>>>
> >>>> I think users could generate more efficient nonce such as an
> >>>> auto-increment
> >>>> one. Therefore, it seems to provide more optimization chances if we
> let
> >>>> users to generate the nonce.
> >>>>
> >>>>
> >>>> ### Alternative Option
> >>>>
> >>>> public interface GlobalCommit<CommT, GlobalCommT> {
> >>>>         // provide some runtime context such as
> >>>> attempt-id,job-id,task-id.
> >>>>         void open(InitContext context);
> >>>>
> >>>>         // This GlobalCommit would aggregate the committable to a
> >>>> GlobalCommit before doing the commit operation.
> >>>>         GlobalCommT combine(List<Committable> commitables)
> >>>>
> >>>>         // This method would be called after committing all the
> >>>> GlobalCommit producing in the previous session.
> >>>>         void recoveredGlobalCommittables(List<GlobalCommit>
> >>>> globalCommits)
> >>>>
> >>>>         // developer would guarantee the idempotency by himself
> >>>>         void commit(GlobalCommit globalCommit);
> >>>> }
> >>>>
> >>>> User could guarantee the idenpointecy himself in a more efficient or
> >>>> application specific way. If the user wants the `GlobalCommit` to be
> >>>> executed in a distributed way, the user could use the runtime
> >>>> information
> >>>> to generate the partial order id himself.(We could ignore the clean up
> >>>> first)
> >>>>
> >>>> Currently the sink might be looks like following:
> >>>>
> >>>> Sink<IN, LC, LCO, GC> {
> >>>>         Writer<IN, LC> createWriter();
> >>>>         Optional<Committer<LC, LCO>> createCommitter();
> >>>>         Optional<GlobalCommitter<LCO, GC>> createGlobalCommitter();
> >>>> }
> >>>>
> >>>> ## Hive
> >>>>
> >>>> The HiveSink needs to compute whether a directory is finished or not.
> >>>> But
> >>>> HiveSink can not use the above `combine` method to decide whether a
> >>>> directory is finished or not.
> >>>>
> >>>> For example we assume that whether the directory is finished or not is
> >>>> decided by the event time. There might be a topology that the source
> and
> >>>> sink are forward. The event time might be different in different
> >>>> instances
> >>>> of the `writer`. So the GlobalCommit’s combine can not produce a
> >>>> GlobalCommT when the snapshot happens.
> >>>>
> >>>> In addition to the above case we should also consider the unaligned
> >>>> checkpoint. Because the watermark does not skip. So there might be the
> >>>> same
> >>>> problem in the unaligned checkpoint.
> >>>>
> >>>> ### Option1:
> >>>>
> >>>> public interface GlobalCommit<CommT, GlobalCommT, StateT, ShareT> {
> >>>>         // provide some runtime context such as
> >>>> attempt-id,job-id,task-id,
> >>>> maybe the event time;provide the restore state
> >>>>         void open(InitContext context, StateT state);
> >>>>
> >>>>         // This is for the HiveSink. When all the writer say that the
> >>>> the
> >>>> bucket is finished it would return a GlobalCommitT
> >>>>         Optional<GlobalCommT> combine(Committable commitables)
> >>>>
> >>>>         // This is for IcebergSink. Producing a GlobalCommitT every
> >>>> checkpoint.
> >>>>         Optional<GlobalCommT> preCommit();
> >>>>
> >>>>         // Maybe we need the shareState? After we decide the directory
> >>>> we
> >>>> make more detailed consideration then. The id could be remembered
> here.
> >>>>         StateT snapshotState();
> >>>>
> >>>>         // developer would guarantee the idempotency by himself
> >>>>         void commit(GlobalCommit globalCommit);
> >>>> }
> >>>>
> >>>> ### Option2
> >>>>
> >>>> Actually the `GlobalCommit` in the option1 mixes the `Writer` and
> >>>> `Committer` together. So it is intuitive to decouple the two
> functions.
> >>>> For
> >>>> support the hive we could prove a sink look like following
> >>>>
> >>>> Sink<In, LC, LCO, LCG> {
> >>>>         Writer<In, LC> createWriter();
> >>>>         Optional<Committer<LC, LCO>> createCommitter(); // we need
> this
> >>>> to
> >>>> change name.
> >>>>         Optional<Writer<LCO, LCG>> createGlobalAgg();
> >>>>         Optional<Committer<LCG, void>> createGlobalCommitter();
> >>>> }
> >>>>
> >>>> The pro of this method is that we use two basic concepts: `Committer`
> >>>> and
> >>>> `Writer` to build a HiveSink.
> >>>>
> >>>> ### CompactHiveSink / MergeHiveSink
> >>>>
> >>>> There are still other complicated cases, which are not satisfied by
> the
> >>>> above option. Users often complain about writing out many small files,
> >>>> which will affect file reading efficiency and the performance and
> >>>> stability
> >>>> of the distributed file system. CompactHiveSink/MergeHiveSink hopes to
> >>>> merge all files generated by this job in a single Checkpoint.
> >>>>
> >>>> The CompactHiveSink/MergeHiveSink topology can simply describe this
> >>>> topology as follows:
> >>>>
> >>>> CompactSubTopology -> GlobalAgg -> GobalCommitter.
> >>>>
> >>>> The CompactSubTopology would look like following:
> >>>>
> >>>> TmpFileWriter -> CompactCoodinator -> CompactorFileWriter
> >>>>
> >>>> Maybe the topology could be simpler but please keep in mind I just
> want
> >>>> to
> >>>> show that there might be very complicated topology requirements for
> >>>> users.
> >>>>
> >>>>
> >>>> A possible alternative option would be let the user build the topology
> >>>> himself. But considering we have two execution modes we could only use
> >>>> `Writer` and `Committer` to build the sink topology.
> >>>>
> >>>> ### Build Topology Option
> >>>>
> >>>> Sink<IN, OUT> {
> >>>>         Sink<In, Out> addWriter(Writer<In, Out> Writer); // Maybe a
> >>>> WriterBuidler
> >>>>         Sink<In, Out> addCommitter(Committer<In, Out> committer); //
> >>>> Maybe
> >>>> we could make this return Void if we do not consider code reuse and
> >>>> introduce the cleaner
> >>>> }
> >>>>
> >>>> ## Summary
> >>>> The requirements of sink might be different, maybe we could use two
> >>>> basic
> >>>> bricks(Writer/Committer) to let the user build their own sink
> topology.
> >>>> What do you guys think?
> >>>>
> >>>> I know the name stuff might be trikky for now but I want to discuss
> >>>> these
> >>>> things after we get the consus on the direction first.
> >>>>
> >>>> Best,
> >>>> Guowei
> >>>>
> >>>>
> >>>> On Sat, Sep 19, 2020 at 12:15 AM Steven Wu <[hidden email]>
> >>>> wrote:
> >>>>
> >>>> > Aljoscha,
> >>>> >
> >>>> > > Instead the sink would have to check for each set of committables
> >>>> > seperately if they had already been committed. Do you think this is
> >>>> > feasible?
> >>>> >
> >>>> > Yes, that is how it works in our internal implementation [1]. We
> >>>> don't use
> >>>> > checkpointId. We generate a manifest file (GlobalCommT) to bundle
> all
> >>>> the
> >>>> > data files that the committer received in one checkpoint cycle. Then
> >>>> we
> >>>> > generate a unique manifest id for by hashing the location of the
> >>>> manifest
> >>>> > file. The manifest ids are stored in Iceberg snapshot metadata. Upon
> >>>> > restore, we check each of the restored manifest files against
> Iceberg
> >>>> table
> >>>> > snapshot metadata to determine if we should discard or keep the
> >>>> restored
> >>>> > manifest files. If a commit has multiple manifest files (e.g.
> >>>> accumulated
> >>>> > from previous failed commits), we store the comma-separated manifest
> >>>> ids in
> >>>> > Iceberg snapshot metadata.
> >>>> >
> >>>> > > During normal operation this set would be very small, it would
> >>>> usually
> >>>> > only be the committables for the last checkpoint. Only when there is
> >>>> an
> >>>> > outage would multiple sets of committables pile up.
> >>>> >
> >>>> > You are absolutely right here. Even if there are multiple sets of
> >>>> > committables, it is usually the last a few or dozen of snapshots we
> >>>> need to
> >>>> > check. Even with our current inefficient implementation of
> traversing
> >>>> all
> >>>> > table snapshots (in the scale of thousands) from oldest to latest,
> it
> >>>> only
> >>>> > took avg 60 ms and max 800 ms. so it is really not a concern for
> >>>> Iceberg.
> >>>> >
> >>>> > > CommitStatus commitGlobally(List<Committable>, Nonce)
> >>>> >
> >>>> > Just to clarify on the terminology here. Assuming here the
> Committable
> >>>> > meant the `GlobalCommT` (like ManifestFile in Iceberg) in
> >>>> > previous discussions, right? `CommT` means the Iceberg DataFile from
> >>>> writer
> >>>> > to committer.
> >>>> >
> >>>> > This can work assuming we *don't have concurrent executions
> >>>> > of commitGlobally* even with concurrent checkpoints. Here is the
> >>>> scenario
> >>>> > regarding failure recovery I want to avoid.
> >>>> >
> >>>> > Assuming checkpoints 1, 2, 3 all completed. Each checkpoint
> generates
> >>>> a
> >>>> > manifest file, manifest-1, 2, 3.
> >>>> > timeline
> >>>> >
> >>>>
> ------------------------------------------------------------------------->
> >>>> > now
> >>>> > commitGlobally(manifest-1, nonce-1) started
> >>>> >          commitGlobally(manifest-2, nonce-2) started
> >>>> >                     commitGlobally(manifest-2, nonce-2) failed
> >>>> >                             commitGlobally(manifest-2 and
> manifest-3,
> >>>> > nonce-3) started
> >>>> >                                     commitGlobally(manifest-1,
> >>>> nonce-1)
> >>>> > failed
> >>>> >
>  commitGlobally(manifest-2
> >>>> and
> >>>> > manifest-3, nonce-3) succeeded
> >>>> >
> >>>> > Now the job failed and was restored from checkpoint 3, which
> contains
> >>>> > manifest file 1,2,3. We found nonce-3 was committed when checking
> >>>> Iceberg
> >>>> > table snapshot metadata. But in this case we won't be able to
> >>>> correctly
> >>>> > determine which manifest files were committed or not.
> >>>> >
> >>>> > If it is possible to have concurrent executions of  commitGlobally,
> >>>> the
> >>>> > alternative is to generate the unique id/nonce per GlobalCommT. Then
> >>>> we can
> >>>> > check each individual GlobalCommT (ManifestFile) with Iceberg
> snapshot
> >>>> > metadata.
> >>>> >
> >>>> > Thanks,
> >>>> > Steven
> >>>> >
> >>>> > [1]
> >>>> >
> >>>> >
> >>>>
> https://github.com/Netflix-Skunkworks/nfflink-connector-iceberg/blob/master/nfflink-connector-iceberg/src/main/java/com/netflix/spaas/nfflink/connector/iceberg/sink/IcebergCommitter.java#L569
> >>>> >
> >>>> > On Fri, Sep 18, 2020 at 2:44 AM Aljoscha Krettek <
> [hidden email]
> >>>> >
> >>>> > wrote:
> >>>> >
> >>>> > > Steven,
> >>>> > >
> >>>> > > we were also wondering if it is a strict requirement that "later"
> >>>> > > updates to Iceberg subsume earlier updates. In the current
> version,
> >>>> you
> >>>> > > only check whether checkpoint X made it to Iceberg and then
> discard
> >>>> all
> >>>> > > committable state from Flink state for checkpoints smaller X.
> >>>> > >
> >>>> > > If we go with a (somewhat random) nonce, this would not work.
> >>>> Instead
> >>>> > > the sink would have to check for each set of committables
> >>>> seperately if
> >>>> > > they had already been committed. Do you think this is feasible?
> >>>> During
> >>>> > > normal operation this set would be very small, it would usually
> >>>> only be
> >>>> > > the committables for the last checkpoint. Only when there is an
> >>>> outage
> >>>> > > would multiple sets of committables pile up.
> >>>> > >
> >>>> > > We were thinking to extend the GlobalCommitter interface to allow
> >>>> it to
> >>>> > > report success or failure and then let the framework retry. I
> think
> >>>> this
> >>>> > > is something that you would need for the Iceberg case. The
> signature
> >>>> > > could be like this:
> >>>> > >
> >>>> > > CommitStatus commitGlobally(List<Committable>, Nonce)
> >>>> > >
> >>>> > > where CommitStatus could be an enum of SUCCESS, TERMINAL_FAILURE,
> >>>> and
> >>>> > > RETRY.
> >>>> > >
> >>>> > > Best,
> >>>> > > Aljoscha
> >>>> > >
> >>>> >
> >>>>
> >>>
>
Reply | Threaded
Open this post in threaded view
|

Re: [DISCUSS] FLIP-143: Unified Sink API

Aljoscha Krettek-2
Hi all,

I'll try and summarize my thoughts after Guowei, Yun, Kostas, Dawid, and
me had an offline discussion about this.

Also, I would like to give credit to Guowei for initially coming up with
the idea of a topology sink in the context of this discussion. I think
it's a good idea and we should pursue it in the future. And yes, Beam
already does it like this but I hadn't thought about it now when
thinking about the sink APIs because having a more limited API gives
more freedom to the framework.

## Topology Sink vs. Transactional Sink

 From the discussion, it seems clear to me that to support all kinds of
different use cases we will have to offer some sort of API that allows
Sink developers to specify mostly arbitrary operator topologies. I
think, however, that we will not manage to finish such a (semi
user-facing) API within the 1.12 release cycle with satisfactory
results. Therefore, I would say that we need to go with a more
straightforward TransactionalSink API (name TBD) that lets sink
developers specify basic Writer, Committer, GlobalCommitter components
as discussed above.

This Sink interface would initially support a FileSink that supports
HDFS/S3 and behaves like the StreamingFileSink does for STREAM execution
mode. Additionally, it would seamlessly work for BATCH execution mode.
With the addition of a properly designed GlobalCommitter this should
also work for Iceberg.

It seems to me that the Hive use cases are still to fuzzy and not well
defined to allow us to come up with a good solution.

## Committer vs. GlobalCommitter or both

To make it short, we should make both optional but also allow both to be
used by the same sink.

The Committer is the interface that should be preferred because it
allows the framework to distribute the work of committing, i.e. it has
more potential for being optimised.

Iceberg would use only a GlobalCommitter.

The FileSink would use only Committer but can optionally use a
GlobalCommitter to create a _SUCCESS file in the output directory to
emulate Hadoop to some degree. Writing such a _SUCCESS file would only
work in BATCH execution mode and it would write a _SUCCESS file in the
toplevel output directory. Writing _SUCCESS files in individual buckets
or subdirectories whenever these buckets are considered "done" is a
different problem, and one I don't think we can solve well right now.

Initially, I would propose these interfaces that have been floated by
various people above:

interface Committer<CommittableT> {
   CommitResult commit(CommittableT);
}

interface GlobalCommitter<CommittableT, GlobalCommittableT> {
   GlobalCommittableT merge(List<CommittableT>);
   CommitResult commit(GlobalCommittableT);
}

enum CommitResult {
   SUCCESS, FAILURE, RETRY
}

Alternatively, we could consider changing commit() to take a List<> to
allow the sink to better check if commits are already in the external
system. For example, Iceberg would check for the whole batch whether
they are already committed.

Also alternatively, we could change the GlobalCommitter to basically
return an AggregateFunction instead of the simple merge() function.

What do you think?

Best,
Aljoscha

On 21.09.20 10:06, Piotr Nowojski wrote:

> Hi Guowei,
>
>> I believe that we could support such an async sink writer
>> very easily in the future. What do you think?
>
> How would you see the expansion in the future? Do you mean just adding
> `isAvailable()` method with a default implementation later on?
>
> Piotrek
>
> pon., 21 wrz 2020 o 02:39 Steven Wu <[hidden email]> napisał(a):
>
>>> I think Iceberg sink needs to do the dedup in the `commit` call. The
>> `recoveredGlobalCommittables` is just for restoring the ids.
>>
>>
>> @Guowei Ma <[hidden email]>  It is undesirable to do the dedup check
>> in the `commit` call, because it happens for each checkpoint cycle. We only
>> need to do the de-dup check one time when restoring GlobalCommT list from
>> the checkpoint.
>>
>>
>> Can you clarify the purpose of `recoveredGlobalCommittables`? If it is to
>> let sink implementations know the recovered GlobalCommT list, it is
>> probably not a sufficient API. For the Iceberg sink, we can try to
>> implement the de-dup check  inside the `recoveredGlobalCommittables` method
>> and commit any uncommitted GlobalCommT items. But how do we handle the
>> commit failed?
>>
>>
>> One alternative is to allow sink implementations to override "Li
>> st<GlobalCommT> recoverGlobalCommittables()". Framework handles the
>> checkpoint/state, and sink implementations can further customize the
>> restored list with de-dup check and filtering. Recovered uncommitted
>> GlobalCommT list will be committed in the next cycle. It is the same
>> rollover strategy for commit failure handling that we have been discussing.
>>
>>
>> ## topologies
>>
>>
>> Regarding the topology options, if we agree that there is no one size fit
>> for all, we can let sink implementations choose the best topology. Maybe
>> the framework can provide 2-3 pre-defined topology implementations to help
>> the sinks.
>>
>>
>>
>>
>> On Sun, Sep 20, 2020 at 3:27 AM Guowei Ma <[hidden email]> wrote:
>>
>>> I would like to summarize the file type sink in the thread and their
>>> possible topologies.  I also try to give pros and cons of every topology
>>> option. Correct me if I am wrong.
>>>
>>> ### FileSink
>>>
>>> Topology Option: TmpFileWriter + Committer.
>>>
>>> ### IceBerg Sink
>>>
>>> #### Topology Option1: `DataFileWriter` + `GlobalCommitterV0`.
>>> Pro:
>>> 1. Same group has some id.
>>> Cons:
>>> 1. May limit users’ optimization space;
>>> 2. The topology does not meet the Hive’s requirements.
>>>
>>> #### Topology Option 2: `DataFileWriter` + `GlobalCommitterV1`
>>> Pro:
>>> 1. User has the opportunity to optimize the implementation of idempotence
>>> Cons:
>>> 2. Make the GlobalCommit more complicated.
>>> 3. The topology does not meets the Hive’s requirements
>>>
>>> ### Topology Option3: DataFileWriter + AggWriter + Committer
>>>
>>> Pros:
>>> 1. Use two basic `Writer` & `Commiter` to meet the IceBerge’s
>> requirements.
>>> 2. Opportunity to optimize the implementation of idempotence
>>> 3. The topology meets the Hive’s requirements.(See flowing)
>>> Con:
>>> 1. It introduce a relative complex topologies
>>>
>>> ## HiveSink
>>>
>>> ### Topology Option1: `TmpFileWriter` + `Committer` +
>> `GlobalCommitterV2`.
>>> Pro:
>>> 1. Could skip the cleanup problem at first.
>>> Con:
>>> 1. This style topology does not meet the CompactHiveSink requirements.
>>>
>>> ### Topology Option2: `TmpFileWriter` + `Committer` + `AggWriter` +
>>> `Committer`
>>> Pros
>>> 1. Could skip the clean up problem at first.
>>> 2. Decouple the GlobalCommitterV2 to `AggWriter` + `Committer`
>>> Cons
>>> 1. This style topology does not meet the CompactHiveSink requirements.
>>> 2. There are two general `Committers` in the topology. For Hive’s case
>>> there might be no problem. But there might be a problem in 1.12. For
>>> example where to execute the sub-topology following the `Committer` in
>>> batch execution mode for the general case. Because the topology is built
>>> from `Writer` and `Committer` we might put all the sub-topology in the
>>> OperatorCoordinator. But if the topology is too complicated it might be
>>> very complicated. See following.
>>>
>>> ### Topology Option3 `FileWriter` + `AggWriter` + `Committer`
>>> Pro
>>> 1. There is only one general committer.
>>> Cons
>>> 1. It has to consider the cleanup problem. (In theory both the Option1
>> and
>>> Option2 need to cleanup)
>>> 2. This style topology does not meet the CompactHiveSink requirements.
>>> 3. Have to figure out how to make the current version compatible.
>>>
>>> ### CompactHiveSink/MergeHiveSink
>>>
>>> #### Topology Option1 `TmpFileWriter` + `Committer` + `MergerCoordinator`
>>> + `MergeWriter` + `GlobalCommiterV2`
>>> Pro
>>> 1. Could skip the clean up problem at first.
>>> Cons
>>> 2. Where to execute the sub-topology following the `Committer`.
>>>
>>> #### Topology Option2 `TmpFileWriter` + `Committer` + `MergerCoordinator`
>>> + `MergeWriter` + AggWriter + Committer
>>> Pros
>>> 1. Could skip the clean up problem at first
>>> 2. Decouple the GlobalCommitterV2 to `AggWriter` + `Committer`
>>> Con
>>> 1. Where to execute the sub-topology following the `Committer`.
>>>
>>> ### Option3 FileWriter + MergeCoordinator + MergeFileWriter + Writer(Agg)
>>> + Committer
>>> Pro
>>> 1. There is only one committer. It is very easy to support in the batch
>>> execution mode.
>>> Con
>>> 2. It has to consider the cleanup problem. (In theory both the Option1
>> and
>>> Option2 need to cleanup)
>>>
>>>
>>> ### Summary
>>>
>>>  From above we could divide the sink topology into two parts:
>>> 1. Write topology.
>>> 2. And One committer
>>>
>>> So we could provide a unified sink API looks like the following:
>>>
>>> public interface Sink<CommT> {
>>>          List<Writer<?, ?>> getWriters();
>>>          Committer<CommT> createCommitter()
>>> }
>>>
>>> In the long run maybe we could give the user more powerful ability like
>>> this (Currently some transformation still belongs to runtime):
>>> Sink<CommT> {
>>>          Transformation<CommT> createWriteTopology();
>>>           CommitFunction<CommT> createCommitter();
>>> }
>>>
>>> Best,
>>> Guowei
>>>
>>>
>>> On Sun, Sep 20, 2020 at 6:09 PM Guowei Ma <[hidden email]> wrote:
>>>
>>>> Hi, Stevn
>>>> I want to make a clarification first, the following reply only considers
>>>> the Iceberge sink, but does not consider other sinks.  Before make
>> decision
>>>> we should consider all the sink.I would try to summary all the sink
>>>> requirments in the next mail
>>>>
>>>>
>>>>>>   run global committer in jobmanager (e.g. like sink coordinator)
>>>>
>>>> I think it could be.
>>>>
>>>>
>>>>>> You meant GlobalCommit -> GlobalCommT, right?
>>>>
>>>> Yes. Thanks :)
>>>>
>>>>
>>>>>> Is this called when restored from checkpoint/savepoint?
>>>>
>>>> Yes.
>>>>
>>>>
>>>>>> Iceberg sink needs to do a dup check here on which GlobalCommT were
>>>> committed and which weren't. Should it return the filtered/de-duped
>> list of
>>>> GlobalCommT?
>>>>
>>>>
>>>> I think Iceberg sink needs to do the dedup in the `commit` call. The
>>>> `recoveredGlobalCommittables` is just for restoring the ids.
>>>>
>>>>
>>>>>> Sink implementation can decide if it wants to commit immediately or
>>>> just leave
>>>>
>>>> I think only the frame knows *when* call the commit function.
>>>>
>>>>
>>>>>> should this be "commit(List<GlobalCommT>)"?
>>>>
>>>> It could be. thanks.
>>>>
>>>>
>>>> Best,
>>>> Guowei
>>>>
>>>>
>>>> On Sun, Sep 20, 2020 at 12:11 AM Steven Wu <[hidden email]>
>> wrote:
>>>>
>>>>>> I prefer to let the developer produce id to dedupe. I think this
>> gives
>>>>> the developer more opportunity to optimize.
>>>>>
>>>>> Thinking about it again, I totally agree with Guowei on this. We don't
>>>>> really need the framework to generate the unique id for Iceberg sink.
>>>>> De-dup logic is totally internal to Iceberg sink and should be isolated
>>>>> inside. My earlier question regarding
>> "commitGlobally(List<GlobalCommT>)
>>>>> can be concurrent or not" also becomes irrelevant, as long as the
>> framework
>>>>> handles the GlobalCommT list properly (even with concurrent calls).
>>>>>
>>>>> Here are the things where framework can help
>>>>>
>>>>>     1. run global committer in jobmanager (e.g. like sink coordinator)
>>>>>     2. help with checkpointing, bookkeeping, commit failure handling,
>>>>>     recovery
>>>>>
>>>>>
>>>>> @Guowei Ma <[hidden email]> regarding the GlobalCommitter
>>>>> interface, I have some clarifying questions.
>>>>>
>>>>>> void recoveredGlobalCommittables(List<GlobalCommit> globalCommits)
>>>>>
>>>>>     1. You meant GlobalCommit -> GlobalCommT, right?
>>>>>     2. Is this called when restored from checkpoint/savepoint?
>>>>>     3.  Iceberg sink needs to do a dup check here on which GlobalCommT
>>>>>     were committed and which weren't. Should it return the
>> filtered/de-duped
>>>>>     list of GlobalCommT?
>>>>>     4. Sink implementation can decide if it wants to commit immediately
>>>>>     or just leave
>>>>>
>>>>>> void commit(GlobalCommit globalCommit);
>>>>>
>>>>> should this be "commit(List<GlobalCommT>)"?
>>>>>
>>>>> Thanks,
>>>>> Steven
>>>>>
>>>>>
>>>>> On Sat, Sep 19, 2020 at 1:56 AM Guowei Ma <[hidden email]>
>> wrote:
>>>>>
>>>>>> Hi, all
>>>>>>
>>>>>>>> Just to add to what Aljoscha said regarding the unique id. Iceberg
>>>>>> sink
>>>>>>>> checkpoints the unique id into state during snapshot. It also
>> inserts
>>>>>> the
>>>>>>>> unique id into the Iceberg snapshot metadata during commit. When a
>> job
>>>>>>>> restores the state after failure, it needs to know if the restored
>>>>>>>> transactions/commits were successful or not. It basically iterates
>>>>>> through
>>>>>>>> the list of table snapshots from Iceberg and matches the unique ids
>>>>>> with
>>>>>>>> what is stored in Iceberg snapshot metadata.
>>>>>>
>>>>>> Thanks Steven for these detailed explanations. It makes me know the
>>>>>> IceBerg
>>>>>> better. However, I prefer to let the developer produce id to dedupe. I
>>>>>> think this gives the developer more opportunity to optimize. You could
>>>>>> see
>>>>>> the following for more details. Please correct me if I misunderstand
>>>>>> you.
>>>>>>
>>>>>>>> 3. Whether the `Writer` supports async functionality or not.
>>>>>> Currently I
>>>>>> do
>>>>>>>> not know which sink could benefit from it. Maybe it is just my own
>>>>>> problem.
>>>>>>
>>>>>>>> Here, I don't really know. We can introduce an "isAvailable()"
>> method
>>>>>>>> and mostly ignore it for now and sinks can just always return true.
>>>>>> Or,
>>>>>>>> as an alternative, we don't add the method now but can add it later
>>>>>> with
>>>>>>>> a default implementation. Either way, we will probably not take
>>>>>>>> advantage of the "isAvailable()" now because that would require
>> more
>>>>>>>> runtime changes.
>>>>>>
>>>>>>  From the @Pitor's explanation I could see the other benefit that might
>>>>>> be
>>>>>> gained in the future. For example decoupling the task number and the
>>>>>> thread
>>>>>> number. But I have to admit that introducing `isAvailable` might
>>>>>> introduce
>>>>>> some complications in the runtime. You could see my alternative API
>>>>>> option
>>>>>> in the following. I believe that we could support such an async sink
>>>>>> writer
>>>>>> very easily in the future. What do you think?
>>>>>>
>>>>>>>> Yes, this is still tricky. What is the current state, would the
>>>>>>>> introduction of a "LocalCommit" and a "GlobalCommit" already solve
>>>>>> both
>>>>>>>> the Iceberg and Hive cases? I believe Hive is the most tricky one
>>>>>> here,
>>>>>>>> but if we introduce the "combine" method on GlobalCommit, that
>> could
>>>>>>>> serve the same purpose as the "aggregation operation" on the
>>>>>> individual
>>>>>>>> files, and we could even execute that "combine" in a distributed
>> way.
>>>>>>>> We assume that GlobalCommit is a Agg/Combiner?
>>>>>>
>>>>>> I would share what possible problems that I am seeing currently and
>> the
>>>>>> alternative options.
>>>>>>
>>>>>> ## IceBerg Sink
>>>>>>
>>>>>> ### Concern about generating nonce by framework.
>>>>>>
>>>>>> If let the `GlobalCommitter` provide a random nonce for the
>>>>>> `IceBergSink` I
>>>>>> think that it might not be efficient.  Because even if there are a
>> very
>>>>>> small number of committables in the state you still need to iterate
>> all
>>>>>> the
>>>>>> iceberg snapshot files to check whether the committable is committed
>>>>>> already. Even if it is efficient for the IceBergSink it might not be
>> the
>>>>>> case for other sinks.
>>>>>>
>>>>>> If the framework generates auto-increment nonce instead, it might
>> still
>>>>>> not
>>>>>> be optimal for users. For example, users might want to use some
>>>>>> business id
>>>>>> so that after failover they could query whether the commit is
>> successful
>>>>>> after failover.
>>>>>>
>>>>>> I think users could generate more efficient nonce such as an
>>>>>> auto-increment
>>>>>> one. Therefore, it seems to provide more optimization chances if we
>> let
>>>>>> users to generate the nonce.
>>>>>>
>>>>>>
>>>>>> ### Alternative Option
>>>>>>
>>>>>> public interface GlobalCommit<CommT, GlobalCommT> {
>>>>>>          // provide some runtime context such as
>>>>>> attempt-id,job-id,task-id.
>>>>>>          void open(InitContext context);
>>>>>>
>>>>>>          // This GlobalCommit would aggregate the committable to a
>>>>>> GlobalCommit before doing the commit operation.
>>>>>>          GlobalCommT combine(List<Committable> commitables)
>>>>>>
>>>>>>          // This method would be called after committing all the
>>>>>> GlobalCommit producing in the previous session.
>>>>>>          void recoveredGlobalCommittables(List<GlobalCommit>
>>>>>> globalCommits)
>>>>>>
>>>>>>          // developer would guarantee the idempotency by himself
>>>>>>          void commit(GlobalCommit globalCommit);
>>>>>> }
>>>>>>
>>>>>> User could guarantee the idenpointecy himself in a more efficient or
>>>>>> application specific way. If the user wants the `GlobalCommit` to be
>>>>>> executed in a distributed way, the user could use the runtime
>>>>>> information
>>>>>> to generate the partial order id himself.(We could ignore the clean up
>>>>>> first)
>>>>>>
>>>>>> Currently the sink might be looks like following:
>>>>>>
>>>>>> Sink<IN, LC, LCO, GC> {
>>>>>>          Writer<IN, LC> createWriter();
>>>>>>          Optional<Committer<LC, LCO>> createCommitter();
>>>>>>          Optional<GlobalCommitter<LCO, GC>> createGlobalCommitter();
>>>>>> }
>>>>>>
>>>>>> ## Hive
>>>>>>
>>>>>> The HiveSink needs to compute whether a directory is finished or not.
>>>>>> But
>>>>>> HiveSink can not use the above `combine` method to decide whether a
>>>>>> directory is finished or not.
>>>>>>
>>>>>> For example we assume that whether the directory is finished or not is
>>>>>> decided by the event time. There might be a topology that the source
>> and
>>>>>> sink are forward. The event time might be different in different
>>>>>> instances
>>>>>> of the `writer`. So the GlobalCommit’s combine can not produce a
>>>>>> GlobalCommT when the snapshot happens.
>>>>>>
>>>>>> In addition to the above case we should also consider the unaligned
>>>>>> checkpoint. Because the watermark does not skip. So there might be the
>>>>>> same
>>>>>> problem in the unaligned checkpoint.
>>>>>>
>>>>>> ### Option1:
>>>>>>
>>>>>> public interface GlobalCommit<CommT, GlobalCommT, StateT, ShareT> {
>>>>>>          // provide some runtime context such as
>>>>>> attempt-id,job-id,task-id,
>>>>>> maybe the event time;provide the restore state
>>>>>>          void open(InitContext context, StateT state);
>>>>>>
>>>>>>          // This is for the HiveSink. When all the writer say that the
>>>>>> the
>>>>>> bucket is finished it would return a GlobalCommitT
>>>>>>          Optional<GlobalCommT> combine(Committable commitables)
>>>>>>
>>>>>>          // This is for IcebergSink. Producing a GlobalCommitT every
>>>>>> checkpoint.
>>>>>>          Optional<GlobalCommT> preCommit();
>>>>>>
>>>>>>          // Maybe we need the shareState? After we decide the directory
>>>>>> we
>>>>>> make more detailed consideration then. The id could be remembered
>> here.
>>>>>>          StateT snapshotState();
>>>>>>
>>>>>>          // developer would guarantee the idempotency by himself
>>>>>>          void commit(GlobalCommit globalCommit);
>>>>>> }
>>>>>>
>>>>>> ### Option2
>>>>>>
>>>>>> Actually the `GlobalCommit` in the option1 mixes the `Writer` and
>>>>>> `Committer` together. So it is intuitive to decouple the two
>> functions.
>>>>>> For
>>>>>> support the hive we could prove a sink look like following
>>>>>>
>>>>>> Sink<In, LC, LCO, LCG> {
>>>>>>          Writer<In, LC> createWriter();
>>>>>>          Optional<Committer<LC, LCO>> createCommitter(); // we need
>> this
>>>>>> to
>>>>>> change name.
>>>>>>          Optional<Writer<LCO, LCG>> createGlobalAgg();
>>>>>>          Optional<Committer<LCG, void>> createGlobalCommitter();
>>>>>> }
>>>>>>
>>>>>> The pro of this method is that we use two basic concepts: `Committer`
>>>>>> and
>>>>>> `Writer` to build a HiveSink.
>>>>>>
>>>>>> ### CompactHiveSink / MergeHiveSink
>>>>>>
>>>>>> There are still other complicated cases, which are not satisfied by
>> the
>>>>>> above option. Users often complain about writing out many small files,
>>>>>> which will affect file reading efficiency and the performance and
>>>>>> stability
>>>>>> of the distributed file system. CompactHiveSink/MergeHiveSink hopes to
>>>>>> merge all files generated by this job in a single Checkpoint.
>>>>>>
>>>>>> The CompactHiveSink/MergeHiveSink topology can simply describe this
>>>>>> topology as follows:
>>>>>>
>>>>>> CompactSubTopology -> GlobalAgg -> GobalCommitter.
>>>>>>
>>>>>> The CompactSubTopology would look like following:
>>>>>>
>>>>>> TmpFileWriter -> CompactCoodinator -> CompactorFileWriter
>>>>>>
>>>>>> Maybe the topology could be simpler but please keep in mind I just
>> want
>>>>>> to
>>>>>> show that there might be very complicated topology requirements for
>>>>>> users.
>>>>>>
>>>>>>
>>>>>> A possible alternative option would be let the user build the topology
>>>>>> himself. But considering we have two execution modes we could only use
>>>>>> `Writer` and `Committer` to build the sink topology.
>>>>>>
>>>>>> ### Build Topology Option
>>>>>>
>>>>>> Sink<IN, OUT> {
>>>>>>          Sink<In, Out> addWriter(Writer<In, Out> Writer); // Maybe a
>>>>>> WriterBuidler
>>>>>>          Sink<In, Out> addCommitter(Committer<In, Out> committer); //
>>>>>> Maybe
>>>>>> we could make this return Void if we do not consider code reuse and
>>>>>> introduce the cleaner
>>>>>> }
>>>>>>
>>>>>> ## Summary
>>>>>> The requirements of sink might be different, maybe we could use two
>>>>>> basic
>>>>>> bricks(Writer/Committer) to let the user build their own sink
>> topology.
>>>>>> What do you guys think?
>>>>>>
>>>>>> I know the name stuff might be trikky for now but I want to discuss
>>>>>> these
>>>>>> things after we get the consus on the direction first.
>>>>>>
>>>>>> Best,
>>>>>> Guowei
>>>>>>
>>>>>>
>>>>>> On Sat, Sep 19, 2020 at 12:15 AM Steven Wu <[hidden email]>
>>>>>> wrote:
>>>>>>
>>>>>>> Aljoscha,
>>>>>>>
>>>>>>>> Instead the sink would have to check for each set of committables
>>>>>>> seperately if they had already been committed. Do you think this is
>>>>>>> feasible?
>>>>>>>
>>>>>>> Yes, that is how it works in our internal implementation [1]. We
>>>>>> don't use
>>>>>>> checkpointId. We generate a manifest file (GlobalCommT) to bundle
>> all
>>>>>> the
>>>>>>> data files that the committer received in one checkpoint cycle. Then
>>>>>> we
>>>>>>> generate a unique manifest id for by hashing the location of the
>>>>>> manifest
>>>>>>> file. The manifest ids are stored in Iceberg snapshot metadata. Upon
>>>>>>> restore, we check each of the restored manifest files against
>> Iceberg
>>>>>> table
>>>>>>> snapshot metadata to determine if we should discard or keep the
>>>>>> restored
>>>>>>> manifest files. If a commit has multiple manifest files (e.g.
>>>>>> accumulated
>>>>>>> from previous failed commits), we store the comma-separated manifest
>>>>>> ids in
>>>>>>> Iceberg snapshot metadata.
>>>>>>>
>>>>>>>> During normal operation this set would be very small, it would
>>>>>> usually
>>>>>>> only be the committables for the last checkpoint. Only when there is
>>>>>> an
>>>>>>> outage would multiple sets of committables pile up.
>>>>>>>
>>>>>>> You are absolutely right here. Even if there are multiple sets of
>>>>>>> committables, it is usually the last a few or dozen of snapshots we
>>>>>> need to
>>>>>>> check. Even with our current inefficient implementation of
>> traversing
>>>>>> all
>>>>>>> table snapshots (in the scale of thousands) from oldest to latest,
>> it
>>>>>> only
>>>>>>> took avg 60 ms and max 800 ms. so it is really not a concern for
>>>>>> Iceberg.
>>>>>>>
>>>>>>>> CommitStatus commitGlobally(List<Committable>, Nonce)
>>>>>>>
>>>>>>> Just to clarify on the terminology here. Assuming here the
>> Committable
>>>>>>> meant the `GlobalCommT` (like ManifestFile in Iceberg) in
>>>>>>> previous discussions, right? `CommT` means the Iceberg DataFile from
>>>>>> writer
>>>>>>> to committer.
>>>>>>>
>>>>>>> This can work assuming we *don't have concurrent executions
>>>>>>> of commitGlobally* even with concurrent checkpoints. Here is the
>>>>>> scenario
>>>>>>> regarding failure recovery I want to avoid.
>>>>>>>
>>>>>>> Assuming checkpoints 1, 2, 3 all completed. Each checkpoint
>> generates
>>>>>> a
>>>>>>> manifest file, manifest-1, 2, 3.
>>>>>>> timeline
>>>>>>>
>>>>>>
>> ------------------------------------------------------------------------->
>>>>>>> now
>>>>>>> commitGlobally(manifest-1, nonce-1) started
>>>>>>>           commitGlobally(manifest-2, nonce-2) started
>>>>>>>                      commitGlobally(manifest-2, nonce-2) failed
>>>>>>>                              commitGlobally(manifest-2 and
>> manifest-3,
>>>>>>> nonce-3) started
>>>>>>>                                      commitGlobally(manifest-1,
>>>>>> nonce-1)
>>>>>>> failed
>>>>>>>
>>   commitGlobally(manifest-2
>>>>>> and
>>>>>>> manifest-3, nonce-3) succeeded
>>>>>>>
>>>>>>> Now the job failed and was restored from checkpoint 3, which
>> contains
>>>>>>> manifest file 1,2,3. We found nonce-3 was committed when checking
>>>>>> Iceberg
>>>>>>> table snapshot metadata. But in this case we won't be able to
>>>>>> correctly
>>>>>>> determine which manifest files were committed or not.
>>>>>>>
>>>>>>> If it is possible to have concurrent executions of  commitGlobally,
>>>>>> the
>>>>>>> alternative is to generate the unique id/nonce per GlobalCommT. Then
>>>>>> we can
>>>>>>> check each individual GlobalCommT (ManifestFile) with Iceberg
>> snapshot
>>>>>>> metadata.
>>>>>>>
>>>>>>> Thanks,
>>>>>>> Steven
>>>>>>>
>>>>>>> [1]
>>>>>>>
>>>>>>>
>>>>>>
>> https://github.com/Netflix-Skunkworks/nfflink-connector-iceberg/blob/master/nfflink-connector-iceberg/src/main/java/com/netflix/spaas/nfflink/connector/iceberg/sink/IcebergCommitter.java#L569
>>>>>>>
>>>>>>> On Fri, Sep 18, 2020 at 2:44 AM Aljoscha Krettek <
>> [hidden email]
>>>>>>>
>>>>>>> wrote:
>>>>>>>
>>>>>>>> Steven,
>>>>>>>>
>>>>>>>> we were also wondering if it is a strict requirement that "later"
>>>>>>>> updates to Iceberg subsume earlier updates. In the current
>> version,
>>>>>> you
>>>>>>>> only check whether checkpoint X made it to Iceberg and then
>> discard
>>>>>> all
>>>>>>>> committable state from Flink state for checkpoints smaller X.
>>>>>>>>
>>>>>>>> If we go with a (somewhat random) nonce, this would not work.
>>>>>> Instead
>>>>>>>> the sink would have to check for each set of committables
>>>>>> seperately if
>>>>>>>> they had already been committed. Do you think this is feasible?
>>>>>> During
>>>>>>>> normal operation this set would be very small, it would usually
>>>>>> only be
>>>>>>>> the committables for the last checkpoint. Only when there is an
>>>>>> outage
>>>>>>>> would multiple sets of committables pile up.
>>>>>>>>
>>>>>>>> We were thinking to extend the GlobalCommitter interface to allow
>>>>>> it to
>>>>>>>> report success or failure and then let the framework retry. I
>> think
>>>>>> this
>>>>>>>> is something that you would need for the Iceberg case. The
>> signature
>>>>>>>> could be like this:
>>>>>>>>
>>>>>>>> CommitStatus commitGlobally(List<Committable>, Nonce)
>>>>>>>>
>>>>>>>> where CommitStatus could be an enum of SUCCESS, TERMINAL_FAILURE,
>>>>>> and
>>>>>>>> RETRY.
>>>>>>>>
>>>>>>>> Best,
>>>>>>>> Aljoscha
>>>>>>>>
>>>>>>>
>>>>>>
>>>>>
>>
>

Reply | Threaded
Open this post in threaded view
|

Re: [DISCUSS] FLIP-143: Unified Sink API

Steven Wu
Aljoscha/Guowei,

I think we are pretty close with aligning on the Iceberg sink requirements.
This new sink API can really benefit and simplify Iceberg sink
implementation. Looking forward to the initial scope with 1.12 release.

>   CommitResult commit(GlobalCommittableT);

I like the CommitResult return type. Since CommitResult can have RETRY,
which is probably the default behavior for commit failure. Framework would
accumulate a list of GlobalCommittableT. Then when the next checkpoint
happens, we will have more than one GlobalCommittableT item. Hence, I think
the commit method probably should take a list.

In addition, it is undesirable to do the committed-or-not check in the
commit method, which happens for each checkpoint cycle. CommitResult
already indicates SUCCESS or not. when framework calls commit with a list
of GlobalCommittableT, it should be certain they are uncommitted. The only
time we aren't sure is when a list of  GlobalCommittableT is restored from
a checkpoint. `*recoverGlobalCommittables*` is the ideal place to do such a
check and filter out the ones that were already committed. Retained ones
will be committed in the next checkpoint cycle. Since framework takes care
of the checkpoint and restore, we need some hook for the sink to add the
custom logic on the restored list.

Thanks,
Steven


On Mon, Sep 21, 2020 at 10:37 AM Aljoscha Krettek <[hidden email]>
wrote:

> Hi all,
>
> I'll try and summarize my thoughts after Guowei, Yun, Kostas, Dawid, and
> me had an offline discussion about this.
>
> Also, I would like to give credit to Guowei for initially coming up with
> the idea of a topology sink in the context of this discussion. I think
> it's a good idea and we should pursue it in the future. And yes, Beam
> already does it like this but I hadn't thought about it now when
> thinking about the sink APIs because having a more limited API gives
> more freedom to the framework.
>
> ## Topology Sink vs. Transactional Sink
>
>  From the discussion, it seems clear to me that to support all kinds of
> different use cases we will have to offer some sort of API that allows
> Sink developers to specify mostly arbitrary operator topologies. I
> think, however, that we will not manage to finish such a (semi
> user-facing) API within the 1.12 release cycle with satisfactory
> results. Therefore, I would say that we need to go with a more
> straightforward TransactionalSink API (name TBD) that lets sink
> developers specify basic Writer, Committer, GlobalCommitter components
> as discussed above.
>
> This Sink interface would initially support a FileSink that supports
> HDFS/S3 and behaves like the StreamingFileSink does for STREAM execution
> mode. Additionally, it would seamlessly work for BATCH execution mode.
> With the addition of a properly designed GlobalCommitter this should
> also work for Iceberg.
>
> It seems to me that the Hive use cases are still to fuzzy and not well
> defined to allow us to come up with a good solution.
>
> ## Committer vs. GlobalCommitter or both
>
> To make it short, we should make both optional but also allow both to be
> used by the same sink.
>
> The Committer is the interface that should be preferred because it
> allows the framework to distribute the work of committing, i.e. it has
> more potential for being optimised.
>
> Iceberg would use only a GlobalCommitter.
>
> The FileSink would use only Committer but can optionally use a
> GlobalCommitter to create a _SUCCESS file in the output directory to
> emulate Hadoop to some degree. Writing such a _SUCCESS file would only
> work in BATCH execution mode and it would write a _SUCCESS file in the
> toplevel output directory. Writing _SUCCESS files in individual buckets
> or subdirectories whenever these buckets are considered "done" is a
> different problem, and one I don't think we can solve well right now.
>
> Initially, I would propose these interfaces that have been floated by
> various people above:
>
> interface Committer<CommittableT> {
>    CommitResult commit(CommittableT);
> }
>
> interface GlobalCommitter<CommittableT, GlobalCommittableT> {
>    GlobalCommittableT merge(List<CommittableT>);
>    CommitResult commit(GlobalCommittableT);
> }
>
> enum CommitResult {
>    SUCCESS, FAILURE, RETRY
> }
>
> Alternatively, we could consider changing commit() to take a List<> to
> allow the sink to better check if commits are already in the external
> system. For example, Iceberg would check for the whole batch whether
> they are already committed.
>
> Also alternatively, we could change the GlobalCommitter to basically
> return an AggregateFunction instead of the simple merge() function.
>
> What do you think?
>
> Best,
> Aljoscha
>
> On 21.09.20 10:06, Piotr Nowojski wrote:
> > Hi Guowei,
> >
> >> I believe that we could support such an async sink writer
> >> very easily in the future. What do you think?
> >
> > How would you see the expansion in the future? Do you mean just adding
> > `isAvailable()` method with a default implementation later on?
> >
> > Piotrek
> >
> > pon., 21 wrz 2020 o 02:39 Steven Wu <[hidden email]> napisał(a):
> >
> >>> I think Iceberg sink needs to do the dedup in the `commit` call. The
> >> `recoveredGlobalCommittables` is just for restoring the ids.
> >>
> >>
> >> @Guowei Ma <[hidden email]>  It is undesirable to do the dedup
> check
> >> in the `commit` call, because it happens for each checkpoint cycle. We
> only
> >> need to do the de-dup check one time when restoring GlobalCommT list
> from
> >> the checkpoint.
> >>
> >>
> >> Can you clarify the purpose of `recoveredGlobalCommittables`? If it is
> to
> >> let sink implementations know the recovered GlobalCommT list, it is
> >> probably not a sufficient API. For the Iceberg sink, we can try to
> >> implement the de-dup check  inside the `recoveredGlobalCommittables`
> method
> >> and commit any uncommitted GlobalCommT items. But how do we handle the
> >> commit failed?
> >>
> >>
> >> One alternative is to allow sink implementations to override "Li
> >> st<GlobalCommT> recoverGlobalCommittables()". Framework handles the
> >> checkpoint/state, and sink implementations can further customize the
> >> restored list with de-dup check and filtering. Recovered uncommitted
> >> GlobalCommT list will be committed in the next cycle. It is the same
> >> rollover strategy for commit failure handling that we have been
> discussing.
> >>
> >>
> >> ## topologies
> >>
> >>
> >> Regarding the topology options, if we agree that there is no one size
> fit
> >> for all, we can let sink implementations choose the best topology. Maybe
> >> the framework can provide 2-3 pre-defined topology implementations to
> help
> >> the sinks.
> >>
> >>
> >>
> >>
> >> On Sun, Sep 20, 2020 at 3:27 AM Guowei Ma <[hidden email]> wrote:
> >>
> >>> I would like to summarize the file type sink in the thread and their
> >>> possible topologies.  I also try to give pros and cons of every
> topology
> >>> option. Correct me if I am wrong.
> >>>
> >>> ### FileSink
> >>>
> >>> Topology Option: TmpFileWriter + Committer.
> >>>
> >>> ### IceBerg Sink
> >>>
> >>> #### Topology Option1: `DataFileWriter` + `GlobalCommitterV0`.
> >>> Pro:
> >>> 1. Same group has some id.
> >>> Cons:
> >>> 1. May limit users’ optimization space;
> >>> 2. The topology does not meet the Hive’s requirements.
> >>>
> >>> #### Topology Option 2: `DataFileWriter` + `GlobalCommitterV1`
> >>> Pro:
> >>> 1. User has the opportunity to optimize the implementation of
> idempotence
> >>> Cons:
> >>> 2. Make the GlobalCommit more complicated.
> >>> 3. The topology does not meets the Hive’s requirements
> >>>
> >>> ### Topology Option3: DataFileWriter + AggWriter + Committer
> >>>
> >>> Pros:
> >>> 1. Use two basic `Writer` & `Commiter` to meet the IceBerge’s
> >> requirements.
> >>> 2. Opportunity to optimize the implementation of idempotence
> >>> 3. The topology meets the Hive’s requirements.(See flowing)
> >>> Con:
> >>> 1. It introduce a relative complex topologies
> >>>
> >>> ## HiveSink
> >>>
> >>> ### Topology Option1: `TmpFileWriter` + `Committer` +
> >> `GlobalCommitterV2`.
> >>> Pro:
> >>> 1. Could skip the cleanup problem at first.
> >>> Con:
> >>> 1. This style topology does not meet the CompactHiveSink requirements.
> >>>
> >>> ### Topology Option2: `TmpFileWriter` + `Committer` + `AggWriter` +
> >>> `Committer`
> >>> Pros
> >>> 1. Could skip the clean up problem at first.
> >>> 2. Decouple the GlobalCommitterV2 to `AggWriter` + `Committer`
> >>> Cons
> >>> 1. This style topology does not meet the CompactHiveSink requirements.
> >>> 2. There are two general `Committers` in the topology. For Hive’s case
> >>> there might be no problem. But there might be a problem in 1.12. For
> >>> example where to execute the sub-topology following the `Committer` in
> >>> batch execution mode for the general case. Because the topology is
> built
> >>> from `Writer` and `Committer` we might put all the sub-topology in the
> >>> OperatorCoordinator. But if the topology is too complicated it might be
> >>> very complicated. See following.
> >>>
> >>> ### Topology Option3 `FileWriter` + `AggWriter` + `Committer`
> >>> Pro
> >>> 1. There is only one general committer.
> >>> Cons
> >>> 1. It has to consider the cleanup problem. (In theory both the Option1
> >> and
> >>> Option2 need to cleanup)
> >>> 2. This style topology does not meet the CompactHiveSink requirements.
> >>> 3. Have to figure out how to make the current version compatible.
> >>>
> >>> ### CompactHiveSink/MergeHiveSink
> >>>
> >>> #### Topology Option1 `TmpFileWriter` + `Committer` +
> `MergerCoordinator`
> >>> + `MergeWriter` + `GlobalCommiterV2`
> >>> Pro
> >>> 1. Could skip the clean up problem at first.
> >>> Cons
> >>> 2. Where to execute the sub-topology following the `Committer`.
> >>>
> >>> #### Topology Option2 `TmpFileWriter` + `Committer` +
> `MergerCoordinator`
> >>> + `MergeWriter` + AggWriter + Committer
> >>> Pros
> >>> 1. Could skip the clean up problem at first
> >>> 2. Decouple the GlobalCommitterV2 to `AggWriter` + `Committer`
> >>> Con
> >>> 1. Where to execute the sub-topology following the `Committer`.
> >>>
> >>> ### Option3 FileWriter + MergeCoordinator + MergeFileWriter +
> Writer(Agg)
> >>> + Committer
> >>> Pro
> >>> 1. There is only one committer. It is very easy to support in the batch
> >>> execution mode.
> >>> Con
> >>> 2. It has to consider the cleanup problem. (In theory both the Option1
> >> and
> >>> Option2 need to cleanup)
> >>>
> >>>
> >>> ### Summary
> >>>
> >>>  From above we could divide the sink topology into two parts:
> >>> 1. Write topology.
> >>> 2. And One committer
> >>>
> >>> So we could provide a unified sink API looks like the following:
> >>>
> >>> public interface Sink<CommT> {
> >>>          List<Writer<?, ?>> getWriters();
> >>>          Committer<CommT> createCommitter()
> >>> }
> >>>
> >>> In the long run maybe we could give the user more powerful ability like
> >>> this (Currently some transformation still belongs to runtime):
> >>> Sink<CommT> {
> >>>          Transformation<CommT> createWriteTopology();
> >>>           CommitFunction<CommT> createCommitter();
> >>> }
> >>>
> >>> Best,
> >>> Guowei
> >>>
> >>>
> >>> On Sun, Sep 20, 2020 at 6:09 PM Guowei Ma <[hidden email]>
> wrote:
> >>>
> >>>> Hi, Stevn
> >>>> I want to make a clarification first, the following reply only
> considers
> >>>> the Iceberge sink, but does not consider other sinks.  Before make
> >> decision
> >>>> we should consider all the sink.I would try to summary all the sink
> >>>> requirments in the next mail
> >>>>
> >>>>
> >>>>>>   run global committer in jobmanager (e.g. like sink coordinator)
> >>>>
> >>>> I think it could be.
> >>>>
> >>>>
> >>>>>> You meant GlobalCommit -> GlobalCommT, right?
> >>>>
> >>>> Yes. Thanks :)
> >>>>
> >>>>
> >>>>>> Is this called when restored from checkpoint/savepoint?
> >>>>
> >>>> Yes.
> >>>>
> >>>>
> >>>>>> Iceberg sink needs to do a dup check here on which GlobalCommT were
> >>>> committed and which weren't. Should it return the filtered/de-duped
> >> list of
> >>>> GlobalCommT?
> >>>>
> >>>>
> >>>> I think Iceberg sink needs to do the dedup in the `commit` call. The
> >>>> `recoveredGlobalCommittables` is just for restoring the ids.
> >>>>
> >>>>
> >>>>>> Sink implementation can decide if it wants to commit immediately or
> >>>> just leave
> >>>>
> >>>> I think only the frame knows *when* call the commit function.
> >>>>
> >>>>
> >>>>>> should this be "commit(List<GlobalCommT>)"?
> >>>>
> >>>> It could be. thanks.
> >>>>
> >>>>
> >>>> Best,
> >>>> Guowei
> >>>>
> >>>>
> >>>> On Sun, Sep 20, 2020 at 12:11 AM Steven Wu <[hidden email]>
> >> wrote:
> >>>>
> >>>>>> I prefer to let the developer produce id to dedupe. I think this
> >> gives
> >>>>> the developer more opportunity to optimize.
> >>>>>
> >>>>> Thinking about it again, I totally agree with Guowei on this. We
> don't
> >>>>> really need the framework to generate the unique id for Iceberg sink.
> >>>>> De-dup logic is totally internal to Iceberg sink and should be
> isolated
> >>>>> inside. My earlier question regarding
> >> "commitGlobally(List<GlobalCommT>)
> >>>>> can be concurrent or not" also becomes irrelevant, as long as the
> >> framework
> >>>>> handles the GlobalCommT list properly (even with concurrent calls).
> >>>>>
> >>>>> Here are the things where framework can help
> >>>>>
> >>>>>     1. run global committer in jobmanager (e.g. like sink
> coordinator)
> >>>>>     2. help with checkpointing, bookkeeping, commit failure handling,
> >>>>>     recovery
> >>>>>
> >>>>>
> >>>>> @Guowei Ma <[hidden email]> regarding the GlobalCommitter
> >>>>> interface, I have some clarifying questions.
> >>>>>
> >>>>>> void recoveredGlobalCommittables(List<GlobalCommit> globalCommits)
> >>>>>
> >>>>>     1. You meant GlobalCommit -> GlobalCommT, right?
> >>>>>     2. Is this called when restored from checkpoint/savepoint?
> >>>>>     3.  Iceberg sink needs to do a dup check here on which
> GlobalCommT
> >>>>>     were committed and which weren't. Should it return the
> >> filtered/de-duped
> >>>>>     list of GlobalCommT?
> >>>>>     4. Sink implementation can decide if it wants to commit
> immediately
> >>>>>     or just leave
> >>>>>
> >>>>>> void commit(GlobalCommit globalCommit);
> >>>>>
> >>>>> should this be "commit(List<GlobalCommT>)"?
> >>>>>
> >>>>> Thanks,
> >>>>> Steven
> >>>>>
> >>>>>
> >>>>> On Sat, Sep 19, 2020 at 1:56 AM Guowei Ma <[hidden email]>
> >> wrote:
> >>>>>
> >>>>>> Hi, all
> >>>>>>
> >>>>>>>> Just to add to what Aljoscha said regarding the unique id. Iceberg
> >>>>>> sink
> >>>>>>>> checkpoints the unique id into state during snapshot. It also
> >> inserts
> >>>>>> the
> >>>>>>>> unique id into the Iceberg snapshot metadata during commit. When a
> >> job
> >>>>>>>> restores the state after failure, it needs to know if the restored
> >>>>>>>> transactions/commits were successful or not. It basically iterates
> >>>>>> through
> >>>>>>>> the list of table snapshots from Iceberg and matches the unique
> ids
> >>>>>> with
> >>>>>>>> what is stored in Iceberg snapshot metadata.
> >>>>>>
> >>>>>> Thanks Steven for these detailed explanations. It makes me know the
> >>>>>> IceBerg
> >>>>>> better. However, I prefer to let the developer produce id to
> dedupe. I
> >>>>>> think this gives the developer more opportunity to optimize. You
> could
> >>>>>> see
> >>>>>> the following for more details. Please correct me if I misunderstand
> >>>>>> you.
> >>>>>>
> >>>>>>>> 3. Whether the `Writer` supports async functionality or not.
> >>>>>> Currently I
> >>>>>> do
> >>>>>>>> not know which sink could benefit from it. Maybe it is just my own
> >>>>>> problem.
> >>>>>>
> >>>>>>>> Here, I don't really know. We can introduce an "isAvailable()"
> >> method
> >>>>>>>> and mostly ignore it for now and sinks can just always return
> true.
> >>>>>> Or,
> >>>>>>>> as an alternative, we don't add the method now but can add it
> later
> >>>>>> with
> >>>>>>>> a default implementation. Either way, we will probably not take
> >>>>>>>> advantage of the "isAvailable()" now because that would require
> >> more
> >>>>>>>> runtime changes.
> >>>>>>
> >>>>>>  From the @Pitor's explanation I could see the other benefit that
> might
> >>>>>> be
> >>>>>> gained in the future. For example decoupling the task number and the
> >>>>>> thread
> >>>>>> number. But I have to admit that introducing `isAvailable` might
> >>>>>> introduce
> >>>>>> some complications in the runtime. You could see my alternative API
> >>>>>> option
> >>>>>> in the following. I believe that we could support such an async sink
> >>>>>> writer
> >>>>>> very easily in the future. What do you think?
> >>>>>>
> >>>>>>>> Yes, this is still tricky. What is the current state, would the
> >>>>>>>> introduction of a "LocalCommit" and a "GlobalCommit" already solve
> >>>>>> both
> >>>>>>>> the Iceberg and Hive cases? I believe Hive is the most tricky one
> >>>>>> here,
> >>>>>>>> but if we introduce the "combine" method on GlobalCommit, that
> >> could
> >>>>>>>> serve the same purpose as the "aggregation operation" on the
> >>>>>> individual
> >>>>>>>> files, and we could even execute that "combine" in a distributed
> >> way.
> >>>>>>>> We assume that GlobalCommit is a Agg/Combiner?
> >>>>>>
> >>>>>> I would share what possible problems that I am seeing currently and
> >> the
> >>>>>> alternative options.
> >>>>>>
> >>>>>> ## IceBerg Sink
> >>>>>>
> >>>>>> ### Concern about generating nonce by framework.
> >>>>>>
> >>>>>> If let the `GlobalCommitter` provide a random nonce for the
> >>>>>> `IceBergSink` I
> >>>>>> think that it might not be efficient.  Because even if there are a
> >> very
> >>>>>> small number of committables in the state you still need to iterate
> >> all
> >>>>>> the
> >>>>>> iceberg snapshot files to check whether the committable is committed
> >>>>>> already. Even if it is efficient for the IceBergSink it might not be
> >> the
> >>>>>> case for other sinks.
> >>>>>>
> >>>>>> If the framework generates auto-increment nonce instead, it might
> >> still
> >>>>>> not
> >>>>>> be optimal for users. For example, users might want to use some
> >>>>>> business id
> >>>>>> so that after failover they could query whether the commit is
> >> successful
> >>>>>> after failover.
> >>>>>>
> >>>>>> I think users could generate more efficient nonce such as an
> >>>>>> auto-increment
> >>>>>> one. Therefore, it seems to provide more optimization chances if we
> >> let
> >>>>>> users to generate the nonce.
> >>>>>>
> >>>>>>
> >>>>>> ### Alternative Option
> >>>>>>
> >>>>>> public interface GlobalCommit<CommT, GlobalCommT> {
> >>>>>>          // provide some runtime context such as
> >>>>>> attempt-id,job-id,task-id.
> >>>>>>          void open(InitContext context);
> >>>>>>
> >>>>>>          // This GlobalCommit would aggregate the committable to a
> >>>>>> GlobalCommit before doing the commit operation.
> >>>>>>          GlobalCommT combine(List<Committable> commitables)
> >>>>>>
> >>>>>>          // This method would be called after committing all the
> >>>>>> GlobalCommit producing in the previous session.
> >>>>>>          void recoveredGlobalCommittables(List<GlobalCommit>
> >>>>>> globalCommits)
> >>>>>>
> >>>>>>          // developer would guarantee the idempotency by himself
> >>>>>>          void commit(GlobalCommit globalCommit);
> >>>>>> }
> >>>>>>
> >>>>>> User could guarantee the idenpointecy himself in a more efficient or
> >>>>>> application specific way. If the user wants the `GlobalCommit` to be
> >>>>>> executed in a distributed way, the user could use the runtime
> >>>>>> information
> >>>>>> to generate the partial order id himself.(We could ignore the clean
> up
> >>>>>> first)
> >>>>>>
> >>>>>> Currently the sink might be looks like following:
> >>>>>>
> >>>>>> Sink<IN, LC, LCO, GC> {
> >>>>>>          Writer<IN, LC> createWriter();
> >>>>>>          Optional<Committer<LC, LCO>> createCommitter();
> >>>>>>          Optional<GlobalCommitter<LCO, GC>> createGlobalCommitter();
> >>>>>> }
> >>>>>>
> >>>>>> ## Hive
> >>>>>>
> >>>>>> The HiveSink needs to compute whether a directory is finished or
> not.
> >>>>>> But
> >>>>>> HiveSink can not use the above `combine` method to decide whether a
> >>>>>> directory is finished or not.
> >>>>>>
> >>>>>> For example we assume that whether the directory is finished or not
> is
> >>>>>> decided by the event time. There might be a topology that the source
> >> and
> >>>>>> sink are forward. The event time might be different in different
> >>>>>> instances
> >>>>>> of the `writer`. So the GlobalCommit’s combine can not produce a
> >>>>>> GlobalCommT when the snapshot happens.
> >>>>>>
> >>>>>> In addition to the above case we should also consider the unaligned
> >>>>>> checkpoint. Because the watermark does not skip. So there might be
> the
> >>>>>> same
> >>>>>> problem in the unaligned checkpoint.
> >>>>>>
> >>>>>> ### Option1:
> >>>>>>
> >>>>>> public interface GlobalCommit<CommT, GlobalCommT, StateT, ShareT> {
> >>>>>>          // provide some runtime context such as
> >>>>>> attempt-id,job-id,task-id,
> >>>>>> maybe the event time;provide the restore state
> >>>>>>          void open(InitContext context, StateT state);
> >>>>>>
> >>>>>>          // This is for the HiveSink. When all the writer say that
> the
> >>>>>> the
> >>>>>> bucket is finished it would return a GlobalCommitT
> >>>>>>          Optional<GlobalCommT> combine(Committable commitables)
> >>>>>>
> >>>>>>          // This is for IcebergSink. Producing a GlobalCommitT every
> >>>>>> checkpoint.
> >>>>>>          Optional<GlobalCommT> preCommit();
> >>>>>>
> >>>>>>          // Maybe we need the shareState? After we decide the
> directory
> >>>>>> we
> >>>>>> make more detailed consideration then. The id could be remembered
> >> here.
> >>>>>>          StateT snapshotState();
> >>>>>>
> >>>>>>          // developer would guarantee the idempotency by himself
> >>>>>>          void commit(GlobalCommit globalCommit);
> >>>>>> }
> >>>>>>
> >>>>>> ### Option2
> >>>>>>
> >>>>>> Actually the `GlobalCommit` in the option1 mixes the `Writer` and
> >>>>>> `Committer` together. So it is intuitive to decouple the two
> >> functions.
> >>>>>> For
> >>>>>> support the hive we could prove a sink look like following
> >>>>>>
> >>>>>> Sink<In, LC, LCO, LCG> {
> >>>>>>          Writer<In, LC> createWriter();
> >>>>>>          Optional<Committer<LC, LCO>> createCommitter(); // we need
> >> this
> >>>>>> to
> >>>>>> change name.
> >>>>>>          Optional<Writer<LCO, LCG>> createGlobalAgg();
> >>>>>>          Optional<Committer<LCG, void>> createGlobalCommitter();
> >>>>>> }
> >>>>>>
> >>>>>> The pro of this method is that we use two basic concepts:
> `Committer`
> >>>>>> and
> >>>>>> `Writer` to build a HiveSink.
> >>>>>>
> >>>>>> ### CompactHiveSink / MergeHiveSink
> >>>>>>
> >>>>>> There are still other complicated cases, which are not satisfied by
> >> the
> >>>>>> above option. Users often complain about writing out many small
> files,
> >>>>>> which will affect file reading efficiency and the performance and
> >>>>>> stability
> >>>>>> of the distributed file system. CompactHiveSink/MergeHiveSink hopes
> to
> >>>>>> merge all files generated by this job in a single Checkpoint.
> >>>>>>
> >>>>>> The CompactHiveSink/MergeHiveSink topology can simply describe this
> >>>>>> topology as follows:
> >>>>>>
> >>>>>> CompactSubTopology -> GlobalAgg -> GobalCommitter.
> >>>>>>
> >>>>>> The CompactSubTopology would look like following:
> >>>>>>
> >>>>>> TmpFileWriter -> CompactCoodinator -> CompactorFileWriter
> >>>>>>
> >>>>>> Maybe the topology could be simpler but please keep in mind I just
> >> want
> >>>>>> to
> >>>>>> show that there might be very complicated topology requirements for
> >>>>>> users.
> >>>>>>
> >>>>>>
> >>>>>> A possible alternative option would be let the user build the
> topology
> >>>>>> himself. But considering we have two execution modes we could only
> use
> >>>>>> `Writer` and `Committer` to build the sink topology.
> >>>>>>
> >>>>>> ### Build Topology Option
> >>>>>>
> >>>>>> Sink<IN, OUT> {
> >>>>>>          Sink<In, Out> addWriter(Writer<In, Out> Writer); // Maybe a
> >>>>>> WriterBuidler
> >>>>>>          Sink<In, Out> addCommitter(Committer<In, Out> committer);
> //
> >>>>>> Maybe
> >>>>>> we could make this return Void if we do not consider code reuse and
> >>>>>> introduce the cleaner
> >>>>>> }
> >>>>>>
> >>>>>> ## Summary
> >>>>>> The requirements of sink might be different, maybe we could use two
> >>>>>> basic
> >>>>>> bricks(Writer/Committer) to let the user build their own sink
> >> topology.
> >>>>>> What do you guys think?
> >>>>>>
> >>>>>> I know the name stuff might be trikky for now but I want to discuss
> >>>>>> these
> >>>>>> things after we get the consus on the direction first.
> >>>>>>
> >>>>>> Best,
> >>>>>> Guowei
> >>>>>>
> >>>>>>
> >>>>>> On Sat, Sep 19, 2020 at 12:15 AM Steven Wu <[hidden email]>
> >>>>>> wrote:
> >>>>>>
> >>>>>>> Aljoscha,
> >>>>>>>
> >>>>>>>> Instead the sink would have to check for each set of committables
> >>>>>>> seperately if they had already been committed. Do you think this is
> >>>>>>> feasible?
> >>>>>>>
> >>>>>>> Yes, that is how it works in our internal implementation [1]. We
> >>>>>> don't use
> >>>>>>> checkpointId. We generate a manifest file (GlobalCommT) to bundle
> >> all
> >>>>>> the
> >>>>>>> data files that the committer received in one checkpoint cycle.
> Then
> >>>>>> we
> >>>>>>> generate a unique manifest id for by hashing the location of the
> >>>>>> manifest
> >>>>>>> file. The manifest ids are stored in Iceberg snapshot metadata.
> Upon
> >>>>>>> restore, we check each of the restored manifest files against
> >> Iceberg
> >>>>>> table
> >>>>>>> snapshot metadata to determine if we should discard or keep the
> >>>>>> restored
> >>>>>>> manifest files. If a commit has multiple manifest files (e.g.
> >>>>>> accumulated
> >>>>>>> from previous failed commits), we store the comma-separated
> manifest
> >>>>>> ids in
> >>>>>>> Iceberg snapshot metadata.
> >>>>>>>
> >>>>>>>> During normal operation this set would be very small, it would
> >>>>>> usually
> >>>>>>> only be the committables for the last checkpoint. Only when there
> is
> >>>>>> an
> >>>>>>> outage would multiple sets of committables pile up.
> >>>>>>>
> >>>>>>> You are absolutely right here. Even if there are multiple sets of
> >>>>>>> committables, it is usually the last a few or dozen of snapshots we
> >>>>>> need to
> >>>>>>> check. Even with our current inefficient implementation of
> >> traversing
> >>>>>> all
> >>>>>>> table snapshots (in the scale of thousands) from oldest to latest,
> >> it
> >>>>>> only
> >>>>>>> took avg 60 ms and max 800 ms. so it is really not a concern for
> >>>>>> Iceberg.
> >>>>>>>
> >>>>>>>> CommitStatus commitGlobally(List<Committable>, Nonce)
> >>>>>>>
> >>>>>>> Just to clarify on the terminology here. Assuming here the
> >> Committable
> >>>>>>> meant the `GlobalCommT` (like ManifestFile in Iceberg) in
> >>>>>>> previous discussions, right? `CommT` means the Iceberg DataFile
> from
> >>>>>> writer
> >>>>>>> to committer.
> >>>>>>>
> >>>>>>> This can work assuming we *don't have concurrent executions
> >>>>>>> of commitGlobally* even with concurrent checkpoints. Here is the
> >>>>>> scenario
> >>>>>>> regarding failure recovery I want to avoid.
> >>>>>>>
> >>>>>>> Assuming checkpoints 1, 2, 3 all completed. Each checkpoint
> >> generates
> >>>>>> a
> >>>>>>> manifest file, manifest-1, 2, 3.
> >>>>>>> timeline
> >>>>>>>
> >>>>>>
> >>
> ------------------------------------------------------------------------->
> >>>>>>> now
> >>>>>>> commitGlobally(manifest-1, nonce-1) started
> >>>>>>>           commitGlobally(manifest-2, nonce-2) started
> >>>>>>>                      commitGlobally(manifest-2, nonce-2) failed
> >>>>>>>                              commitGlobally(manifest-2 and
> >> manifest-3,
> >>>>>>> nonce-3) started
> >>>>>>>                                      commitGlobally(manifest-1,
> >>>>>> nonce-1)
> >>>>>>> failed
> >>>>>>>
> >>   commitGlobally(manifest-2
> >>>>>> and
> >>>>>>> manifest-3, nonce-3) succeeded
> >>>>>>>
> >>>>>>> Now the job failed and was restored from checkpoint 3, which
> >> contains
> >>>>>>> manifest file 1,2,3. We found nonce-3 was committed when checking
> >>>>>> Iceberg
> >>>>>>> table snapshot metadata. But in this case we won't be able to
> >>>>>> correctly
> >>>>>>> determine which manifest files were committed or not.
> >>>>>>>
> >>>>>>> If it is possible to have concurrent executions of  commitGlobally,
> >>>>>> the
> >>>>>>> alternative is to generate the unique id/nonce per GlobalCommT.
> Then
> >>>>>> we can
> >>>>>>> check each individual GlobalCommT (ManifestFile) with Iceberg
> >> snapshot
> >>>>>>> metadata.
> >>>>>>>
> >>>>>>> Thanks,
> >>>>>>> Steven
> >>>>>>>
> >>>>>>> [1]
> >>>>>>>
> >>>>>>>
> >>>>>>
> >>
> https://github.com/Netflix-Skunkworks/nfflink-connector-iceberg/blob/master/nfflink-connector-iceberg/src/main/java/com/netflix/spaas/nfflink/connector/iceberg/sink/IcebergCommitter.java#L569
> >>>>>>>
> >>>>>>> On Fri, Sep 18, 2020 at 2:44 AM Aljoscha Krettek <
> >> [hidden email]
> >>>>>>>
> >>>>>>> wrote:
> >>>>>>>
> >>>>>>>> Steven,
> >>>>>>>>
> >>>>>>>> we were also wondering if it is a strict requirement that "later"
> >>>>>>>> updates to Iceberg subsume earlier updates. In the current
> >> version,
> >>>>>> you
> >>>>>>>> only check whether checkpoint X made it to Iceberg and then
> >> discard
> >>>>>> all
> >>>>>>>> committable state from Flink state for checkpoints smaller X.
> >>>>>>>>
> >>>>>>>> If we go with a (somewhat random) nonce, this would not work.
> >>>>>> Instead
> >>>>>>>> the sink would have to check for each set of committables
> >>>>>> seperately if
> >>>>>>>> they had already been committed. Do you think this is feasible?
> >>>>>> During
> >>>>>>>> normal operation this set would be very small, it would usually
> >>>>>> only be
> >>>>>>>> the committables for the last checkpoint. Only when there is an
> >>>>>> outage
> >>>>>>>> would multiple sets of committables pile up.
> >>>>>>>>
> >>>>>>>> We were thinking to extend the GlobalCommitter interface to allow
> >>>>>> it to
> >>>>>>>> report success or failure and then let the framework retry. I
> >> think
> >>>>>> this
> >>>>>>>> is something that you would need for the Iceberg case. The
> >> signature
> >>>>>>>> could be like this:
> >>>>>>>>
> >>>>>>>> CommitStatus commitGlobally(List<Committable>, Nonce)
> >>>>>>>>
> >>>>>>>> where CommitStatus could be an enum of SUCCESS, TERMINAL_FAILURE,
> >>>>>> and
> >>>>>>>> RETRY.
> >>>>>>>>
> >>>>>>>> Best,
> >>>>>>>> Aljoscha
> >>>>>>>>
> >>>>>>>
> >>>>>>
> >>>>>
> >>
> >
>
>
1234