You are viewing a plain text version of this content. The canonical link for it is here.
Posted to user@flink.apache.org by Guowei Ma <gu...@gmail.com> on 2020/09/10 13:43:58 UTC

[DISCUSS] FLIP-143: Unified Sink API

Hi, devs & users

As discussed in FLIP-131[1], Flink will deprecate the DataSet API in favor
of DataStream API and Table API. Users should be able to use DataStream API
to write jobs that support both bounded and unbounded execution modes.
However, Flink does not provide a sink API to guarantee the Exactly-once
semantics in both bounded and unbounded scenarios, which blocks the
unification.

So we want to introduce a new unified sink API which could let the user
develop the sink once and run it everywhere. You could find more details in
FLIP-143[2].

The FLIP contains some open questions that I'd really appreciate inputs
from the community. Some of the open questions include:

   1. We provide two alternative Sink API in the FLIP. The only difference
   between the two versions is how to expose the state to the user. We want to
   know which one is your preference?
   2. How does the sink API support to write to the Hive?
   3. Is the sink an operator or a topology?

[1]
https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=158866741
[2]
https://cwiki.apache.org/confluence/display/FLINK/FLIP-143%3A+Unified+Sink+API

Best,
Guowei

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

Posted by Piotr Nowojski <pn...@apache.org>.
Hi,

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].

I'm aware that many implementations wouldn't be able to implement it, but
some may. For example `FlinkKafkaProducer` could
use `FlinkKafkaProducer#pendingRecords` to control `Writer`'s availability.
Also any sink that would be implementing records handover to some writer
thread could easily provide availability as well.

Non blocking calls are important for many things, for example they are
crucial for unaligned checkpoints to complete quickly.

Piotrek

[1] org.apache.flink.api.connector.source.SourceReader#isAvailable
[2] org.apache.flink.runtime.io.AvailabilityProvider

pon., 14 wrz 2020 o 01:23 Steven Wu <st...@gmail.com> napisał(a):

> Aljoscha, thanks a lot for the detailed response. Now I have a better
> understanding of the initial scope.
>
> To me, there are possibly three different committer behaviors. For the lack
> of better names, let's call them
> * No/NoopCommitter
> * Committer / LocalCommitter (file sink?)
> * GlobalCommitter (Iceberg)
>
> ## 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.
> -----------------
> // before barrier emitted to downstream
> void prepareSnapshot(long checkpointId) throws Exception;
>
> // or will flush be called automatically before the barrier emitted
> downstream?
> // if yes, we need the checkpointId arg for the reason listed in [1]
> void flush(WriterOutput<CommT> output) throws IOException;
>
> 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.
> For that reason, writers need to know the checkpointId where the restore
> happened. Can we add a RestoreContext interface to the restoreWriter
> method?
> ---------------
> Writer<IN, CommT, WriterS, SharedS> restoreWriter(InitContext context,
> RestoreContext restoreContext, List<WriterS> state, List<SharedS> share);
>
> interface RestoreContext {
>   long getCheckpointId();
> }
>
>
> ## Committer interface
>
> 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.
> ------------------
> void commit(CommT committable) throws Exception;
>       ==>
> void collect(CommT committable) throws Exception;
> void commit() throws Exception;
>
> As discussed in [1] and mentioned above, the Iceberg committer needs to
> know which checkpointId is the commit for. So can we add checkpiontId arg
> to the commit API. However, I don't know how this would affect the batch
> execution where checkpoints are disabled.
> ------------------
> void commit(long checkpointId) throws Exception;
>
> 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);
>
>
> Thanks,
> Steven
>
> [1] https://github.com/apache/iceberg/pull/1185#discussion_r479589663
> [2] https://github.com/apache/iceberg/pull/1185#discussion_r479457104
>
>
>
> On Fri, Sep 11, 2020 at 3:27 AM Aljoscha Krettek <al...@apache.org>
> wrote:
>
> > Regarding the FLIP itself, I like the motivation section and the
> > What/How/When/Where section a lot!
> >
> > I don't understand why we need the "Drain and Snapshot" section. It
> > seems to be some details about stop-with-savepoint and drain, and the
> > relation to BATCH execution but I don't know if it is needed to
> > understand the rest of the document. I'm happy to be wrong here, though,
> > if there's good reasons for the section.
> >
> > On the question of Alternative 1 and 2, I have a strong preference for
> > Alternative 1 because we could avoid strong coupling to other modules.
> > With Alternative 2 we would depend on `flink-streaming-java` and even
> > `flink-runtime`. For the new source API (FLIP-27) we managed to keep the
> > dependencies slim and the code is in flink-core. I'd be very happy if we
> > can manage the same for the new sink API.
> >
> > Best,
> > Aljoscha
> >
> > On 11.09.20 12:02, Aljoscha Krettek wrote:
> > > Hi Everyone,
> > >
> > > thanks to Guowei for publishing the FLIP, and thanks Steven for the
> very
> > > thoughtful email!
> > >
> > > We thought a lot internally about some of the questions you posted but
> > > left a lot (almost all) of the implementation details out of the FLIP
> > > for now because we wanted to focus on semantics and API. I'll try and
> > > address the points below.
> > >
> > > ## Initial Scope of the new Sink API
> > >
> > > We need to accept some initial scope that we want to achieve for Flink
> > > 1.12. I don't think we can try and find the solution that will work for
> > > all current and future external systems. For me, the initial goal would
> > > be to produce a Sink API and implementations for systems where you can
> > > prepare "committables" in one process and commit those from another
> > > process. Those are systems that support "real" transactions as you need
> > > them in a two-phase commit protocol. This includes:
> > >
> > >   - File Sink, including HDFS, S3 via special part-file uploads
> > >   - Iceberg
> > >   - HDFS
> > >
> > > The work should include runtime support for both BATCH and STREAMING
> > > execution as outlined in https://s.apache.org/FLIP-134.
> > >
> > > Supporting Kafka already becomes difficult but I'll go into that below.
> > >
> > > ## Where to run the Committer
> > >
> > > Guowei hinted at this in the FLIP: the idea is that the Framework
> > > decides where to run the committer based on requirements and based on
> > > the execution mode (STREAMING or BATCH).
> > >
> > > Something that is not in the FLIP but which we thought about is that we
> > > need to allow different types of committers. I'm currently thinking we
> > > need at least a normal "Committer" and a "GlobalCommiter" (name TBD).
> > >
> > > 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.
> > >
> > > An interesting read on this topic is the discussion on
> > > https://issues.apache.org/jira/browse/MAPREDUCE-4815. About the Hadoop
> > > FileOutputCommitter and the two different available algorithms for
> > > committing final job/task results.
> > >
> > > These interfaces untie the sink implementation from the Runtime and we
> > > could, for example, have a runtime like this:
> > >
> > > ### BATCH
> > >
> > >   - Collect all committables and store them in a fault tolerant way
> > > until the job finishes
> > >   - For a normal Commit function, call it on the individual commits. We
> > > can potentially distribute this if it becomes a bottleneck
> > >   - For GlobalCommit function, call it will all the commits. This
> cannot
> > > be distributed
> > >
> > > We can collect the committables in an OperatorCoordinator or
> potentially
> > > somehow in a task. Though I prefer an OperatorCoordinator right now.
> The
> > > operator coordinator needs to keep the commits in a fault-tolerant way.
> > >
> > > ### STREAMING
> > >
> > >   - For normal Commit, keep the committables in state on the individual
> > > tasks, commit them when a checkpoint completes
> > >   - For global CommitFunction we have options: collect them in a DOP-1
> > > operator in the topology or send them to an OperatorCoordinator to do
> > > the commit there. This is where the source/sink duality that Steven
> > > mentions becomes visible.
> > >
> > > ## Kafka
> > >
> > > Kafka is a problematic case because it doesn't really support
> > > transactions as outlined above. Our current Sink implementations works
> > > around that with hacks but that only gets us so far.
> > >
> > > The problem with Kafka is that we need to aggressively clean up pending
> > > transactions in case a failure happens. Otherwise stale transactions
> > > would block downstream consumers. See here for details:
> > > http://kafka.apache.org/documentation/#isolation.level.
> > >
> > > The way we solve this in the current Kafka sink is by using a fixed
> pool
> > > of transactional IDs and then cancelling all outstanding transactions
> > > for the IDs when we restore from a savepoint. In order for this to work
> > > we need to recycle the IDs, so there needs to be a back-channel from
> the
> > > Committer to the Writter, or they need to share internal state.
> > >
> > > I don't get see a satisfying solution for this so I think we should
> > > exclude this from the initial version.
> > >
> > > ## On Write-Ahead-Log Sinks
> > >
> > > Some sinks, like ES or Cassandra would require that we keep a WAL in
> > > Flink and then ship the contents to the external system on checkpoint.
> > > The reason is that these systems don't support real transactions where
> > > you can prepare them in one process and commit them from another
> process.
> > >
> > > Best,
> > > Aljoscha
> > >
> > >
> > > On 11.09.20 02:23, Steven Wu wrote:
> > >> Guowei,
> > >>
> > >> Thanks a lot for the proposal and starting the discussion thread. Very
> > >> excited.
> > >>
> > >> For the big question of "Is the sink an operator or a topology?", I
> > >> have a
> > >> few related sub questions.
> > >> * Where should we run the committers?
> > >> * Is the committer parallel or single parallelism?
> > >> * Can a single choice satisfy all sinks?
> > >>
> > >> Trying to envision how some sinks can be implemented with this new
> > >> unified
> > >> sink interface.
> > >>
> > >> 1. Kafka sink
> > >>
> > >> Kafka supports non-transactional and transactional writes
> > >> * Non-transaction writes don't need commit action. we can have
> *parallel
> > >> writers and no/no-op committers*. This is probably true for other
> > >> non-transactional message queues.
> > >> * Transaction writes can be implemented as *parallel writers and
> > parallel
> > >> committers*. In this case, I don't know if it makes sense to separate
> > >> writers and committers into two separate operators, because they
> > probably
> > >> need to share the same KafkaProducer object.
> > >>
> > >> Either way, both writers and committers probably should *run inside
> task
> > >> managers*.
> > >>
> > >> 2. ES sink
> > >>
> > >> ES sink typically buffers the data up to a certain size or time
> > threshold
> > >> and then uploads/commits a batch to ES. Writers buffer data and flush
> > >> when
> > >> needed, and committer does the HTTP bulk upload to commit. To avoid
> > >> serialization/deserialization cost, we should run *parallel writers
> and
> > >> parallel committers* and they *should be* *chained or bundled
> together*
> > >> while *running inside task managers*.
> > >>
> > >> It can also be implemented as *parallel writers and no/no-op
> > committers*,
> > >> where all logics (batching and upload) are put inside the writers.
> > >>
> > >> 3. Iceberg [1] sink
> > >>
> > >> It is currently implemented as two-stage operators with *parallel
> > writers
> > >> and single-parallelism committers*.
> > >> * *parallel writers* that write records into data files. Upon
> > checkpoint,
> > >> writers flush and upload the files, and send the metadata/location of
> > the
> > >> data files to the downstream committer. Writers need to do the flush
> > >> inside
> > >> the "prepareSnapshotPreBarrier" method (NOT "snapshotState" method)
> > >> before
> > >> forwarding the checkpoint barrier to the committer
> > >> * single-parallelism committer operator. It collects data files from
> > >> upstream writers. During "snapshotState", it saves collected data
> > >> files (or
> > >> an uber metadata file) into state. When the checkpoint is completed,
> > >> inside
> > >> "notifyCheckpointComplete" it commits those data files to Iceberg
> > >> tables. *The
> > >> committer has to be single parallelism*, because we don't want
> > >> hundreds or
> > >> thousands of parallel committers to compete for commit operations with
> > >> opportunistic concurrency control. It will be very inefficient and
> > >> probably
> > >> infeasible if the parallelism is high. Too many tiny
> > commits/transactions
> > >> can also slow down both the table write and read paths due to too many
> > >> manifest files.
> > >>
> > >> Right now, both Iceberg writer and committer operators run inside task
> > >> managers. It has one major drawback. With Iceberg sink, embarrassingly
> > >> parallel jobs won't be embarrassingly parallel anymore. That breaks
> the
> > >> benefit of region recovery for embarrassingly parallel DAG.
> > Conceptually,
> > >> the Writer-Committer sink pattern is like the mirroring of the FLIP-27
> > >> Enumerator-Reader source pattern. It will be better *if the committer
> > can
> > >> run inside the job manager* like the SplitEnumerator for the FLIP-27
> > >> source.
> > >>
> > >> -----------------------
> > >> Additional questions regarding the doc/API
> > >> * Any example for the writer shared state
> (Writer#snapshotSharedState)?
> > >> * We allow the case where the writer has no state, right? Meaning
> > WriterS
> > >> can be Void.
> > >>
> > >> [1] https://iceberg.apache.org/
> > >>
> > >> Thanks,
> > >> Steven
> > >>
> > >> On Thu, Sep 10, 2020 at 6:44 AM Guowei Ma <gu...@gmail.com>
> wrote:
> > >>
> > >>> Hi, devs & users
> > >>>
> > >>> As discussed in FLIP-131[1], Flink will deprecate the DataSet API in
> > >>> favor
> > >>> of DataStream API and Table API. Users should be able to use
> > >>> DataStream API
> > >>> to write jobs that support both bounded and unbounded execution
> modes.
> > >>> However, Flink does not provide a sink API to guarantee the
> > Exactly-once
> > >>> semantics in both bounded and unbounded scenarios, which blocks the
> > >>> unification.
> > >>>
> > >>> So we want to introduce a new unified sink API which could let the
> user
> > >>> develop the sink once and run it everywhere. You could find more
> > >>> details in
> > >>> FLIP-143[2].
> > >>>
> > >>> The FLIP contains some open questions that I'd really appreciate
> inputs
> > >>> from the community. Some of the open questions include:
> > >>>
> > >>>     1. We provide two alternative Sink API in the FLIP. The only
> > >>>     difference between the two versions is how to expose the state to
> > >>> the user.
> > >>>     We want to know which one is your preference?
> > >>>     2. How does the sink API support to write to the Hive?
> > >>>     3. Is the sink an operator or a topology?
> > >>>
> > >>> [1]
> > >>>
> >
> https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=158866741
> > >>>
> > >>> [2]
> > >>>
> >
> https://cwiki.apache.org/confluence/display/FLINK/FLIP-143%3A+Unified+Sink+API
> > >>>
> > >>>
> > >>> Best,
> > >>> Guowei
> > >>>
> > >>
> > >
> >
> >
>

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

Posted by Aljoscha Krettek <al...@apache.org>.
On 15.09.20 06:05, Guowei Ma wrote:
> ## Using checkpointId
> In the batch execution mode there would be no normal checkpoint any more.
> That is why we do not introduce the checkpoint id in the API. So it is a
> great thing that sink decouples its implementation from checkpointid. :)

Yes, this is a very important point of the design!

On 15.09.20 06:28, Guowei Ma wrote:
> I am open to alternative 1. Maybe I miss something but I do not get why the
> second alternative would depend on `flink-runtime` or
> `flink-streaming-java`. The all the state api currently is in the
> flink-core. Could you give some further explanation?  thanks :)

You're right, yes. It seems I was thinking about other things. I also 
prefer alternative 1 because it's more "declarative", i.e. the sink has 
a fixed method where it can return something. Instead of the framework 
giving an interface to the sink that it can then use in arbitrary ways. 
Alternative 1 leaves more freedom to the framework but limits the sink. 
Alternative 2 leaves more freedom to the sink but limits the framework, 
potentially.

Best,
Aljoscha




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

Posted by Guowei Ma <gu...@gmail.com>.
Hi, aljoscha

>I don't understand why we need the "Drain and Snapshot" section. It
>seems to be some details about stop-with-savepoint and drain, and the
>relation to BATCH execution but I don't know if it is needed to
>understand the rest of the document. I'm happy to be wrong here, though,
>if there's good reasons for the section.

The new unified sink API should provide a way for the sink developer to
deal with EOI(Drain) to guarantee the Exactly-once semantics. This is what
I want to say mostly in this section. Current streaming style sink API does
not provide a good way to deal with it. It is why the `StreamingFileSink`
does not commit the last part of data in the bounded scenario. Our theme is
unified. I am afraid that I will let users misunderstand that adding this
requirement to the new sink API is only for bounded scenarios, so I
explained in this paragraph that stop-with-savepoint might also have the
similar requirement.

For the snapshot I also want to prevent users from misunderstanding that it
is specially prepared for the unbounded scenario. Actually it might be also
possible with bounded + batch execution mode in the future.

I could reorganize the section if this section makes the reader confused
but I think we might need to keep the drain at least. WDYT?

>On the question of Alternative 1 and 2, I have a strong preference for
>Alternative 1 because we could avoid strong coupling to other modules.
>With Alternative 2 we would depend on `flink-streaming-java` and even
>`flink-runtime`. For the new source API (FLIP-27) we managed to keep the
>dependencies slim and the code is in flink-core. I'd be very happy if we
>can manage the same for the new sink API.

I am open to alternative 1. Maybe I miss something but I do not get why the
second alternative would depend on `flink-runtime` or
`flink-streaming-java`. The all the state api currently is in the
flink-core. Could you give some further explanation?  thanks :)

Best,
Guowei


On Tue, Sep 15, 2020 at 12:05 PM Guowei Ma <gu...@gmail.com> wrote:

> ## Concurrent checkpoints
> AFAIK the committer would not see the file-1-2 when ck1 happens in the
> ExactlyOnce mode.
>
> ## Committable bookkeeping and combining
>
> I agree with you that the "CombineGlobalCommitter" would work. But we put
> more optimization logic in the committer, which will make the committer
> more and more complicated, and eventually become the same as the
> Writer. For example, The committer needs to clean up some unused manifest
> file when restoring from a failure if we introduce the optimizations to the
> committer.
>
> In this case another alternative might be to put this "merging"
> optimization to a separate agg operator(maybe just like another `Writer`?).
> The agg could produce an aggregated committable to the committer. The agg
> operator could manage the whole life cycle of the manifest file it created.
> It would make the committer have single responsibility.
>
> >>The main question is if this pattern is generic to be put into the sink
> framework or not.
> Maybe I am wrong. But what I can feel from the current discussion is that
> different requirements have different topological requirements.
>
> ## Using checkpointId
> In the batch execution mode there would be no normal checkpoint any more.
> That is why we do not introduce the checkpoint id in the API. So it is a
> great thing that sink decouples its implementation from checkpointid. :)
>
> Best,
> Guowei
>
>
> On Tue, Sep 15, 2020 at 7:33 AM Steven Wu <st...@gmail.com> wrote:
>
>>
>> ## concurrent checkpoints
>>
>> @Aljoscha Krettek <al...@apache.org> regarding the concurrent
>> checkpoints, let me illustrate with a simple DAG below.
>> [image: image.png]
>>
>> Let's assume each writer emits one file per checkpoint cycle and *writer-2
>> is slow*. Now let's look at what the global committer receives
>>
>> timeline:
>> ----------------------------------------------------------> Now
>> from Writer-1:      file-1-1, ck-1, file-1-2, ck-2
>> from Writer-2:
>> file-2-1, ck-1
>>
>> In this case, the committer shouldn't include "file-1-2" into the commit
>> for ck-1.
>>
>> ## Committable bookkeeping and combining
>>
>> I like David's proposal where the framework takes care of the
>> bookkeeping of committables and provides a combiner API (CommT ->
>> GlobalCommT) for GlobalCommitter. The only requirement is to tie the
>> commit/CommT/GlobalCommT to a checkpoint.
>>
>> When a commit is successful for checkpoint-N, the framework needs to
>> remove the GlobalCommT from the state corresponding to checkpoints <= N. If
>> a commit fails, the GlobalCommT accumulates and will be included in the
>> next cycle. That is how the Iceberg sink works. I think it is good to
>> piggyback retries with Flink's periodic checkpoints for Iceberg sink.
>> Otherwise, it can get complicated to implement retry logic that won't
>> interfere with Flink checkpoints.
>>
>> The main question is if this pattern is generic to be put into the sink
>> framework or not.
>>
>> > A alternative topology option for the IcebergSink might be :
>> DataFileWriter
>> -> Agg -> GlobalCommitter. One pro of this method is that we can let Agg
>> take care of the cleanup instead of coupling the cleanup logic to the
>> committer.
>>
>> @Guowei Ma <gu...@gmail.com> I would favor David's suggestion of
>> "combine" API rather than a separate "Agg" operator.
>>
>> ## Using checkpointId
>>
>> > I think this can have some problems, for example when checkpoint ids are
>> not strictly sequential, when we wrap around, or when the JobID changes.
>> This will happen when doing a stop/start-from-savepoint cycle, for
>> example.
>>
>> checkpointId can work if it is monotonically increasing, which I believe
>> is the case for Flink today. Restoring from checkpoint or savepoint will
>> resume the checkpointIds.
>>
>> We can deal with JobID change by saving it into the state and Iceberg
>> snapshot metadata. There is already a PR [1] for that.
>>
>> ## Nonce
>>
>> > Flink provide a nonce to the GlobalCommitter where Flink guarantees
>> that this nonce is unique
>>
>> That is actually how we implemented internally. Flink Iceberg sink
>> basically hashes the Manifest file location as the nonce. Since the Flink
>> generated Manifest file location is unique, it  guarantees the nonce is
>> unique.
>>
>> IMO, checkpointId is also one way of implementing Nonce based on today's
>> Flink behavior.
>>
>> > and will not change for repeated invocations of the GlobalCommitter
>> with the same set of committables
>>
>>  if the same set of committables are combined into one GlobalCommT (like
>> ManifestFile in Iceberg), then the Nonce could be part of the GlobalCommT
>> interface.
>>
>> BTW, as David pointed out, the ManifestFile optimization is only in our
>> internal implementation [2] right now. For the open source version, there
>> is a github issue [3] to track follow-up improvements.
>>
>> Thanks,
>> Steven
>>
>> [1] https://github.com/apache/iceberg/pull/1404
>> [2]
>> 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#L363
>> [3] https://github.com/apache/iceberg/issues/1403
>>
>>
>> On Mon, Sep 14, 2020 at 12:03 PM Guowei Ma <gu...@gmail.com> wrote:
>>
>>> Hi all,
>>>
>>>
>>> Very thanks for the discussion and the valuable opinions! Currently there
>>> are several ongoing issues and we would like to show what we are thinking
>>> in the next few mails.
>>>
>>> It seems that the biggest issue now is about the topology of the sinks.
>>> Before deciding what the sink API would look like, I would like to first
>>> summarize the different topologies we have mentioned so that we could
>>> sync
>>> on the same page and gain more insights about this issue. There are four
>>> types of topology I could see. Please correct me if I misunderstand what
>>> you mean:
>>>
>>>    1.
>>>
>>>    Commit individual files. (StreamingFileSink)
>>>    1.
>>>
>>>       FileWriter -> FileCommitter
>>>       2.
>>>
>>>    Commit a directory (HiveSink)
>>>    1.
>>>
>>>       FileWriter -> FileCommitter -> GlobalCommitter
>>>       3.
>>>
>>>    Commit a bundle of files (Iceberg)
>>>    1.
>>>
>>>       DataFileWriter  -> GlobalCommitter
>>>       4.
>>>
>>>    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
>>>
>>>
>>> It can be seen from the above that the topologies are different according
>>> to different requirements. Not only that there may be other options for
>>> the
>>> second and third categories. E.g
>>>
>>> A alternative topology option for the IcebergSink might be :
>>> DataFileWriter
>>> -> Agg -> GlobalCommitter. One pro of this method is that we can let Agg
>>> take care of the cleanup instead of coupling the cleanup logic to the
>>> committer.
>>>
>>>
>>> In the long run I think we might provide the sink developer the ability
>>> to
>>> build arbitrary topologies. Maybe Flink could only provide a basic commit
>>> transformation and let the user build other parts of the topology. In the
>>> 1.12 we might first provide different patterns for these different
>>> scenarios at first and I think these components could be reused in the
>>> future.
>>>
>>> Best,
>>> Guowei
>>>
>>>
>>> On Mon, Sep 14, 2020 at 11:19 PM Dawid Wysakowicz <
>>> dwysakowicz@apache.org>
>>> 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
>>> > >
>>> >
>>> >
>>>
>>

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

Posted by Guowei Ma <gu...@gmail.com>.
## Concurrent checkpoints
AFAIK the committer would not see the file-1-2 when ck1 happens in the
ExactlyOnce mode.

## Committable bookkeeping and combining

I agree with you that the "CombineGlobalCommitter" would work. But we put
more optimization logic in the committer, which will make the committer
more and more complicated, and eventually become the same as the
Writer. For example, The committer needs to clean up some unused manifest
file when restoring from a failure if we introduce the optimizations to the
committer.

In this case another alternative might be to put this "merging"
optimization to a separate agg operator(maybe just like another `Writer`?).
The agg could produce an aggregated committable to the committer. The agg
operator could manage the whole life cycle of the manifest file it created.
It would make the committer have single responsibility.

>>The main question is if this pattern is generic to be put into the sink
framework or not.
Maybe I am wrong. But what I can feel from the current discussion is that
different requirements have different topological requirements.

## Using checkpointId
In the batch execution mode there would be no normal checkpoint any more.
That is why we do not introduce the checkpoint id in the API. So it is a
great thing that sink decouples its implementation from checkpointid. :)

Best,
Guowei


On Tue, Sep 15, 2020 at 7:33 AM Steven Wu <st...@gmail.com> wrote:

>
> ## concurrent checkpoints
>
> @Aljoscha Krettek <al...@apache.org> regarding the concurrent
> checkpoints, let me illustrate with a simple DAG below.
> [image: image.png]
>
> Let's assume each writer emits one file per checkpoint cycle and *writer-2
> is slow*. Now let's look at what the global committer receives
>
> timeline:
> ----------------------------------------------------------> Now
> from Writer-1:      file-1-1, ck-1, file-1-2, ck-2
> from Writer-2:
> file-2-1, ck-1
>
> In this case, the committer shouldn't include "file-1-2" into the commit
> for ck-1.
>
> ## Committable bookkeeping and combining
>
> I like David's proposal where the framework takes care of the
> bookkeeping of committables and provides a combiner API (CommT ->
> GlobalCommT) for GlobalCommitter. The only requirement is to tie the
> commit/CommT/GlobalCommT to a checkpoint.
>
> When a commit is successful for checkpoint-N, the framework needs to
> remove the GlobalCommT from the state corresponding to checkpoints <= N. If
> a commit fails, the GlobalCommT accumulates and will be included in the
> next cycle. That is how the Iceberg sink works. I think it is good to
> piggyback retries with Flink's periodic checkpoints for Iceberg sink.
> Otherwise, it can get complicated to implement retry logic that won't
> interfere with Flink checkpoints.
>
> The main question is if this pattern is generic to be put into the sink
> framework or not.
>
> > A alternative topology option for the IcebergSink might be :
> DataFileWriter
> -> Agg -> GlobalCommitter. One pro of this method is that we can let Agg
> take care of the cleanup instead of coupling the cleanup logic to the
> committer.
>
> @Guowei Ma <gu...@gmail.com> I would favor David's suggestion of
> "combine" API rather than a separate "Agg" operator.
>
> ## Using checkpointId
>
> > I think this can have some problems, for example when checkpoint ids are
> not strictly sequential, when we wrap around, or when the JobID changes.
> This will happen when doing a stop/start-from-savepoint cycle, for example.
>
> checkpointId can work if it is monotonically increasing, which I believe
> is the case for Flink today. Restoring from checkpoint or savepoint will
> resume the checkpointIds.
>
> We can deal with JobID change by saving it into the state and Iceberg
> snapshot metadata. There is already a PR [1] for that.
>
> ## Nonce
>
> > Flink provide a nonce to the GlobalCommitter where Flink guarantees that
> this nonce is unique
>
> That is actually how we implemented internally. Flink Iceberg sink
> basically hashes the Manifest file location as the nonce. Since the Flink
> generated Manifest file location is unique, it  guarantees the nonce is
> unique.
>
> IMO, checkpointId is also one way of implementing Nonce based on today's
> Flink behavior.
>
> > and will not change for repeated invocations of the GlobalCommitter with
> the same set of committables
>
>  if the same set of committables are combined into one GlobalCommT (like
> ManifestFile in Iceberg), then the Nonce could be part of the GlobalCommT
> interface.
>
> BTW, as David pointed out, the ManifestFile optimization is only in our
> internal implementation [2] right now. For the open source version, there
> is a github issue [3] to track follow-up improvements.
>
> Thanks,
> Steven
>
> [1] https://github.com/apache/iceberg/pull/1404
> [2]
> 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#L363
> [3] https://github.com/apache/iceberg/issues/1403
>
>
> On Mon, Sep 14, 2020 at 12:03 PM Guowei Ma <gu...@gmail.com> wrote:
>
>> Hi all,
>>
>>
>> Very thanks for the discussion and the valuable opinions! Currently there
>> are several ongoing issues and we would like to show what we are thinking
>> in the next few mails.
>>
>> It seems that the biggest issue now is about the topology of the sinks.
>> Before deciding what the sink API would look like, I would like to first
>> summarize the different topologies we have mentioned so that we could sync
>> on the same page and gain more insights about this issue. There are four
>> types of topology I could see. Please correct me if I misunderstand what
>> you mean:
>>
>>    1.
>>
>>    Commit individual files. (StreamingFileSink)
>>    1.
>>
>>       FileWriter -> FileCommitter
>>       2.
>>
>>    Commit a directory (HiveSink)
>>    1.
>>
>>       FileWriter -> FileCommitter -> GlobalCommitter
>>       3.
>>
>>    Commit a bundle of files (Iceberg)
>>    1.
>>
>>       DataFileWriter  -> GlobalCommitter
>>       4.
>>
>>    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
>>
>>
>> It can be seen from the above that the topologies are different according
>> to different requirements. Not only that there may be other options for
>> the
>> second and third categories. E.g
>>
>> A alternative topology option for the IcebergSink might be :
>> DataFileWriter
>> -> Agg -> GlobalCommitter. One pro of this method is that we can let Agg
>> take care of the cleanup instead of coupling the cleanup logic to the
>> committer.
>>
>>
>> In the long run I think we might provide the sink developer the ability to
>> build arbitrary topologies. Maybe Flink could only provide a basic commit
>> transformation and let the user build other parts of the topology. In the
>> 1.12 we might first provide different patterns for these different
>> scenarios at first and I think these components could be reused in the
>> future.
>>
>> Best,
>> Guowei
>>
>>
>> On Mon, Sep 14, 2020 at 11:19 PM Dawid Wysakowicz <dwysakowicz@apache.org
>> >
>> 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
>> > >
>> >
>> >
>>
>

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

Posted by Aljoscha Krettek <al...@apache.org>.
On 15.09.20 01:33, Steven Wu wrote:
> ## concurrent checkpoints
> 
> @Aljoscha Krettek <al...@apache.org> regarding the concurrent
> checkpoints, let me illustrate with a simple DAG below.
> [image: image.png]

Hi Steven,

images don't make it through to the mailing lists. You would need to 
host the file somewhere and send a link.

Best,
Aljoscha

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

Posted by Steven Wu <st...@gmail.com>.
## concurrent checkpoints

@Aljoscha Krettek <al...@apache.org> regarding the concurrent
checkpoints, let me illustrate with a simple DAG below.
[image: image.png]

Let's assume each writer emits one file per checkpoint cycle and *writer-2
is slow*. Now let's look at what the global committer receives

timeline:
----------------------------------------------------------> Now
from Writer-1:      file-1-1, ck-1, file-1-2, ck-2
from Writer-2:
file-2-1, ck-1

In this case, the committer shouldn't include "file-1-2" into the commit
for ck-1.

## Committable bookkeeping and combining

I like David's proposal where the framework takes care of the
bookkeeping of committables and provides a combiner API (CommT ->
GlobalCommT) for GlobalCommitter. The only requirement is to tie the
commit/CommT/GlobalCommT to a checkpoint.

When a commit is successful for checkpoint-N, the framework needs to remove
the GlobalCommT from the state corresponding to checkpoints <= N. If a
commit fails, the GlobalCommT accumulates and will be included in the next
cycle. That is how the Iceberg sink works. I think it is good to piggyback
retries with Flink's periodic checkpoints for Iceberg sink. Otherwise, it
can get complicated to implement retry logic that won't interfere with
Flink checkpoints.

The main question is if this pattern is generic to be put into the sink
framework or not.

> A alternative topology option for the IcebergSink might be :
DataFileWriter
-> Agg -> GlobalCommitter. One pro of this method is that we can let Agg
take care of the cleanup instead of coupling the cleanup logic to the
committer.

@Guowei Ma <gu...@gmail.com> I would favor David's suggestion of
"combine" API rather than a separate "Agg" operator.

## Using checkpointId

> I think this can have some problems, for example when checkpoint ids are
not strictly sequential, when we wrap around, or when the JobID changes.
This will happen when doing a stop/start-from-savepoint cycle, for example.

checkpointId can work if it is monotonically increasing, which I believe is
the case for Flink today. Restoring from checkpoint or savepoint will
resume the checkpointIds.

We can deal with JobID change by saving it into the state and Iceberg
snapshot metadata. There is already a PR [1] for that.

## Nonce

> Flink provide a nonce to the GlobalCommitter where Flink guarantees that
this nonce is unique

That is actually how we implemented internally. Flink Iceberg sink
basically hashes the Manifest file location as the nonce. Since the Flink
generated Manifest file location is unique, it  guarantees the nonce is
unique.

IMO, checkpointId is also one way of implementing Nonce based on today's
Flink behavior.

> and will not change for repeated invocations of the GlobalCommitter with
the same set of committables

 if the same set of committables are combined into one GlobalCommT (like
ManifestFile in Iceberg), then the Nonce could be part of the GlobalCommT
interface.

BTW, as David pointed out, the ManifestFile optimization is only in our
internal implementation [2] right now. For the open source version, there
is a github issue [3] to track follow-up improvements.

Thanks,
Steven

[1] https://github.com/apache/iceberg/pull/1404
[2]
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#L363
[3] https://github.com/apache/iceberg/issues/1403


On Mon, Sep 14, 2020 at 12:03 PM Guowei Ma <gu...@gmail.com> wrote:

> Hi all,
>
>
> Very thanks for the discussion and the valuable opinions! Currently there
> are several ongoing issues and we would like to show what we are thinking
> in the next few mails.
>
> It seems that the biggest issue now is about the topology of the sinks.
> Before deciding what the sink API would look like, I would like to first
> summarize the different topologies we have mentioned so that we could sync
> on the same page and gain more insights about this issue. There are four
> types of topology I could see. Please correct me if I misunderstand what
> you mean:
>
>    1.
>
>    Commit individual files. (StreamingFileSink)
>    1.
>
>       FileWriter -> FileCommitter
>       2.
>
>    Commit a directory (HiveSink)
>    1.
>
>       FileWriter -> FileCommitter -> GlobalCommitter
>       3.
>
>    Commit a bundle of files (Iceberg)
>    1.
>
>       DataFileWriter  -> GlobalCommitter
>       4.
>
>    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
>
>
> It can be seen from the above that the topologies are different according
> to different requirements. Not only that there may be other options for the
> second and third categories. E.g
>
> A alternative topology option for the IcebergSink might be : DataFileWriter
> -> Agg -> GlobalCommitter. One pro of this method is that we can let Agg
> take care of the cleanup instead of coupling the cleanup logic to the
> committer.
>
>
> In the long run I think we might provide the sink developer the ability to
> build arbitrary topologies. Maybe Flink could only provide a basic commit
> transformation and let the user build other parts of the topology. In the
> 1.12 we might first provide different patterns for these different
> scenarios at first and I think these components could be reused in the
> future.
>
> Best,
> Guowei
>
>
> On Mon, Sep 14, 2020 at 11:19 PM Dawid Wysakowicz <dw...@apache.org>
> 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
> > >
> >
> >
>

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

Posted by Guowei Ma <gu...@gmail.com>.
Hi all,


Very thanks for the discussion and the valuable opinions! Currently there
are several ongoing issues and we would like to show what we are thinking
in the next few mails.

It seems that the biggest issue now is about the topology of the sinks.
Before deciding what the sink API would look like, I would like to first
summarize the different topologies we have mentioned so that we could sync
on the same page and gain more insights about this issue. There are four
types of topology I could see. Please correct me if I misunderstand what
you mean:

   1.

   Commit individual files. (StreamingFileSink)
   1.

      FileWriter -> FileCommitter
      2.

   Commit a directory (HiveSink)
   1.

      FileWriter -> FileCommitter -> GlobalCommitter
      3.

   Commit a bundle of files (Iceberg)
   1.

      DataFileWriter  -> GlobalCommitter
      4.

   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


It can be seen from the above that the topologies are different according
to different requirements. Not only that there may be other options for the
second and third categories. E.g

A alternative topology option for the IcebergSink might be : DataFileWriter
-> Agg -> GlobalCommitter. One pro of this method is that we can let Agg
take care of the cleanup instead of coupling the cleanup logic to the
committer.


In the long run I think we might provide the sink developer the ability to
build arbitrary topologies. Maybe Flink could only provide a basic commit
transformation and let the user build other parts of the topology. In the
1.12 we might first provide different patterns for these different
scenarios at first and I think these components could be reused in the
future.

Best,
Guowei


On Mon, Sep 14, 2020 at 11:19 PM Dawid Wysakowicz <dw...@apache.org>
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
> >
>
>

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

Posted by Aljoscha Krettek <al...@apache.org>.
On 15.09.20 09:55, Dawid Wysakowicz wrote:
> 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.

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.

I can see why from a systems perspective it would be useful to have, of 
course.

Best,
Aljoscha

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

Posted by Piotr Nowojski <pn...@apache.org>.
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 <gu...@gmail.com> 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 <st...@gmail.com> 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 <pi...@gmail.com>
>> 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 <gu...@gmail.com> 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 <gu...@gmail.com>
>> 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 <st...@gmail.com>
>> wrote:
>> >> >
>> >> >> > AFAIK the committer would not see the file-1-2 when ck1 happens in
>> >> the
>> >> >> ExactlyOnce mode.
>> >> >>
>> >> >> @Guowei Ma <gu...@gmail.com> 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 <st...@gmail.com>
>> >> 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 <gu...@gmail.com>
>> >> 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 <zh...@alibaba-inc.com> 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 <dw...@apache.org>  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 <pi...@ververica.com>  Please correct me if  I
>> >> >>>> misunderstand you. thanks.
>> >> >>>>
>> >> >>>> Best,
>> >> >>>> Guowei
>> >> >>>>
>> >> >>>>
>> >> >>>> On Tue, Sep 15, 2020 at 3:55 PM Dawid Wysakowicz <
>> >> >>>> dwysakowicz@apache.org>
>> >> >>>> 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 <
>> >> >>>> dwysakowicz@apache.org> <dw...@apache.org>
>> >> >>>> > 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
>> >> >>>> >
>> >> >>>> >
>> >> >>>> >
>> >> >>>>
>> >> >>>
>> >>
>> >
>>
>

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

Posted by Guowei Ma <gu...@gmail.com>.
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 <st...@gmail.com> 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 <pi...@gmail.com>
> 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 <gu...@gmail.com> 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 <gu...@gmail.com>
> 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 <st...@gmail.com>
> wrote:
> >> >
> >> >> > AFAIK the committer would not see the file-1-2 when ck1 happens in
> >> the
> >> >> ExactlyOnce mode.
> >> >>
> >> >> @Guowei Ma <gu...@gmail.com> 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 <st...@gmail.com>
> >> 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 <gu...@gmail.com>
> >> 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 <zh...@alibaba-inc.com> 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 <dw...@apache.org>  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 <pi...@ververica.com>  Please correct me if  I
> >> >>>> misunderstand you. thanks.
> >> >>>>
> >> >>>> Best,
> >> >>>> Guowei
> >> >>>>
> >> >>>>
> >> >>>> On Tue, Sep 15, 2020 at 3:55 PM Dawid Wysakowicz <
> >> >>>> dwysakowicz@apache.org>
> >> >>>> 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 <
> >> >>>> dwysakowicz@apache.org> <dw...@apache.org>
> >> >>>> > 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
> >> >>>> >
> >> >>>> >
> >> >>>> >
> >> >>>>
> >> >>>
> >>
> >
>

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

Posted by Steven Wu <st...@gmail.com>.
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 <pi...@gmail.com>
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 <gu...@gmail.com> 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 <gu...@gmail.com> 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 <st...@gmail.com> wrote:
>> >
>> >> > AFAIK the committer would not see the file-1-2 when ck1 happens in
>> the
>> >> ExactlyOnce mode.
>> >>
>> >> @Guowei Ma <gu...@gmail.com> 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 <st...@gmail.com>
>> 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 <gu...@gmail.com>
>> 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 <zh...@alibaba-inc.com> 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 <dw...@apache.org>  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 <pi...@ververica.com>  Please correct me if  I
>> >>>> misunderstand you. thanks.
>> >>>>
>> >>>> Best,
>> >>>> Guowei
>> >>>>
>> >>>>
>> >>>> On Tue, Sep 15, 2020 at 3:55 PM Dawid Wysakowicz <
>> >>>> dwysakowicz@apache.org>
>> >>>> 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 <
>> >>>> dwysakowicz@apache.org> <dw...@apache.org>
>> >>>> > 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
>> >>>> >
>> >>>> >
>> >>>> >
>> >>>>
>> >>>
>>
>

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

Posted by Guowei Ma <gu...@gmail.com>.
Thank @piotr <pi...@ververica.com>  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 <pi...@gmail.com>
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 <gu...@gmail.com> 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 <gu...@gmail.com> 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 <st...@gmail.com>
> wrote:
> > >
> > >> > AFAIK the committer would not see the file-1-2 when ck1 happens in
> the
> > >> ExactlyOnce mode.
> > >>
> > >> @Guowei Ma <gu...@gmail.com> 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 <st...@gmail.com>
> 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 <gu...@gmail.com>
> > 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 <zh...@alibaba-inc.com> 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 <dw...@apache.org>  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 <pi...@ververica.com>  Please correct me if  I
> > >>>> misunderstand you. thanks.
> > >>>>
> > >>>> Best,
> > >>>> Guowei
> > >>>>
> > >>>>
> > >>>> On Tue, Sep 15, 2020 at 3:55 PM Dawid Wysakowicz <
> > >>>> dwysakowicz@apache.org>
> > >>>> 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 <
> > >>>> dwysakowicz@apache.org> <dw...@apache.org>
> > >>>> > 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
> > >>>> >
> > >>>> >
> > >>>> >
> > >>>>
> > >>>
> >
>

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

Posted by Piotr Nowojski <pi...@gmail.com>.
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 <gu...@gmail.com> 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 <gu...@gmail.com> 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 <st...@gmail.com> wrote:
> >
> >> > AFAIK the committer would not see the file-1-2 when ck1 happens in the
> >> ExactlyOnce mode.
> >>
> >> @Guowei Ma <gu...@gmail.com> 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 <st...@gmail.com> 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 <gu...@gmail.com>
> 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 <zh...@alibaba-inc.com> 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 <dw...@apache.org>  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 <pi...@ververica.com>  Please correct me if  I
> >>>> misunderstand you. thanks.
> >>>>
> >>>> Best,
> >>>> Guowei
> >>>>
> >>>>
> >>>> On Tue, Sep 15, 2020 at 3:55 PM Dawid Wysakowicz <
> >>>> dwysakowicz@apache.org>
> >>>> 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 <
> >>>> dwysakowicz@apache.org> <dw...@apache.org>
> >>>> > 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
> >>>> >
> >>>> >
> >>>> >
> >>>>
> >>>
>

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

Posted by Steven Wu <st...@gmail.com>.
It is more about extended outages of metastore. E.g. If we commit every 2
minutes, 4 hours of metastore outage can lead to over 120 GlobalCommitT.
And regarding metastore outages, it is undesirable for streaming jobs to
fail the job and keep restarting. It is better to keep processing records
(avoiding backlog) and upload to DFS (like S3). Commit will succeed
whenever the metastore comes back. It also provides a nice automatic
recovery story. Since GlobalCommT combines all data files (hundreds or
thousands in one checkpoint cycle) into a single item in state, this really
makes it scalable and efficient to deal with extended metastore outages.

"CommitResult commit(GlobalCommitT)" API can work, although it is less
efficient and flexible for some sinks. It is probably better to let sink
implementations decide what is the best retry behavior: one by one vs a big
batch/transaction. Hence I would propose APIs like these.
------------------------------
interface GlobalCommitter {
  // commit all pending GlobalCommitT items accumulated
  CommitResult commit(List<GlobalCommitT>)
}

interface CommitResult {
  List<GlobalCommitT> getSucceededCommitables();
  List<GlobalCommitT> getFailedCommitables();

  // most likely, framework just need to check and roll over the retryable
list to the next commit try
  List<GlobalCommitT> getRetrableCommittables();
}
---------------------------

Anyway, I am going to vote yes on the voting thread, since it is important
to move forward to meet the 1.12 goal. We can also discuss the small tweak
during the implementation phase.

Thanks,
Steven


On Sat, Sep 26, 2020 at 8:46 PM Guowei Ma <gu...@gmail.com> wrote:

> Hi Steven
>
> Thank you very much for your detailed explanation.
>
> Now I got your point, I could see that there are benefits from committing a
> collection of `GlobalCommT` as a whole when the external metastore
> environment is unstable at some time.
>
> But I have two little concern about introducing committing the collection
> of `GlobalCommit`:
>
> 1. For Option1: CommitResult commit(List<GlobalCommitT>). This option
> implies that users should commit to the collection of `GlobalCommit` as a
> whole.
> But maybe not all the system could do it as a whole, for example changing
> some file names could not do it. If it is the case I think maybe some guy
> would always ask the same question as I asked in the previous mail.
>
> 2. For Option2: List<CommitResult> commit(List<GlobalCommitT>). This option
> is more clear than the first one. But IMHO this option has only benefits
> when the external metastore is unstable and we want to retry many times and
> not fail the job. Maybe we should not rety so many times and end up with a
> lot of the uncommitted `GlobalCommitT`. If this is the case maybe we should
> make the api more clear/simple for the normal scenario. In addition there
> is only a globalcommit instance so I think the external system could bear
> the pressure.
>
> So personally I would like to say we might keep the API simpler at the
> beginning in 1.12
>
> What do you think?
>
> Best,
> Guowei
>
>
> On Fri, Sep 25, 2020 at 9:30 PM Steven Wu <st...@gmail.com> wrote:
>
> > I should clarify my last email a little more.
> >
> > For the example of commits for checkpoints 1-100 failed, the job is still
> > up (processing records and uploading files). When commit for checkpoint
> 101
> > came, IcebergSink would prefer the framework to pass in all 101
> GlobalCommT
> > (100 old + 1 new) so that it can commit all of them in one transaction.
> it
> > is more efficient than 101 separate transactions.
> >
> > Maybe the GlobalCommitter#commit semantics is to give the sink all
> > uncommitted GlobalCommT items and let sink implementation decide whether
> to
> > retry one by one or in a single transaction. It could mean that we need
> to
> > expand the CommitResult (e.g. a list for each result type, SUCCESS,
> > FAILURE, RETRY) interface. We can also start with the simple enum style
> > result for the whole list for now. If we need to break the experimental
> > API, it is also not a big deal since we only need to update a few sink
> > implementations.
> >
> > Thanks,
> > Steven
> >
> > On Fri, Sep 25, 2020 at 5:56 AM Steven Wu <st...@gmail.com> wrote:
> >
> > > > 1. The frame can not know which `GlobalCommT` to retry if we use the
> > > > List<GlobalCommT> as parameter when the `commit` returns `RETRY`.
> > > > 2. Of course we can let the `commit` return more detailed info but it
> > > might
> > > > be too complicated.
> > >
> > > If commit(List<GlobalCommT>) returns RETRY, it means the whole list
> needs
> > > to be retried. E.g. we have some outage with metadata service, commits
> > for
> > > checkpoints 1-100 failed. We can accumulate 100 GlobalCommT items. we
> > don't
> > > want to commit them one by one. It is faster to commit the whole list
> as
> > > one batch.
> > >
> > > > 3. On the other hand, I think only when restoring IcebergSink needs a
> > > > collection of `GlobalCommT` and giving back another collection of
> > > > `GlobalCommT` that are not committed
> > >
> > > That is when the job restarted due to failure or deployment.
> > >
> > >
> > > On Fri, Sep 25, 2020 at 5:24 AM Guowei Ma <gu...@gmail.com>
> wrote:
> > >
> > >> Hi, all
> > >>
> > >> From the above discussion we could find that FLIP focuses on providing
> > an
> > >> unified transactional sink API. So I updated the FLIP's title to
> > "Unified
> > >> Transactional Sink API". But I found that the old link could not be
> > opened
> > >> again.
> > >>
> > >> I would update the link[1] here. Sorry for the inconvenience.
> > >>
> > >> [1]https://cwiki.apache.org/confluence/x/KEJ4CQ
> > >>
> > >> Best,
> > >> Guowei
> > >>
> > >>
> > >> On Fri, Sep 25, 2020 at 3:26 PM Guowei Ma <gu...@gmail.com>
> wrote:
> > >>
> > >> > Hi, Steven
> > >> >
> > >> > >>I also have a clarifying question regarding the WriterStateT.
> Since
> > >> > >>IcebergWriter won't need to checkpoint any state, should we set it
> > to
> > >> > *Void*
> > >> > >>type? Since getWriterStateSerializer() returns Optional, that is
> > clear
> > >> > and
> > >> > >>we can return Optional.empty().
> > >> >
> > >> > Yes I think you could do it. If you return Optional.empty() we would
> > >> > ignore all the state you return.
> > >> >
> > >> > Best,
> > >> > Guowei
> > >> >
> > >> >
> > >> > On Fri, Sep 25, 2020 at 3:14 PM Guowei Ma <gu...@gmail.com>
> > wrote:
> > >> >
> > >> >> Hi,Steven
> > >> >>
> > >> >> Thank you for reading the FLIP so carefully.
> > >> >> 1. The frame can not know which `GlobalCommT` to retry if we use
> the
> > >> >> List<GlobalCommT> as parameter when the `commit` returns `RETRY`.
> > >> >> 2. Of course we can let the `commit` return more detailed info but
> it
> > >> >> might be too complicated.
> > >> >> 3. On the other hand, I think only when restoring IcebergSink
> needs a
> > >> >> collection of `GlobalCommT` and giving back another collection of
> > >> >> `GlobalCommT` that are not committed.
> > >> >>
> > >> >> Best,
> > >> >> Guowei
> > >> >>
> > >> >>
> > >> >> On Fri, Sep 25, 2020 at 1:45 AM Steven Wu <st...@gmail.com>
> > >> wrote:
> > >> >>
> > >> >>> Guowei,
> > >> >>>
> > >> >>> Thanks a lot for updating the wiki page. It looks great.
> > >> >>>
> > >> >>> I noticed one inconsistency in the wiki with your last summary
> email
> > >> for
> > >> >>> GlobalCommitter interface. I think the version in the summary
> email
> > is
> > >> >>> the
> > >> >>> intended one, because rollover from previous failed commits can
> > >> >>> accumulate
> > >> >>> a list.
> > >> >>> CommitResult commit(GlobalCommT globalCommittable); // in the wiki
> > >> >>> =>
> > >> >>> CommitResult commit(List<GlobalCommT> globalCommittable);  // in
> the
> > >> >>> summary email
> > >> >>>
> > >> >>> I also have a clarifying question regarding the WriterStateT.
> Since
> > >> >>> IcebergWriter won't need to checkpoint any state, should we set it
> > to
> > >> >>> *Void*
> > >> >>> type? Since getWriterStateSerializer() returns Optional, that is
> > clear
> > >> >>> and
> > >> >>> we can return Optional.empty().
> > >> >>>
> > >> >>> Thanks,
> > >> >>> Steven
> > >> >>>
> > >> >>> On Wed, Sep 23, 2020 at 6:59 PM Guowei Ma <gu...@gmail.com>
> > >> wrote:
> > >> >>>
> > >> >>> > Thanks Aljoscha for your suggestion.  I have updated FLIP. Any
> > >> >>> comments are
> > >> >>> > welcome.
> > >> >>> >
> > >> >>> > Best,
> > >> >>> > Guowei
> > >> >>> >
> > >> >>> >
> > >> >>> > On Wed, Sep 23, 2020 at 4:25 PM Aljoscha Krettek <
> > >> aljoscha@apache.org>
> > >> >>> > wrote:
> > >> >>> >
> > >> >>> > > Yes, that sounds good! I'll probably have some comments on the
> > >> FLIP
> > >> >>> > > about the names of generic parameters and the Javadoc but we
> can
> > >> >>> address
> > >> >>> > > them later or during implementation.
> > >> >>> > >
> > >> >>> > > I also think that we probably need the FAIL,RETRY,SUCCESS
> result
> > >> for
> > >> >>> > > globalCommit() but we can also do that as a later addition.
> > >> >>> > >
> > >> >>> > > So I think we're good to go to update the FLIP, do any last
> > minute
> > >> >>> > > changes and then vote.
> > >> >>> > >
> > >> >>> > > Best,
> > >> >>> > > Aljoscha
> > >> >>> > >
> > >> >>> > > On 23.09.20 06:13, Guowei Ma wrote:
> > >> >>> > > > Hi, all
> > >> >>> > > >
> > >> >>> > > > Thank everyone very much for your ideas and suggestions. I
> > would
> > >> >>> try to
> > >> >>> > > > summarize again the consensus :). Correct me if I am wrong
> or
> > >> >>> > > misunderstand
> > >> >>> > > > you.
> > >> >>> > > >
> > >> >>> > > > ## Consensus-1
> > >> >>> > > >
> > >> >>> > > > 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.
> > >> >>> > > >
> > >> >>> > > > ## Consensus-2
> > >> >>> > > >
> > >> >>> > > > 1. What should the “Unified Sink API” support/cover? It
> > includes
> > >> >>> two
> > >> >>> > > > aspects. 1. The same sink implementation would work for both
> > the
> > >> >>> batch
> > >> >>> > > and
> > >> >>> > > > stream execution mode. 2. In the long run we should give the
> > >> sink
> > >> >>> > > developer
> > >> >>> > > > the ability of building “arbitrary” topologies. But for
> > >> Flink-1.12
> > >> >>> we
> > >> >>> > > > should be more focused on only satisfying the
> S3/HDFS/Iceberg
> > >> sink.
> > >> >>> > > > 2. Because the batch execution mode does not have the normal
> > >> >>> checkpoint
> > >> >>> > > the
> > >> >>> > > > sink developer should not depend on it any more if we want a
> > >> >>> unified
> > >> >>> > > sink.
> > >> >>> > > > 3. We can benefit by providing an asynchronous `Writer`
> > version.
> > >> >>> But
> > >> >>> > > > because the unified sink is already very complicated, we
> don’t
> > >> add
> > >> >>> this
> > >> >>> > > in
> > >> >>> > > > the first version.
> > >> >>> > > >
> > >> >>> > > >
> > >> >>> > > > According to these consensus I would propose the first
> version
> > >> of
> > >> >>> the
> > >> >>> > new
> > >> >>> > > > sink api as follows. What do you think? Any comments are
> > >> welcome.
> > >> >>> > > >
> > >> >>> > > > /**
> > >> >>> > > >   * This interface lets the sink developer build a simple
> > >> >>> transactional
> > >> >>> > > sink
> > >> >>> > > > topology pattern, which satisfies the HDFS/S3/Iceberg sink.
> > >> >>> > > >   * This sink topology includes one {@link Writer} + one
> > {@link
> > >> >>> > > Committer} +
> > >> >>> > > > one {@link GlobalCommitter}.
> > >> >>> > > >   * The {@link Writer} is responsible for producing the
> > >> >>> committable.
> > >> >>> > > >   * The {@link Committer} is responsible for committing a
> > single
> > >> >>> > > > committables.
> > >> >>> > > >   * The {@link GlobalCommitter} is responsible for
> committing
> > an
> > >> >>> > > aggregated
> > >> >>> > > > committable, which we called global committables.
> > >> >>> > > >   *
> > >> >>> > > >   * But both the {@link Committer} and the {@link
> > >> GlobalCommitter}
> > >> >>> are
> > >> >>> > > > optional.
> > >> >>> > > >   */
> > >> >>> > > > interface TSink<IN, CommT, GCommT, WriterS> {
> > >> >>> > > >
> > >> >>> > > >          Writer<IN, CommT, WriterS> createWriter(InitContext
> > >> >>> > > initContext);
> > >> >>> > > >
> > >> >>> > > >          Writer<IN, CommT, WriterS>
> restoreWriter(InitContext
> > >> >>> > > initContext,
> > >> >>> > > > List<WriterS> states);
> > >> >>> > > >
> > >> >>> > > >          Optional<Committer<CommT>> createCommitter();
> > >> >>> > > >
> > >> >>> > > >          Optional<GlobalCommitter<CommT, GCommT>>
> > >> >>> > > createGlobalCommitter();
> > >> >>> > > >
> > >> >>> > > >          SimpleVersionedSerializer<CommT>
> > >> >>> getCommittableSerializer();
> > >> >>> > > >
> > >> >>> > > >          Optional<SimpleVersionedSerializer<GCommT>>
> > >> >>> > > > getGlobalCommittableSerializer();
> > >> >>> > > > }
> > >> >>> > > >
> > >> >>> > > > /**
> > >> >>> > > >   * The {@link GlobalCommitter} is responsible for
> committing
> > an
> > >> >>> > > aggregated
> > >> >>> > > > committable, which we called global committables.
> > >> >>> > > >   */
> > >> >>> > > > interface GlobalCommitter<CommT, GCommT> {
> > >> >>> > > >
> > >> >>> > > >          /**
> > >> >>> > > >           * This method is called when restoring from a
> > >> failover.
> > >> >>> > > >           * @param globalCommittables the global
> committables
> > >> that
> > >> >>> are
> > >> >>> > > not
> > >> >>> > > > committed in the previous session.
> > >> >>> > > >           * @return the global committables that should be
> > >> >>> committed
> > >> >>> > > again
> > >> >>> > > > in the current session.
> > >> >>> > > >           */
> > >> >>> > > >          List<GCommT>
> filterRecoveredCommittables(List<GCommT>
> > >> >>> > > > globalCommittables);
> > >> >>> > > >
> > >> >>> > > >          /**
> > >> >>> > > >           * Compute an aggregated committable from a
> > collection
> > >> of
> > >> >>> > > > committables.
> > >> >>> > > >           * @param committables a collection of committables
> > >> that
> > >> >>> are
> > >> >>> > > needed
> > >> >>> > > > to combine
> > >> >>> > > >           * @return an aggregated committable
> > >> >>> > > >           */
> > >> >>> > > >          GCommT combine(List<CommT> committables);
> > >> >>> > > >
> > >> >>> > > >          void commit(List<GCommT> globalCommittables);
> > >> >>> > > >
> > >> >>> > > >          /**
> > >> >>> > > >           * There are no committables any more.
> > >> >>> > > >           */
> > >> >>> > > >          void endOfInput();
> > >> >>> > > > }
> > >> >>> > > >
> > >> >>> > > > Best,
> > >> >>> > > > Guowei
> > >> >>> > >
> > >> >>> > >
> > >> >>> >
> > >> >>>
> > >> >>
> > >>
> > >
> >
>

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

Posted by Guowei Ma <gu...@gmail.com>.
Hi Steven

Thank you very much for your detailed explanation.

Now I got your point, I could see that there are benefits from committing a
collection of `GlobalCommT` as a whole when the external metastore
environment is unstable at some time.

But I have two little concern about introducing committing the collection
of `GlobalCommit`:

1. For Option1: CommitResult commit(List<GlobalCommitT>). This option
implies that users should commit to the collection of `GlobalCommit` as a
whole.
But maybe not all the system could do it as a whole, for example changing
some file names could not do it. If it is the case I think maybe some guy
would always ask the same question as I asked in the previous mail.

2. For Option2: List<CommitResult> commit(List<GlobalCommitT>). This option
is more clear than the first one. But IMHO this option has only benefits
when the external metastore is unstable and we want to retry many times and
not fail the job. Maybe we should not rety so many times and end up with a
lot of the uncommitted `GlobalCommitT`. If this is the case maybe we should
make the api more clear/simple for the normal scenario. In addition there
is only a globalcommit instance so I think the external system could bear
the pressure.

So personally I would like to say we might keep the API simpler at the
beginning in 1.12

What do you think?

Best,
Guowei


On Fri, Sep 25, 2020 at 9:30 PM Steven Wu <st...@gmail.com> wrote:

> I should clarify my last email a little more.
>
> For the example of commits for checkpoints 1-100 failed, the job is still
> up (processing records and uploading files). When commit for checkpoint 101
> came, IcebergSink would prefer the framework to pass in all 101 GlobalCommT
> (100 old + 1 new) so that it can commit all of them in one transaction. it
> is more efficient than 101 separate transactions.
>
> Maybe the GlobalCommitter#commit semantics is to give the sink all
> uncommitted GlobalCommT items and let sink implementation decide whether to
> retry one by one or in a single transaction. It could mean that we need to
> expand the CommitResult (e.g. a list for each result type, SUCCESS,
> FAILURE, RETRY) interface. We can also start with the simple enum style
> result for the whole list for now. If we need to break the experimental
> API, it is also not a big deal since we only need to update a few sink
> implementations.
>
> Thanks,
> Steven
>
> On Fri, Sep 25, 2020 at 5:56 AM Steven Wu <st...@gmail.com> wrote:
>
> > > 1. The frame can not know which `GlobalCommT` to retry if we use the
> > > List<GlobalCommT> as parameter when the `commit` returns `RETRY`.
> > > 2. Of course we can let the `commit` return more detailed info but it
> > might
> > > be too complicated.
> >
> > If commit(List<GlobalCommT>) returns RETRY, it means the whole list needs
> > to be retried. E.g. we have some outage with metadata service, commits
> for
> > checkpoints 1-100 failed. We can accumulate 100 GlobalCommT items. we
> don't
> > want to commit them one by one. It is faster to commit the whole list as
> > one batch.
> >
> > > 3. On the other hand, I think only when restoring IcebergSink needs a
> > > collection of `GlobalCommT` and giving back another collection of
> > > `GlobalCommT` that are not committed
> >
> > That is when the job restarted due to failure or deployment.
> >
> >
> > On Fri, Sep 25, 2020 at 5:24 AM Guowei Ma <gu...@gmail.com> wrote:
> >
> >> Hi, all
> >>
> >> From the above discussion we could find that FLIP focuses on providing
> an
> >> unified transactional sink API. So I updated the FLIP's title to
> "Unified
> >> Transactional Sink API". But I found that the old link could not be
> opened
> >> again.
> >>
> >> I would update the link[1] here. Sorry for the inconvenience.
> >>
> >> [1]https://cwiki.apache.org/confluence/x/KEJ4CQ
> >>
> >> Best,
> >> Guowei
> >>
> >>
> >> On Fri, Sep 25, 2020 at 3:26 PM Guowei Ma <gu...@gmail.com> wrote:
> >>
> >> > Hi, Steven
> >> >
> >> > >>I also have a clarifying question regarding the WriterStateT. Since
> >> > >>IcebergWriter won't need to checkpoint any state, should we set it
> to
> >> > *Void*
> >> > >>type? Since getWriterStateSerializer() returns Optional, that is
> clear
> >> > and
> >> > >>we can return Optional.empty().
> >> >
> >> > Yes I think you could do it. If you return Optional.empty() we would
> >> > ignore all the state you return.
> >> >
> >> > Best,
> >> > Guowei
> >> >
> >> >
> >> > On Fri, Sep 25, 2020 at 3:14 PM Guowei Ma <gu...@gmail.com>
> wrote:
> >> >
> >> >> Hi,Steven
> >> >>
> >> >> Thank you for reading the FLIP so carefully.
> >> >> 1. The frame can not know which `GlobalCommT` to retry if we use the
> >> >> List<GlobalCommT> as parameter when the `commit` returns `RETRY`.
> >> >> 2. Of course we can let the `commit` return more detailed info but it
> >> >> might be too complicated.
> >> >> 3. On the other hand, I think only when restoring IcebergSink needs a
> >> >> collection of `GlobalCommT` and giving back another collection of
> >> >> `GlobalCommT` that are not committed.
> >> >>
> >> >> Best,
> >> >> Guowei
> >> >>
> >> >>
> >> >> On Fri, Sep 25, 2020 at 1:45 AM Steven Wu <st...@gmail.com>
> >> wrote:
> >> >>
> >> >>> Guowei,
> >> >>>
> >> >>> Thanks a lot for updating the wiki page. It looks great.
> >> >>>
> >> >>> I noticed one inconsistency in the wiki with your last summary email
> >> for
> >> >>> GlobalCommitter interface. I think the version in the summary email
> is
> >> >>> the
> >> >>> intended one, because rollover from previous failed commits can
> >> >>> accumulate
> >> >>> a list.
> >> >>> CommitResult commit(GlobalCommT globalCommittable); // in the wiki
> >> >>> =>
> >> >>> CommitResult commit(List<GlobalCommT> globalCommittable);  // in the
> >> >>> summary email
> >> >>>
> >> >>> I also have a clarifying question regarding the WriterStateT. Since
> >> >>> IcebergWriter won't need to checkpoint any state, should we set it
> to
> >> >>> *Void*
> >> >>> type? Since getWriterStateSerializer() returns Optional, that is
> clear
> >> >>> and
> >> >>> we can return Optional.empty().
> >> >>>
> >> >>> Thanks,
> >> >>> Steven
> >> >>>
> >> >>> On Wed, Sep 23, 2020 at 6:59 PM Guowei Ma <gu...@gmail.com>
> >> wrote:
> >> >>>
> >> >>> > Thanks Aljoscha for your suggestion.  I have updated FLIP. Any
> >> >>> comments are
> >> >>> > welcome.
> >> >>> >
> >> >>> > Best,
> >> >>> > Guowei
> >> >>> >
> >> >>> >
> >> >>> > On Wed, Sep 23, 2020 at 4:25 PM Aljoscha Krettek <
> >> aljoscha@apache.org>
> >> >>> > wrote:
> >> >>> >
> >> >>> > > Yes, that sounds good! I'll probably have some comments on the
> >> FLIP
> >> >>> > > about the names of generic parameters and the Javadoc but we can
> >> >>> address
> >> >>> > > them later or during implementation.
> >> >>> > >
> >> >>> > > I also think that we probably need the FAIL,RETRY,SUCCESS result
> >> for
> >> >>> > > globalCommit() but we can also do that as a later addition.
> >> >>> > >
> >> >>> > > So I think we're good to go to update the FLIP, do any last
> minute
> >> >>> > > changes and then vote.
> >> >>> > >
> >> >>> > > Best,
> >> >>> > > Aljoscha
> >> >>> > >
> >> >>> > > On 23.09.20 06:13, Guowei Ma wrote:
> >> >>> > > > Hi, all
> >> >>> > > >
> >> >>> > > > Thank everyone very much for your ideas and suggestions. I
> would
> >> >>> try to
> >> >>> > > > summarize again the consensus :). Correct me if I am wrong or
> >> >>> > > misunderstand
> >> >>> > > > you.
> >> >>> > > >
> >> >>> > > > ## Consensus-1
> >> >>> > > >
> >> >>> > > > 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.
> >> >>> > > >
> >> >>> > > > ## Consensus-2
> >> >>> > > >
> >> >>> > > > 1. What should the “Unified Sink API” support/cover? It
> includes
> >> >>> two
> >> >>> > > > aspects. 1. The same sink implementation would work for both
> the
> >> >>> batch
> >> >>> > > and
> >> >>> > > > stream execution mode. 2. In the long run we should give the
> >> sink
> >> >>> > > developer
> >> >>> > > > the ability of building “arbitrary” topologies. But for
> >> Flink-1.12
> >> >>> we
> >> >>> > > > should be more focused on only satisfying the S3/HDFS/Iceberg
> >> sink.
> >> >>> > > > 2. Because the batch execution mode does not have the normal
> >> >>> checkpoint
> >> >>> > > the
> >> >>> > > > sink developer should not depend on it any more if we want a
> >> >>> unified
> >> >>> > > sink.
> >> >>> > > > 3. We can benefit by providing an asynchronous `Writer`
> version.
> >> >>> But
> >> >>> > > > because the unified sink is already very complicated, we don’t
> >> add
> >> >>> this
> >> >>> > > in
> >> >>> > > > the first version.
> >> >>> > > >
> >> >>> > > >
> >> >>> > > > According to these consensus I would propose the first version
> >> of
> >> >>> the
> >> >>> > new
> >> >>> > > > sink api as follows. What do you think? Any comments are
> >> welcome.
> >> >>> > > >
> >> >>> > > > /**
> >> >>> > > >   * This interface lets the sink developer build a simple
> >> >>> transactional
> >> >>> > > sink
> >> >>> > > > topology pattern, which satisfies the HDFS/S3/Iceberg sink.
> >> >>> > > >   * This sink topology includes one {@link Writer} + one
> {@link
> >> >>> > > Committer} +
> >> >>> > > > one {@link GlobalCommitter}.
> >> >>> > > >   * The {@link Writer} is responsible for producing the
> >> >>> committable.
> >> >>> > > >   * The {@link Committer} is responsible for committing a
> single
> >> >>> > > > committables.
> >> >>> > > >   * The {@link GlobalCommitter} is responsible for committing
> an
> >> >>> > > aggregated
> >> >>> > > > committable, which we called global committables.
> >> >>> > > >   *
> >> >>> > > >   * But both the {@link Committer} and the {@link
> >> GlobalCommitter}
> >> >>> are
> >> >>> > > > optional.
> >> >>> > > >   */
> >> >>> > > > interface TSink<IN, CommT, GCommT, WriterS> {
> >> >>> > > >
> >> >>> > > >          Writer<IN, CommT, WriterS> createWriter(InitContext
> >> >>> > > initContext);
> >> >>> > > >
> >> >>> > > >          Writer<IN, CommT, WriterS> restoreWriter(InitContext
> >> >>> > > initContext,
> >> >>> > > > List<WriterS> states);
> >> >>> > > >
> >> >>> > > >          Optional<Committer<CommT>> createCommitter();
> >> >>> > > >
> >> >>> > > >          Optional<GlobalCommitter<CommT, GCommT>>
> >> >>> > > createGlobalCommitter();
> >> >>> > > >
> >> >>> > > >          SimpleVersionedSerializer<CommT>
> >> >>> getCommittableSerializer();
> >> >>> > > >
> >> >>> > > >          Optional<SimpleVersionedSerializer<GCommT>>
> >> >>> > > > getGlobalCommittableSerializer();
> >> >>> > > > }
> >> >>> > > >
> >> >>> > > > /**
> >> >>> > > >   * The {@link GlobalCommitter} is responsible for committing
> an
> >> >>> > > aggregated
> >> >>> > > > committable, which we called global committables.
> >> >>> > > >   */
> >> >>> > > > interface GlobalCommitter<CommT, GCommT> {
> >> >>> > > >
> >> >>> > > >          /**
> >> >>> > > >           * This method is called when restoring from a
> >> failover.
> >> >>> > > >           * @param globalCommittables the global committables
> >> that
> >> >>> are
> >> >>> > > not
> >> >>> > > > committed in the previous session.
> >> >>> > > >           * @return the global committables that should be
> >> >>> committed
> >> >>> > > again
> >> >>> > > > in the current session.
> >> >>> > > >           */
> >> >>> > > >          List<GCommT> filterRecoveredCommittables(List<GCommT>
> >> >>> > > > globalCommittables);
> >> >>> > > >
> >> >>> > > >          /**
> >> >>> > > >           * Compute an aggregated committable from a
> collection
> >> of
> >> >>> > > > committables.
> >> >>> > > >           * @param committables a collection of committables
> >> that
> >> >>> are
> >> >>> > > needed
> >> >>> > > > to combine
> >> >>> > > >           * @return an aggregated committable
> >> >>> > > >           */
> >> >>> > > >          GCommT combine(List<CommT> committables);
> >> >>> > > >
> >> >>> > > >          void commit(List<GCommT> globalCommittables);
> >> >>> > > >
> >> >>> > > >          /**
> >> >>> > > >           * There are no committables any more.
> >> >>> > > >           */
> >> >>> > > >          void endOfInput();
> >> >>> > > > }
> >> >>> > > >
> >> >>> > > > Best,
> >> >>> > > > Guowei
> >> >>> > >
> >> >>> > >
> >> >>> >
> >> >>>
> >> >>
> >>
> >
>

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

Posted by Steven Wu <st...@gmail.com>.
I should clarify my last email a little more.

For the example of commits for checkpoints 1-100 failed, the job is still
up (processing records and uploading files). When commit for checkpoint 101
came, IcebergSink would prefer the framework to pass in all 101 GlobalCommT
(100 old + 1 new) so that it can commit all of them in one transaction. it
is more efficient than 101 separate transactions.

Maybe the GlobalCommitter#commit semantics is to give the sink all
uncommitted GlobalCommT items and let sink implementation decide whether to
retry one by one or in a single transaction. It could mean that we need to
expand the CommitResult (e.g. a list for each result type, SUCCESS,
FAILURE, RETRY) interface. We can also start with the simple enum style
result for the whole list for now. If we need to break the experimental
API, it is also not a big deal since we only need to update a few sink
implementations.

Thanks,
Steven

On Fri, Sep 25, 2020 at 5:56 AM Steven Wu <st...@gmail.com> wrote:

> > 1. The frame can not know which `GlobalCommT` to retry if we use the
> > List<GlobalCommT> as parameter when the `commit` returns `RETRY`.
> > 2. Of course we can let the `commit` return more detailed info but it
> might
> > be too complicated.
>
> If commit(List<GlobalCommT>) returns RETRY, it means the whole list needs
> to be retried. E.g. we have some outage with metadata service, commits for
> checkpoints 1-100 failed. We can accumulate 100 GlobalCommT items. we don't
> want to commit them one by one. It is faster to commit the whole list as
> one batch.
>
> > 3. On the other hand, I think only when restoring IcebergSink needs a
> > collection of `GlobalCommT` and giving back another collection of
> > `GlobalCommT` that are not committed
>
> That is when the job restarted due to failure or deployment.
>
>
> On Fri, Sep 25, 2020 at 5:24 AM Guowei Ma <gu...@gmail.com> wrote:
>
>> Hi, all
>>
>> From the above discussion we could find that FLIP focuses on providing an
>> unified transactional sink API. So I updated the FLIP's title to "Unified
>> Transactional Sink API". But I found that the old link could not be opened
>> again.
>>
>> I would update the link[1] here. Sorry for the inconvenience.
>>
>> [1]https://cwiki.apache.org/confluence/x/KEJ4CQ
>>
>> Best,
>> Guowei
>>
>>
>> On Fri, Sep 25, 2020 at 3:26 PM Guowei Ma <gu...@gmail.com> wrote:
>>
>> > Hi, Steven
>> >
>> > >>I also have a clarifying question regarding the WriterStateT. Since
>> > >>IcebergWriter won't need to checkpoint any state, should we set it to
>> > *Void*
>> > >>type? Since getWriterStateSerializer() returns Optional, that is clear
>> > and
>> > >>we can return Optional.empty().
>> >
>> > Yes I think you could do it. If you return Optional.empty() we would
>> > ignore all the state you return.
>> >
>> > Best,
>> > Guowei
>> >
>> >
>> > On Fri, Sep 25, 2020 at 3:14 PM Guowei Ma <gu...@gmail.com> wrote:
>> >
>> >> Hi,Steven
>> >>
>> >> Thank you for reading the FLIP so carefully.
>> >> 1. The frame can not know which `GlobalCommT` to retry if we use the
>> >> List<GlobalCommT> as parameter when the `commit` returns `RETRY`.
>> >> 2. Of course we can let the `commit` return more detailed info but it
>> >> might be too complicated.
>> >> 3. On the other hand, I think only when restoring IcebergSink needs a
>> >> collection of `GlobalCommT` and giving back another collection of
>> >> `GlobalCommT` that are not committed.
>> >>
>> >> Best,
>> >> Guowei
>> >>
>> >>
>> >> On Fri, Sep 25, 2020 at 1:45 AM Steven Wu <st...@gmail.com>
>> wrote:
>> >>
>> >>> Guowei,
>> >>>
>> >>> Thanks a lot for updating the wiki page. It looks great.
>> >>>
>> >>> I noticed one inconsistency in the wiki with your last summary email
>> for
>> >>> GlobalCommitter interface. I think the version in the summary email is
>> >>> the
>> >>> intended one, because rollover from previous failed commits can
>> >>> accumulate
>> >>> a list.
>> >>> CommitResult commit(GlobalCommT globalCommittable); // in the wiki
>> >>> =>
>> >>> CommitResult commit(List<GlobalCommT> globalCommittable);  // in the
>> >>> summary email
>> >>>
>> >>> I also have a clarifying question regarding the WriterStateT. Since
>> >>> IcebergWriter won't need to checkpoint any state, should we set it to
>> >>> *Void*
>> >>> type? Since getWriterStateSerializer() returns Optional, that is clear
>> >>> and
>> >>> we can return Optional.empty().
>> >>>
>> >>> Thanks,
>> >>> Steven
>> >>>
>> >>> On Wed, Sep 23, 2020 at 6:59 PM Guowei Ma <gu...@gmail.com>
>> wrote:
>> >>>
>> >>> > Thanks Aljoscha for your suggestion.  I have updated FLIP. Any
>> >>> comments are
>> >>> > welcome.
>> >>> >
>> >>> > Best,
>> >>> > Guowei
>> >>> >
>> >>> >
>> >>> > On Wed, Sep 23, 2020 at 4:25 PM Aljoscha Krettek <
>> aljoscha@apache.org>
>> >>> > wrote:
>> >>> >
>> >>> > > Yes, that sounds good! I'll probably have some comments on the
>> FLIP
>> >>> > > about the names of generic parameters and the Javadoc but we can
>> >>> address
>> >>> > > them later or during implementation.
>> >>> > >
>> >>> > > I also think that we probably need the FAIL,RETRY,SUCCESS result
>> for
>> >>> > > globalCommit() but we can also do that as a later addition.
>> >>> > >
>> >>> > > So I think we're good to go to update the FLIP, do any last minute
>> >>> > > changes and then vote.
>> >>> > >
>> >>> > > Best,
>> >>> > > Aljoscha
>> >>> > >
>> >>> > > On 23.09.20 06:13, Guowei Ma wrote:
>> >>> > > > Hi, all
>> >>> > > >
>> >>> > > > Thank everyone very much for your ideas and suggestions. I would
>> >>> try to
>> >>> > > > summarize again the consensus :). Correct me if I am wrong or
>> >>> > > misunderstand
>> >>> > > > you.
>> >>> > > >
>> >>> > > > ## Consensus-1
>> >>> > > >
>> >>> > > > 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.
>> >>> > > >
>> >>> > > > ## Consensus-2
>> >>> > > >
>> >>> > > > 1. What should the “Unified Sink API” support/cover? It includes
>> >>> two
>> >>> > > > aspects. 1. The same sink implementation would work for both the
>> >>> batch
>> >>> > > and
>> >>> > > > stream execution mode. 2. In the long run we should give the
>> sink
>> >>> > > developer
>> >>> > > > the ability of building “arbitrary” topologies. But for
>> Flink-1.12
>> >>> we
>> >>> > > > should be more focused on only satisfying the S3/HDFS/Iceberg
>> sink.
>> >>> > > > 2. Because the batch execution mode does not have the normal
>> >>> checkpoint
>> >>> > > the
>> >>> > > > sink developer should not depend on it any more if we want a
>> >>> unified
>> >>> > > sink.
>> >>> > > > 3. We can benefit by providing an asynchronous `Writer` version.
>> >>> But
>> >>> > > > because the unified sink is already very complicated, we don’t
>> add
>> >>> this
>> >>> > > in
>> >>> > > > the first version.
>> >>> > > >
>> >>> > > >
>> >>> > > > According to these consensus I would propose the first version
>> of
>> >>> the
>> >>> > new
>> >>> > > > sink api as follows. What do you think? Any comments are
>> welcome.
>> >>> > > >
>> >>> > > > /**
>> >>> > > >   * This interface lets the sink developer build a simple
>> >>> transactional
>> >>> > > sink
>> >>> > > > topology pattern, which satisfies the HDFS/S3/Iceberg sink.
>> >>> > > >   * This sink topology includes one {@link Writer} + one {@link
>> >>> > > Committer} +
>> >>> > > > one {@link GlobalCommitter}.
>> >>> > > >   * The {@link Writer} is responsible for producing the
>> >>> committable.
>> >>> > > >   * The {@link Committer} is responsible for committing a single
>> >>> > > > committables.
>> >>> > > >   * The {@link GlobalCommitter} is responsible for committing an
>> >>> > > aggregated
>> >>> > > > committable, which we called global committables.
>> >>> > > >   *
>> >>> > > >   * But both the {@link Committer} and the {@link
>> GlobalCommitter}
>> >>> are
>> >>> > > > optional.
>> >>> > > >   */
>> >>> > > > interface TSink<IN, CommT, GCommT, WriterS> {
>> >>> > > >
>> >>> > > >          Writer<IN, CommT, WriterS> createWriter(InitContext
>> >>> > > initContext);
>> >>> > > >
>> >>> > > >          Writer<IN, CommT, WriterS> restoreWriter(InitContext
>> >>> > > initContext,
>> >>> > > > List<WriterS> states);
>> >>> > > >
>> >>> > > >          Optional<Committer<CommT>> createCommitter();
>> >>> > > >
>> >>> > > >          Optional<GlobalCommitter<CommT, GCommT>>
>> >>> > > createGlobalCommitter();
>> >>> > > >
>> >>> > > >          SimpleVersionedSerializer<CommT>
>> >>> getCommittableSerializer();
>> >>> > > >
>> >>> > > >          Optional<SimpleVersionedSerializer<GCommT>>
>> >>> > > > getGlobalCommittableSerializer();
>> >>> > > > }
>> >>> > > >
>> >>> > > > /**
>> >>> > > >   * The {@link GlobalCommitter} is responsible for committing an
>> >>> > > aggregated
>> >>> > > > committable, which we called global committables.
>> >>> > > >   */
>> >>> > > > interface GlobalCommitter<CommT, GCommT> {
>> >>> > > >
>> >>> > > >          /**
>> >>> > > >           * This method is called when restoring from a
>> failover.
>> >>> > > >           * @param globalCommittables the global committables
>> that
>> >>> are
>> >>> > > not
>> >>> > > > committed in the previous session.
>> >>> > > >           * @return the global committables that should be
>> >>> committed
>> >>> > > again
>> >>> > > > in the current session.
>> >>> > > >           */
>> >>> > > >          List<GCommT> filterRecoveredCommittables(List<GCommT>
>> >>> > > > globalCommittables);
>> >>> > > >
>> >>> > > >          /**
>> >>> > > >           * Compute an aggregated committable from a collection
>> of
>> >>> > > > committables.
>> >>> > > >           * @param committables a collection of committables
>> that
>> >>> are
>> >>> > > needed
>> >>> > > > to combine
>> >>> > > >           * @return an aggregated committable
>> >>> > > >           */
>> >>> > > >          GCommT combine(List<CommT> committables);
>> >>> > > >
>> >>> > > >          void commit(List<GCommT> globalCommittables);
>> >>> > > >
>> >>> > > >          /**
>> >>> > > >           * There are no committables any more.
>> >>> > > >           */
>> >>> > > >          void endOfInput();
>> >>> > > > }
>> >>> > > >
>> >>> > > > Best,
>> >>> > > > Guowei
>> >>> > >
>> >>> > >
>> >>> >
>> >>>
>> >>
>>
>

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

Posted by Steven Wu <st...@gmail.com>.
> 1. The frame can not know which `GlobalCommT` to retry if we use the
> List<GlobalCommT> as parameter when the `commit` returns `RETRY`.
> 2. Of course we can let the `commit` return more detailed info but it
might
> be too complicated.

If commit(List<GlobalCommT>) returns RETRY, it means the whole list needs
to be retried. E.g. we have some outage with metadata service, commits for
checkpoints 1-100 failed. We can accumulate 100 GlobalCommT items. we don't
want to commit them one by one. It is faster to commit the whole list as
one batch.

> 3. On the other hand, I think only when restoring IcebergSink needs a
> collection of `GlobalCommT` and giving back another collection of
> `GlobalCommT` that are not committed

That is when the job restarted due to failure or deployment.


On Fri, Sep 25, 2020 at 5:24 AM Guowei Ma <gu...@gmail.com> wrote:

> Hi, all
>
> From the above discussion we could find that FLIP focuses on providing an
> unified transactional sink API. So I updated the FLIP's title to "Unified
> Transactional Sink API". But I found that the old link could not be opened
> again.
>
> I would update the link[1] here. Sorry for the inconvenience.
>
> [1]https://cwiki.apache.org/confluence/x/KEJ4CQ
>
> Best,
> Guowei
>
>
> On Fri, Sep 25, 2020 at 3:26 PM Guowei Ma <gu...@gmail.com> wrote:
>
> > Hi, Steven
> >
> > >>I also have a clarifying question regarding the WriterStateT. Since
> > >>IcebergWriter won't need to checkpoint any state, should we set it to
> > *Void*
> > >>type? Since getWriterStateSerializer() returns Optional, that is clear
> > and
> > >>we can return Optional.empty().
> >
> > Yes I think you could do it. If you return Optional.empty() we would
> > ignore all the state you return.
> >
> > Best,
> > Guowei
> >
> >
> > On Fri, Sep 25, 2020 at 3:14 PM Guowei Ma <gu...@gmail.com> wrote:
> >
> >> Hi,Steven
> >>
> >> Thank you for reading the FLIP so carefully.
> >> 1. The frame can not know which `GlobalCommT` to retry if we use the
> >> List<GlobalCommT> as parameter when the `commit` returns `RETRY`.
> >> 2. Of course we can let the `commit` return more detailed info but it
> >> might be too complicated.
> >> 3. On the other hand, I think only when restoring IcebergSink needs a
> >> collection of `GlobalCommT` and giving back another collection of
> >> `GlobalCommT` that are not committed.
> >>
> >> Best,
> >> Guowei
> >>
> >>
> >> On Fri, Sep 25, 2020 at 1:45 AM Steven Wu <st...@gmail.com> wrote:
> >>
> >>> Guowei,
> >>>
> >>> Thanks a lot for updating the wiki page. It looks great.
> >>>
> >>> I noticed one inconsistency in the wiki with your last summary email
> for
> >>> GlobalCommitter interface. I think the version in the summary email is
> >>> the
> >>> intended one, because rollover from previous failed commits can
> >>> accumulate
> >>> a list.
> >>> CommitResult commit(GlobalCommT globalCommittable); // in the wiki
> >>> =>
> >>> CommitResult commit(List<GlobalCommT> globalCommittable);  // in the
> >>> summary email
> >>>
> >>> I also have a clarifying question regarding the WriterStateT. Since
> >>> IcebergWriter won't need to checkpoint any state, should we set it to
> >>> *Void*
> >>> type? Since getWriterStateSerializer() returns Optional, that is clear
> >>> and
> >>> we can return Optional.empty().
> >>>
> >>> Thanks,
> >>> Steven
> >>>
> >>> On Wed, Sep 23, 2020 at 6:59 PM Guowei Ma <gu...@gmail.com>
> wrote:
> >>>
> >>> > Thanks Aljoscha for your suggestion.  I have updated FLIP. Any
> >>> comments are
> >>> > welcome.
> >>> >
> >>> > Best,
> >>> > Guowei
> >>> >
> >>> >
> >>> > On Wed, Sep 23, 2020 at 4:25 PM Aljoscha Krettek <
> aljoscha@apache.org>
> >>> > wrote:
> >>> >
> >>> > > Yes, that sounds good! I'll probably have some comments on the FLIP
> >>> > > about the names of generic parameters and the Javadoc but we can
> >>> address
> >>> > > them later or during implementation.
> >>> > >
> >>> > > I also think that we probably need the FAIL,RETRY,SUCCESS result
> for
> >>> > > globalCommit() but we can also do that as a later addition.
> >>> > >
> >>> > > So I think we're good to go to update the FLIP, do any last minute
> >>> > > changes and then vote.
> >>> > >
> >>> > > Best,
> >>> > > Aljoscha
> >>> > >
> >>> > > On 23.09.20 06:13, Guowei Ma wrote:
> >>> > > > Hi, all
> >>> > > >
> >>> > > > Thank everyone very much for your ideas and suggestions. I would
> >>> try to
> >>> > > > summarize again the consensus :). Correct me if I am wrong or
> >>> > > misunderstand
> >>> > > > you.
> >>> > > >
> >>> > > > ## Consensus-1
> >>> > > >
> >>> > > > 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.
> >>> > > >
> >>> > > > ## Consensus-2
> >>> > > >
> >>> > > > 1. What should the “Unified Sink API” support/cover? It includes
> >>> two
> >>> > > > aspects. 1. The same sink implementation would work for both the
> >>> batch
> >>> > > and
> >>> > > > stream execution mode. 2. In the long run we should give the sink
> >>> > > developer
> >>> > > > the ability of building “arbitrary” topologies. But for
> Flink-1.12
> >>> we
> >>> > > > should be more focused on only satisfying the S3/HDFS/Iceberg
> sink.
> >>> > > > 2. Because the batch execution mode does not have the normal
> >>> checkpoint
> >>> > > the
> >>> > > > sink developer should not depend on it any more if we want a
> >>> unified
> >>> > > sink.
> >>> > > > 3. We can benefit by providing an asynchronous `Writer` version.
> >>> But
> >>> > > > because the unified sink is already very complicated, we don’t
> add
> >>> this
> >>> > > in
> >>> > > > the first version.
> >>> > > >
> >>> > > >
> >>> > > > According to these consensus I would propose the first version of
> >>> the
> >>> > new
> >>> > > > sink api as follows. What do you think? Any comments are welcome.
> >>> > > >
> >>> > > > /**
> >>> > > >   * This interface lets the sink developer build a simple
> >>> transactional
> >>> > > sink
> >>> > > > topology pattern, which satisfies the HDFS/S3/Iceberg sink.
> >>> > > >   * This sink topology includes one {@link Writer} + one {@link
> >>> > > Committer} +
> >>> > > > one {@link GlobalCommitter}.
> >>> > > >   * The {@link Writer} is responsible for producing the
> >>> committable.
> >>> > > >   * The {@link Committer} is responsible for committing a single
> >>> > > > committables.
> >>> > > >   * The {@link GlobalCommitter} is responsible for committing an
> >>> > > aggregated
> >>> > > > committable, which we called global committables.
> >>> > > >   *
> >>> > > >   * But both the {@link Committer} and the {@link
> GlobalCommitter}
> >>> are
> >>> > > > optional.
> >>> > > >   */
> >>> > > > interface TSink<IN, CommT, GCommT, WriterS> {
> >>> > > >
> >>> > > >          Writer<IN, CommT, WriterS> createWriter(InitContext
> >>> > > initContext);
> >>> > > >
> >>> > > >          Writer<IN, CommT, WriterS> restoreWriter(InitContext
> >>> > > initContext,
> >>> > > > List<WriterS> states);
> >>> > > >
> >>> > > >          Optional<Committer<CommT>> createCommitter();
> >>> > > >
> >>> > > >          Optional<GlobalCommitter<CommT, GCommT>>
> >>> > > createGlobalCommitter();
> >>> > > >
> >>> > > >          SimpleVersionedSerializer<CommT>
> >>> getCommittableSerializer();
> >>> > > >
> >>> > > >          Optional<SimpleVersionedSerializer<GCommT>>
> >>> > > > getGlobalCommittableSerializer();
> >>> > > > }
> >>> > > >
> >>> > > > /**
> >>> > > >   * The {@link GlobalCommitter} is responsible for committing an
> >>> > > aggregated
> >>> > > > committable, which we called global committables.
> >>> > > >   */
> >>> > > > interface GlobalCommitter<CommT, GCommT> {
> >>> > > >
> >>> > > >          /**
> >>> > > >           * This method is called when restoring from a failover.
> >>> > > >           * @param globalCommittables the global committables
> that
> >>> are
> >>> > > not
> >>> > > > committed in the previous session.
> >>> > > >           * @return the global committables that should be
> >>> committed
> >>> > > again
> >>> > > > in the current session.
> >>> > > >           */
> >>> > > >          List<GCommT> filterRecoveredCommittables(List<GCommT>
> >>> > > > globalCommittables);
> >>> > > >
> >>> > > >          /**
> >>> > > >           * Compute an aggregated committable from a collection
> of
> >>> > > > committables.
> >>> > > >           * @param committables a collection of committables that
> >>> are
> >>> > > needed
> >>> > > > to combine
> >>> > > >           * @return an aggregated committable
> >>> > > >           */
> >>> > > >          GCommT combine(List<CommT> committables);
> >>> > > >
> >>> > > >          void commit(List<GCommT> globalCommittables);
> >>> > > >
> >>> > > >          /**
> >>> > > >           * There are no committables any more.
> >>> > > >           */
> >>> > > >          void endOfInput();
> >>> > > > }
> >>> > > >
> >>> > > > Best,
> >>> > > > Guowei
> >>> > >
> >>> > >
> >>> >
> >>>
> >>
>

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

Posted by Guowei Ma <gu...@gmail.com>.
Hi, all

From the above discussion we could find that FLIP focuses on providing an
unified transactional sink API. So I updated the FLIP's title to "Unified
Transactional Sink API". But I found that the old link could not be opened
again.

I would update the link[1] here. Sorry for the inconvenience.

[1]https://cwiki.apache.org/confluence/x/KEJ4CQ

Best,
Guowei


On Fri, Sep 25, 2020 at 3:26 PM Guowei Ma <gu...@gmail.com> wrote:

> Hi, Steven
>
> >>I also have a clarifying question regarding the WriterStateT. Since
> >>IcebergWriter won't need to checkpoint any state, should we set it to
> *Void*
> >>type? Since getWriterStateSerializer() returns Optional, that is clear
> and
> >>we can return Optional.empty().
>
> Yes I think you could do it. If you return Optional.empty() we would
> ignore all the state you return.
>
> Best,
> Guowei
>
>
> On Fri, Sep 25, 2020 at 3:14 PM Guowei Ma <gu...@gmail.com> wrote:
>
>> Hi,Steven
>>
>> Thank you for reading the FLIP so carefully.
>> 1. The frame can not know which `GlobalCommT` to retry if we use the
>> List<GlobalCommT> as parameter when the `commit` returns `RETRY`.
>> 2. Of course we can let the `commit` return more detailed info but it
>> might be too complicated.
>> 3. On the other hand, I think only when restoring IcebergSink needs a
>> collection of `GlobalCommT` and giving back another collection of
>> `GlobalCommT` that are not committed.
>>
>> Best,
>> Guowei
>>
>>
>> On Fri, Sep 25, 2020 at 1:45 AM Steven Wu <st...@gmail.com> wrote:
>>
>>> Guowei,
>>>
>>> Thanks a lot for updating the wiki page. It looks great.
>>>
>>> I noticed one inconsistency in the wiki with your last summary email for
>>> GlobalCommitter interface. I think the version in the summary email is
>>> the
>>> intended one, because rollover from previous failed commits can
>>> accumulate
>>> a list.
>>> CommitResult commit(GlobalCommT globalCommittable); // in the wiki
>>> =>
>>> CommitResult commit(List<GlobalCommT> globalCommittable);  // in the
>>> summary email
>>>
>>> I also have a clarifying question regarding the WriterStateT. Since
>>> IcebergWriter won't need to checkpoint any state, should we set it to
>>> *Void*
>>> type? Since getWriterStateSerializer() returns Optional, that is clear
>>> and
>>> we can return Optional.empty().
>>>
>>> Thanks,
>>> Steven
>>>
>>> On Wed, Sep 23, 2020 at 6:59 PM Guowei Ma <gu...@gmail.com> wrote:
>>>
>>> > Thanks Aljoscha for your suggestion.  I have updated FLIP. Any
>>> comments are
>>> > welcome.
>>> >
>>> > Best,
>>> > Guowei
>>> >
>>> >
>>> > On Wed, Sep 23, 2020 at 4:25 PM Aljoscha Krettek <al...@apache.org>
>>> > wrote:
>>> >
>>> > > Yes, that sounds good! I'll probably have some comments on the FLIP
>>> > > about the names of generic parameters and the Javadoc but we can
>>> address
>>> > > them later or during implementation.
>>> > >
>>> > > I also think that we probably need the FAIL,RETRY,SUCCESS result for
>>> > > globalCommit() but we can also do that as a later addition.
>>> > >
>>> > > So I think we're good to go to update the FLIP, do any last minute
>>> > > changes and then vote.
>>> > >
>>> > > Best,
>>> > > Aljoscha
>>> > >
>>> > > On 23.09.20 06:13, Guowei Ma wrote:
>>> > > > Hi, all
>>> > > >
>>> > > > Thank everyone very much for your ideas and suggestions. I would
>>> try to
>>> > > > summarize again the consensus :). Correct me if I am wrong or
>>> > > misunderstand
>>> > > > you.
>>> > > >
>>> > > > ## Consensus-1
>>> > > >
>>> > > > 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.
>>> > > >
>>> > > > ## Consensus-2
>>> > > >
>>> > > > 1. What should the “Unified Sink API” support/cover? It includes
>>> two
>>> > > > aspects. 1. The same sink implementation would work for both the
>>> batch
>>> > > and
>>> > > > stream execution mode. 2. In the long run we should give the sink
>>> > > developer
>>> > > > the ability of building “arbitrary” topologies. But for Flink-1.12
>>> we
>>> > > > should be more focused on only satisfying the S3/HDFS/Iceberg sink.
>>> > > > 2. Because the batch execution mode does not have the normal
>>> checkpoint
>>> > > the
>>> > > > sink developer should not depend on it any more if we want a
>>> unified
>>> > > sink.
>>> > > > 3. We can benefit by providing an asynchronous `Writer` version.
>>> But
>>> > > > because the unified sink is already very complicated, we don’t add
>>> this
>>> > > in
>>> > > > the first version.
>>> > > >
>>> > > >
>>> > > > According to these consensus I would propose the first version of
>>> the
>>> > new
>>> > > > sink api as follows. What do you think? Any comments are welcome.
>>> > > >
>>> > > > /**
>>> > > >   * This interface lets the sink developer build a simple
>>> transactional
>>> > > sink
>>> > > > topology pattern, which satisfies the HDFS/S3/Iceberg sink.
>>> > > >   * This sink topology includes one {@link Writer} + one {@link
>>> > > Committer} +
>>> > > > one {@link GlobalCommitter}.
>>> > > >   * The {@link Writer} is responsible for producing the
>>> committable.
>>> > > >   * The {@link Committer} is responsible for committing a single
>>> > > > committables.
>>> > > >   * The {@link GlobalCommitter} is responsible for committing an
>>> > > aggregated
>>> > > > committable, which we called global committables.
>>> > > >   *
>>> > > >   * But both the {@link Committer} and the {@link GlobalCommitter}
>>> are
>>> > > > optional.
>>> > > >   */
>>> > > > interface TSink<IN, CommT, GCommT, WriterS> {
>>> > > >
>>> > > >          Writer<IN, CommT, WriterS> createWriter(InitContext
>>> > > initContext);
>>> > > >
>>> > > >          Writer<IN, CommT, WriterS> restoreWriter(InitContext
>>> > > initContext,
>>> > > > List<WriterS> states);
>>> > > >
>>> > > >          Optional<Committer<CommT>> createCommitter();
>>> > > >
>>> > > >          Optional<GlobalCommitter<CommT, GCommT>>
>>> > > createGlobalCommitter();
>>> > > >
>>> > > >          SimpleVersionedSerializer<CommT>
>>> getCommittableSerializer();
>>> > > >
>>> > > >          Optional<SimpleVersionedSerializer<GCommT>>
>>> > > > getGlobalCommittableSerializer();
>>> > > > }
>>> > > >
>>> > > > /**
>>> > > >   * The {@link GlobalCommitter} is responsible for committing an
>>> > > aggregated
>>> > > > committable, which we called global committables.
>>> > > >   */
>>> > > > interface GlobalCommitter<CommT, GCommT> {
>>> > > >
>>> > > >          /**
>>> > > >           * This method is called when restoring from a failover.
>>> > > >           * @param globalCommittables the global committables that
>>> are
>>> > > not
>>> > > > committed in the previous session.
>>> > > >           * @return the global committables that should be
>>> committed
>>> > > again
>>> > > > in the current session.
>>> > > >           */
>>> > > >          List<GCommT> filterRecoveredCommittables(List<GCommT>
>>> > > > globalCommittables);
>>> > > >
>>> > > >          /**
>>> > > >           * Compute an aggregated committable from a collection of
>>> > > > committables.
>>> > > >           * @param committables a collection of committables that
>>> are
>>> > > needed
>>> > > > to combine
>>> > > >           * @return an aggregated committable
>>> > > >           */
>>> > > >          GCommT combine(List<CommT> committables);
>>> > > >
>>> > > >          void commit(List<GCommT> globalCommittables);
>>> > > >
>>> > > >          /**
>>> > > >           * There are no committables any more.
>>> > > >           */
>>> > > >          void endOfInput();
>>> > > > }
>>> > > >
>>> > > > Best,
>>> > > > Guowei
>>> > >
>>> > >
>>> >
>>>
>>

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

Posted by Guowei Ma <gu...@gmail.com>.
Hi, Steven

>>I also have a clarifying question regarding the WriterStateT. Since
>>IcebergWriter won't need to checkpoint any state, should we set it to
*Void*
>>type? Since getWriterStateSerializer() returns Optional, that is clear and
>>we can return Optional.empty().

Yes I think you could do it. If you return Optional.empty() we would ignore
all the state you return.

Best,
Guowei


On Fri, Sep 25, 2020 at 3:14 PM Guowei Ma <gu...@gmail.com> wrote:

> Hi,Steven
>
> Thank you for reading the FLIP so carefully.
> 1. The frame can not know which `GlobalCommT` to retry if we use the
> List<GlobalCommT> as parameter when the `commit` returns `RETRY`.
> 2. Of course we can let the `commit` return more detailed info but it
> might be too complicated.
> 3. On the other hand, I think only when restoring IcebergSink needs a
> collection of `GlobalCommT` and giving back another collection of
> `GlobalCommT` that are not committed.
>
> Best,
> Guowei
>
>
> On Fri, Sep 25, 2020 at 1:45 AM Steven Wu <st...@gmail.com> wrote:
>
>> Guowei,
>>
>> Thanks a lot for updating the wiki page. It looks great.
>>
>> I noticed one inconsistency in the wiki with your last summary email for
>> GlobalCommitter interface. I think the version in the summary email is the
>> intended one, because rollover from previous failed commits can accumulate
>> a list.
>> CommitResult commit(GlobalCommT globalCommittable); // in the wiki
>> =>
>> CommitResult commit(List<GlobalCommT> globalCommittable);  // in the
>> summary email
>>
>> I also have a clarifying question regarding the WriterStateT. Since
>> IcebergWriter won't need to checkpoint any state, should we set it to
>> *Void*
>> type? Since getWriterStateSerializer() returns Optional, that is clear and
>> we can return Optional.empty().
>>
>> Thanks,
>> Steven
>>
>> On Wed, Sep 23, 2020 at 6:59 PM Guowei Ma <gu...@gmail.com> wrote:
>>
>> > Thanks Aljoscha for your suggestion.  I have updated FLIP. Any comments
>> are
>> > welcome.
>> >
>> > Best,
>> > Guowei
>> >
>> >
>> > On Wed, Sep 23, 2020 at 4:25 PM Aljoscha Krettek <al...@apache.org>
>> > wrote:
>> >
>> > > Yes, that sounds good! I'll probably have some comments on the FLIP
>> > > about the names of generic parameters and the Javadoc but we can
>> address
>> > > them later or during implementation.
>> > >
>> > > I also think that we probably need the FAIL,RETRY,SUCCESS result for
>> > > globalCommit() but we can also do that as a later addition.
>> > >
>> > > So I think we're good to go to update the FLIP, do any last minute
>> > > changes and then vote.
>> > >
>> > > Best,
>> > > Aljoscha
>> > >
>> > > On 23.09.20 06:13, Guowei Ma wrote:
>> > > > Hi, all
>> > > >
>> > > > Thank everyone very much for your ideas and suggestions. I would
>> try to
>> > > > summarize again the consensus :). Correct me if I am wrong or
>> > > misunderstand
>> > > > you.
>> > > >
>> > > > ## Consensus-1
>> > > >
>> > > > 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.
>> > > >
>> > > > ## Consensus-2
>> > > >
>> > > > 1. What should the “Unified Sink API” support/cover? It includes two
>> > > > aspects. 1. The same sink implementation would work for both the
>> batch
>> > > and
>> > > > stream execution mode. 2. In the long run we should give the sink
>> > > developer
>> > > > the ability of building “arbitrary” topologies. But for Flink-1.12
>> we
>> > > > should be more focused on only satisfying the S3/HDFS/Iceberg sink.
>> > > > 2. Because the batch execution mode does not have the normal
>> checkpoint
>> > > the
>> > > > sink developer should not depend on it any more if we want a unified
>> > > sink.
>> > > > 3. We can benefit by providing an asynchronous `Writer` version. But
>> > > > because the unified sink is already very complicated, we don’t add
>> this
>> > > in
>> > > > the first version.
>> > > >
>> > > >
>> > > > According to these consensus I would propose the first version of
>> the
>> > new
>> > > > sink api as follows. What do you think? Any comments are welcome.
>> > > >
>> > > > /**
>> > > >   * This interface lets the sink developer build a simple
>> transactional
>> > > sink
>> > > > topology pattern, which satisfies the HDFS/S3/Iceberg sink.
>> > > >   * This sink topology includes one {@link Writer} + one {@link
>> > > Committer} +
>> > > > one {@link GlobalCommitter}.
>> > > >   * The {@link Writer} is responsible for producing the committable.
>> > > >   * The {@link Committer} is responsible for committing a single
>> > > > committables.
>> > > >   * The {@link GlobalCommitter} is responsible for committing an
>> > > aggregated
>> > > > committable, which we called global committables.
>> > > >   *
>> > > >   * But both the {@link Committer} and the {@link GlobalCommitter}
>> are
>> > > > optional.
>> > > >   */
>> > > > interface TSink<IN, CommT, GCommT, WriterS> {
>> > > >
>> > > >          Writer<IN, CommT, WriterS> createWriter(InitContext
>> > > initContext);
>> > > >
>> > > >          Writer<IN, CommT, WriterS> restoreWriter(InitContext
>> > > initContext,
>> > > > List<WriterS> states);
>> > > >
>> > > >          Optional<Committer<CommT>> createCommitter();
>> > > >
>> > > >          Optional<GlobalCommitter<CommT, GCommT>>
>> > > createGlobalCommitter();
>> > > >
>> > > >          SimpleVersionedSerializer<CommT>
>> getCommittableSerializer();
>> > > >
>> > > >          Optional<SimpleVersionedSerializer<GCommT>>
>> > > > getGlobalCommittableSerializer();
>> > > > }
>> > > >
>> > > > /**
>> > > >   * The {@link GlobalCommitter} is responsible for committing an
>> > > aggregated
>> > > > committable, which we called global committables.
>> > > >   */
>> > > > interface GlobalCommitter<CommT, GCommT> {
>> > > >
>> > > >          /**
>> > > >           * This method is called when restoring from a failover.
>> > > >           * @param globalCommittables the global committables that
>> are
>> > > not
>> > > > committed in the previous session.
>> > > >           * @return the global committables that should be committed
>> > > again
>> > > > in the current session.
>> > > >           */
>> > > >          List<GCommT> filterRecoveredCommittables(List<GCommT>
>> > > > globalCommittables);
>> > > >
>> > > >          /**
>> > > >           * Compute an aggregated committable from a collection of
>> > > > committables.
>> > > >           * @param committables a collection of committables that
>> are
>> > > needed
>> > > > to combine
>> > > >           * @return an aggregated committable
>> > > >           */
>> > > >          GCommT combine(List<CommT> committables);
>> > > >
>> > > >          void commit(List<GCommT> globalCommittables);
>> > > >
>> > > >          /**
>> > > >           * There are no committables any more.
>> > > >           */
>> > > >          void endOfInput();
>> > > > }
>> > > >
>> > > > Best,
>> > > > Guowei
>> > >
>> > >
>> >
>>
>

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

Posted by Guowei Ma <gu...@gmail.com>.
Hi,Steven

Thank you for reading the FLIP so carefully.
1. The frame can not know which `GlobalCommT` to retry if we use the
List<GlobalCommT> as parameter when the `commit` returns `RETRY`.
2. Of course we can let the `commit` return more detailed info but it might
be too complicated.
3. On the other hand, I think only when restoring IcebergSink needs a
collection of `GlobalCommT` and giving back another collection of
`GlobalCommT` that are not committed.

Best,
Guowei


On Fri, Sep 25, 2020 at 1:45 AM Steven Wu <st...@gmail.com> wrote:

> Guowei,
>
> Thanks a lot for updating the wiki page. It looks great.
>
> I noticed one inconsistency in the wiki with your last summary email for
> GlobalCommitter interface. I think the version in the summary email is the
> intended one, because rollover from previous failed commits can accumulate
> a list.
> CommitResult commit(GlobalCommT globalCommittable); // in the wiki
> =>
> CommitResult commit(List<GlobalCommT> globalCommittable);  // in the
> summary email
>
> I also have a clarifying question regarding the WriterStateT. Since
> IcebergWriter won't need to checkpoint any state, should we set it to
> *Void*
> type? Since getWriterStateSerializer() returns Optional, that is clear and
> we can return Optional.empty().
>
> Thanks,
> Steven
>
> On Wed, Sep 23, 2020 at 6:59 PM Guowei Ma <gu...@gmail.com> wrote:
>
> > Thanks Aljoscha for your suggestion.  I have updated FLIP. Any comments
> are
> > welcome.
> >
> > Best,
> > Guowei
> >
> >
> > On Wed, Sep 23, 2020 at 4:25 PM Aljoscha Krettek <al...@apache.org>
> > wrote:
> >
> > > Yes, that sounds good! I'll probably have some comments on the FLIP
> > > about the names of generic parameters and the Javadoc but we can
> address
> > > them later or during implementation.
> > >
> > > I also think that we probably need the FAIL,RETRY,SUCCESS result for
> > > globalCommit() but we can also do that as a later addition.
> > >
> > > So I think we're good to go to update the FLIP, do any last minute
> > > changes and then vote.
> > >
> > > Best,
> > > Aljoscha
> > >
> > > On 23.09.20 06:13, Guowei Ma wrote:
> > > > Hi, all
> > > >
> > > > Thank everyone very much for your ideas and suggestions. I would try
> to
> > > > summarize again the consensus :). Correct me if I am wrong or
> > > misunderstand
> > > > you.
> > > >
> > > > ## Consensus-1
> > > >
> > > > 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.
> > > >
> > > > ## Consensus-2
> > > >
> > > > 1. What should the “Unified Sink API” support/cover? It includes two
> > > > aspects. 1. The same sink implementation would work for both the
> batch
> > > and
> > > > stream execution mode. 2. In the long run we should give the sink
> > > developer
> > > > the ability of building “arbitrary” topologies. But for Flink-1.12 we
> > > > should be more focused on only satisfying the S3/HDFS/Iceberg sink.
> > > > 2. Because the batch execution mode does not have the normal
> checkpoint
> > > the
> > > > sink developer should not depend on it any more if we want a unified
> > > sink.
> > > > 3. We can benefit by providing an asynchronous `Writer` version. But
> > > > because the unified sink is already very complicated, we don’t add
> this
> > > in
> > > > the first version.
> > > >
> > > >
> > > > According to these consensus I would propose the first version of the
> > new
> > > > sink api as follows. What do you think? Any comments are welcome.
> > > >
> > > > /**
> > > >   * This interface lets the sink developer build a simple
> transactional
> > > sink
> > > > topology pattern, which satisfies the HDFS/S3/Iceberg sink.
> > > >   * This sink topology includes one {@link Writer} + one {@link
> > > Committer} +
> > > > one {@link GlobalCommitter}.
> > > >   * The {@link Writer} is responsible for producing the committable.
> > > >   * The {@link Committer} is responsible for committing a single
> > > > committables.
> > > >   * The {@link GlobalCommitter} is responsible for committing an
> > > aggregated
> > > > committable, which we called global committables.
> > > >   *
> > > >   * But both the {@link Committer} and the {@link GlobalCommitter}
> are
> > > > optional.
> > > >   */
> > > > interface TSink<IN, CommT, GCommT, WriterS> {
> > > >
> > > >          Writer<IN, CommT, WriterS> createWriter(InitContext
> > > initContext);
> > > >
> > > >          Writer<IN, CommT, WriterS> restoreWriter(InitContext
> > > initContext,
> > > > List<WriterS> states);
> > > >
> > > >          Optional<Committer<CommT>> createCommitter();
> > > >
> > > >          Optional<GlobalCommitter<CommT, GCommT>>
> > > createGlobalCommitter();
> > > >
> > > >          SimpleVersionedSerializer<CommT> getCommittableSerializer();
> > > >
> > > >          Optional<SimpleVersionedSerializer<GCommT>>
> > > > getGlobalCommittableSerializer();
> > > > }
> > > >
> > > > /**
> > > >   * The {@link GlobalCommitter} is responsible for committing an
> > > aggregated
> > > > committable, which we called global committables.
> > > >   */
> > > > interface GlobalCommitter<CommT, GCommT> {
> > > >
> > > >          /**
> > > >           * This method is called when restoring from a failover.
> > > >           * @param globalCommittables the global committables that
> are
> > > not
> > > > committed in the previous session.
> > > >           * @return the global committables that should be committed
> > > again
> > > > in the current session.
> > > >           */
> > > >          List<GCommT> filterRecoveredCommittables(List<GCommT>
> > > > globalCommittables);
> > > >
> > > >          /**
> > > >           * Compute an aggregated committable from a collection of
> > > > committables.
> > > >           * @param committables a collection of committables that are
> > > needed
> > > > to combine
> > > >           * @return an aggregated committable
> > > >           */
> > > >          GCommT combine(List<CommT> committables);
> > > >
> > > >          void commit(List<GCommT> globalCommittables);
> > > >
> > > >          /**
> > > >           * There are no committables any more.
> > > >           */
> > > >          void endOfInput();
> > > > }
> > > >
> > > > Best,
> > > > Guowei
> > >
> > >
> >
>

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

Posted by Steven Wu <st...@gmail.com>.
Guowei,

Thanks a lot for updating the wiki page. It looks great.

I noticed one inconsistency in the wiki with your last summary email for
GlobalCommitter interface. I think the version in the summary email is the
intended one, because rollover from previous failed commits can accumulate
a list.
CommitResult commit(GlobalCommT globalCommittable); // in the wiki
=>
CommitResult commit(List<GlobalCommT> globalCommittable);  // in the
summary email

I also have a clarifying question regarding the WriterStateT. Since
IcebergWriter won't need to checkpoint any state, should we set it to *Void*
type? Since getWriterStateSerializer() returns Optional, that is clear and
we can return Optional.empty().

Thanks,
Steven

On Wed, Sep 23, 2020 at 6:59 PM Guowei Ma <gu...@gmail.com> wrote:

> Thanks Aljoscha for your suggestion.  I have updated FLIP. Any comments are
> welcome.
>
> Best,
> Guowei
>
>
> On Wed, Sep 23, 2020 at 4:25 PM Aljoscha Krettek <al...@apache.org>
> wrote:
>
> > Yes, that sounds good! I'll probably have some comments on the FLIP
> > about the names of generic parameters and the Javadoc but we can address
> > them later or during implementation.
> >
> > I also think that we probably need the FAIL,RETRY,SUCCESS result for
> > globalCommit() but we can also do that as a later addition.
> >
> > So I think we're good to go to update the FLIP, do any last minute
> > changes and then vote.
> >
> > Best,
> > Aljoscha
> >
> > On 23.09.20 06:13, Guowei Ma wrote:
> > > Hi, all
> > >
> > > Thank everyone very much for your ideas and suggestions. I would try to
> > > summarize again the consensus :). Correct me if I am wrong or
> > misunderstand
> > > you.
> > >
> > > ## Consensus-1
> > >
> > > 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.
> > >
> > > ## Consensus-2
> > >
> > > 1. What should the “Unified Sink API” support/cover? It includes two
> > > aspects. 1. The same sink implementation would work for both the batch
> > and
> > > stream execution mode. 2. In the long run we should give the sink
> > developer
> > > the ability of building “arbitrary” topologies. But for Flink-1.12 we
> > > should be more focused on only satisfying the S3/HDFS/Iceberg sink.
> > > 2. Because the batch execution mode does not have the normal checkpoint
> > the
> > > sink developer should not depend on it any more if we want a unified
> > sink.
> > > 3. We can benefit by providing an asynchronous `Writer` version. But
> > > because the unified sink is already very complicated, we don’t add this
> > in
> > > the first version.
> > >
> > >
> > > According to these consensus I would propose the first version of the
> new
> > > sink api as follows. What do you think? Any comments are welcome.
> > >
> > > /**
> > >   * This interface lets the sink developer build a simple transactional
> > sink
> > > topology pattern, which satisfies the HDFS/S3/Iceberg sink.
> > >   * This sink topology includes one {@link Writer} + one {@link
> > Committer} +
> > > one {@link GlobalCommitter}.
> > >   * The {@link Writer} is responsible for producing the committable.
> > >   * The {@link Committer} is responsible for committing a single
> > > committables.
> > >   * The {@link GlobalCommitter} is responsible for committing an
> > aggregated
> > > committable, which we called global committables.
> > >   *
> > >   * But both the {@link Committer} and the {@link GlobalCommitter} are
> > > optional.
> > >   */
> > > interface TSink<IN, CommT, GCommT, WriterS> {
> > >
> > >          Writer<IN, CommT, WriterS> createWriter(InitContext
> > initContext);
> > >
> > >          Writer<IN, CommT, WriterS> restoreWriter(InitContext
> > initContext,
> > > List<WriterS> states);
> > >
> > >          Optional<Committer<CommT>> createCommitter();
> > >
> > >          Optional<GlobalCommitter<CommT, GCommT>>
> > createGlobalCommitter();
> > >
> > >          SimpleVersionedSerializer<CommT> getCommittableSerializer();
> > >
> > >          Optional<SimpleVersionedSerializer<GCommT>>
> > > getGlobalCommittableSerializer();
> > > }
> > >
> > > /**
> > >   * The {@link GlobalCommitter} is responsible for committing an
> > aggregated
> > > committable, which we called global committables.
> > >   */
> > > interface GlobalCommitter<CommT, GCommT> {
> > >
> > >          /**
> > >           * This method is called when restoring from a failover.
> > >           * @param globalCommittables the global committables that are
> > not
> > > committed in the previous session.
> > >           * @return the global committables that should be committed
> > again
> > > in the current session.
> > >           */
> > >          List<GCommT> filterRecoveredCommittables(List<GCommT>
> > > globalCommittables);
> > >
> > >          /**
> > >           * Compute an aggregated committable from a collection of
> > > committables.
> > >           * @param committables a collection of committables that are
> > needed
> > > to combine
> > >           * @return an aggregated committable
> > >           */
> > >          GCommT combine(List<CommT> committables);
> > >
> > >          void commit(List<GCommT> globalCommittables);
> > >
> > >          /**
> > >           * There are no committables any more.
> > >           */
> > >          void endOfInput();
> > > }
> > >
> > > Best,
> > > Guowei
> >
> >
>

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

Posted by Guowei Ma <gu...@gmail.com>.
Thanks Aljoscha for your suggestion.  I have updated FLIP. Any comments are
welcome.

Best,
Guowei


On Wed, Sep 23, 2020 at 4:25 PM Aljoscha Krettek <al...@apache.org>
wrote:

> Yes, that sounds good! I'll probably have some comments on the FLIP
> about the names of generic parameters and the Javadoc but we can address
> them later or during implementation.
>
> I also think that we probably need the FAIL,RETRY,SUCCESS result for
> globalCommit() but we can also do that as a later addition.
>
> So I think we're good to go to update the FLIP, do any last minute
> changes and then vote.
>
> Best,
> Aljoscha
>
> On 23.09.20 06:13, Guowei Ma wrote:
> > Hi, all
> >
> > Thank everyone very much for your ideas and suggestions. I would try to
> > summarize again the consensus :). Correct me if I am wrong or
> misunderstand
> > you.
> >
> > ## Consensus-1
> >
> > 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.
> >
> > ## Consensus-2
> >
> > 1. What should the “Unified Sink API” support/cover? It includes two
> > aspects. 1. The same sink implementation would work for both the batch
> and
> > stream execution mode. 2. In the long run we should give the sink
> developer
> > the ability of building “arbitrary” topologies. But for Flink-1.12 we
> > should be more focused on only satisfying the S3/HDFS/Iceberg sink.
> > 2. Because the batch execution mode does not have the normal checkpoint
> the
> > sink developer should not depend on it any more if we want a unified
> sink.
> > 3. We can benefit by providing an asynchronous `Writer` version. But
> > because the unified sink is already very complicated, we don’t add this
> in
> > the first version.
> >
> >
> > According to these consensus I would propose the first version of the new
> > sink api as follows. What do you think? Any comments are welcome.
> >
> > /**
> >   * This interface lets the sink developer build a simple transactional
> sink
> > topology pattern, which satisfies the HDFS/S3/Iceberg sink.
> >   * This sink topology includes one {@link Writer} + one {@link
> Committer} +
> > one {@link GlobalCommitter}.
> >   * The {@link Writer} is responsible for producing the committable.
> >   * The {@link Committer} is responsible for committing a single
> > committables.
> >   * The {@link GlobalCommitter} is responsible for committing an
> aggregated
> > committable, which we called global committables.
> >   *
> >   * But both the {@link Committer} and the {@link GlobalCommitter} are
> > optional.
> >   */
> > interface TSink<IN, CommT, GCommT, WriterS> {
> >
> >          Writer<IN, CommT, WriterS> createWriter(InitContext
> initContext);
> >
> >          Writer<IN, CommT, WriterS> restoreWriter(InitContext
> initContext,
> > List<WriterS> states);
> >
> >          Optional<Committer<CommT>> createCommitter();
> >
> >          Optional<GlobalCommitter<CommT, GCommT>>
> createGlobalCommitter();
> >
> >          SimpleVersionedSerializer<CommT> getCommittableSerializer();
> >
> >          Optional<SimpleVersionedSerializer<GCommT>>
> > getGlobalCommittableSerializer();
> > }
> >
> > /**
> >   * The {@link GlobalCommitter} is responsible for committing an
> aggregated
> > committable, which we called global committables.
> >   */
> > interface GlobalCommitter<CommT, GCommT> {
> >
> >          /**
> >           * This method is called when restoring from a failover.
> >           * @param globalCommittables the global committables that are
> not
> > committed in the previous session.
> >           * @return the global committables that should be committed
> again
> > in the current session.
> >           */
> >          List<GCommT> filterRecoveredCommittables(List<GCommT>
> > globalCommittables);
> >
> >          /**
> >           * Compute an aggregated committable from a collection of
> > committables.
> >           * @param committables a collection of committables that are
> needed
> > to combine
> >           * @return an aggregated committable
> >           */
> >          GCommT combine(List<CommT> committables);
> >
> >          void commit(List<GCommT> globalCommittables);
> >
> >          /**
> >           * There are no committables any more.
> >           */
> >          void endOfInput();
> > }
> >
> > Best,
> > Guowei
>
>

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

Posted by Aljoscha Krettek <al...@apache.org>.
Yes, that sounds good! I'll probably have some comments on the FLIP 
about the names of generic parameters and the Javadoc but we can address 
them later or during implementation.

I also think that we probably need the FAIL,RETRY,SUCCESS result for 
globalCommit() but we can also do that as a later addition.

So I think we're good to go to update the FLIP, do any last minute 
changes and then vote.

Best,
Aljoscha

On 23.09.20 06:13, Guowei Ma wrote:
> Hi, all
> 
> Thank everyone very much for your ideas and suggestions. I would try to
> summarize again the consensus :). Correct me if I am wrong or misunderstand
> you.
> 
> ## Consensus-1
> 
> 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.
> 
> ## Consensus-2
> 
> 1. What should the “Unified Sink API” support/cover? It includes two
> aspects. 1. The same sink implementation would work for both the batch and
> stream execution mode. 2. In the long run we should give the sink developer
> the ability of building “arbitrary” topologies. But for Flink-1.12 we
> should be more focused on only satisfying the S3/HDFS/Iceberg sink.
> 2. Because the batch execution mode does not have the normal checkpoint the
> sink developer should not depend on it any more if we want a unified sink.
> 3. We can benefit by providing an asynchronous `Writer` version. But
> because the unified sink is already very complicated, we don’t add this in
> the first version.
> 
> 
> According to these consensus I would propose the first version of the new
> sink api as follows. What do you think? Any comments are welcome.
> 
> /**
>   * This interface lets the sink developer build a simple transactional sink
> topology pattern, which satisfies the HDFS/S3/Iceberg sink.
>   * This sink topology includes one {@link Writer} + one {@link Committer} +
> one {@link GlobalCommitter}.
>   * The {@link Writer} is responsible for producing the committable.
>   * The {@link Committer} is responsible for committing a single
> committables.
>   * The {@link GlobalCommitter} is responsible for committing an aggregated
> committable, which we called global committables.
>   *
>   * But both the {@link Committer} and the {@link GlobalCommitter} are
> optional.
>   */
> interface TSink<IN, CommT, GCommT, WriterS> {
> 
>          Writer<IN, CommT, WriterS> createWriter(InitContext initContext);
> 
>          Writer<IN, CommT, WriterS> restoreWriter(InitContext initContext,
> List<WriterS> states);
> 
>          Optional<Committer<CommT>> createCommitter();
> 
>          Optional<GlobalCommitter<CommT, GCommT>> createGlobalCommitter();
> 
>          SimpleVersionedSerializer<CommT> getCommittableSerializer();
> 
>          Optional<SimpleVersionedSerializer<GCommT>>
> getGlobalCommittableSerializer();
> }
> 
> /**
>   * The {@link GlobalCommitter} is responsible for committing an aggregated
> committable, which we called global committables.
>   */
> interface GlobalCommitter<CommT, GCommT> {
> 
>          /**
>           * This method is called when restoring from a failover.
>           * @param globalCommittables the global committables that are not
> committed in the previous session.
>           * @return the global committables that should be committed again
> in the current session.
>           */
>          List<GCommT> filterRecoveredCommittables(List<GCommT>
> globalCommittables);
> 
>          /**
>           * Compute an aggregated committable from a collection of
> committables.
>           * @param committables a collection of committables that are needed
> to combine
>           * @return an aggregated committable
>           */
>          GCommT combine(List<CommT> committables);
> 
>          void commit(List<GCommT> globalCommittables);
> 
>          /**
>           * There are no committables any more.
>           */
>          void endOfInput();
> }
> 
> Best,
> Guowei


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

Posted by Guowei Ma <gu...@gmail.com>.
Hi, all

Thank everyone very much for your ideas and suggestions. I would try to
summarize again the consensus :). Correct me if I am wrong or misunderstand
you.

## Consensus-1

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.

## Consensus-2

1. What should the “Unified Sink API” support/cover? It includes two
aspects. 1. The same sink implementation would work for both the batch and
stream execution mode. 2. In the long run we should give the sink developer
the ability of building “arbitrary” topologies. But for Flink-1.12 we
should be more focused on only satisfying the S3/HDFS/Iceberg sink.
2. Because the batch execution mode does not have the normal checkpoint the
sink developer should not depend on it any more if we want a unified sink.
3. We can benefit by providing an asynchronous `Writer` version. But
because the unified sink is already very complicated, we don’t add this in
the first version.


According to these consensus I would propose the first version of the new
sink api as follows. What do you think? Any comments are welcome.

/**
 * This interface lets the sink developer build a simple transactional sink
topology pattern, which satisfies the HDFS/S3/Iceberg sink.
 * This sink topology includes one {@link Writer} + one {@link Committer} +
one {@link GlobalCommitter}.
 * The {@link Writer} is responsible for producing the committable.
 * The {@link Committer} is responsible for committing a single
committables.
 * The {@link GlobalCommitter} is responsible for committing an aggregated
committable, which we called global committables.
 *
 * But both the {@link Committer} and the {@link GlobalCommitter} are
optional.
 */
interface TSink<IN, CommT, GCommT, WriterS> {

        Writer<IN, CommT, WriterS> createWriter(InitContext initContext);

        Writer<IN, CommT, WriterS> restoreWriter(InitContext initContext,
List<WriterS> states);

        Optional<Committer<CommT>> createCommitter();

        Optional<GlobalCommitter<CommT, GCommT>> createGlobalCommitter();

        SimpleVersionedSerializer<CommT> getCommittableSerializer();

        Optional<SimpleVersionedSerializer<GCommT>>
getGlobalCommittableSerializer();
}

/**
 * The {@link GlobalCommitter} is responsible for committing an aggregated
committable, which we called global committables.
 */
interface GlobalCommitter<CommT, GCommT> {

        /**
         * This method is called when restoring from a failover.
         * @param globalCommittables the global committables that are not
committed in the previous session.
         * @return the global committables that should be committed again
in the current session.
         */
        List<GCommT> filterRecoveredCommittables(List<GCommT>
globalCommittables);

        /**
         * Compute an aggregated committable from a collection of
committables.
         * @param committables a collection of committables that are needed
to combine
         * @return an aggregated committable
         */
        GCommT combine(List<CommT> committables);

        void commit(List<GCommT> globalCommittables);

        /**
         * There are no committables any more.
         */
        void endOfInput();
}

Best,
Guowei


On Wed, Sep 23, 2020 at 12:03 PM Guowei Ma <gu...@gmail.com> wrote:

> >> I think we should go with something like
>
> >> List<GlobalCommitT> filterRecoveredCommittables(List<>)
>
> >> to keep things simple. This should also be easy to do from the framework
> >> side and then the sink doesn't need to do any custom state handling.
>
> I second Aljoscha's  proposal. For the first version there is already much
> stuff to do.
> For now I think it would be satisfied with IceBerg Sink.
>
> Best,
> Guowei
>
>
> On Tue, Sep 22, 2020 at 10:54 PM Aljoscha Krettek <al...@apache.org>
> wrote:
>
>> I think we should go with something like
>>
>> List<GlobalCommitT> filterRecoveredCommittables(List<>)
>>
>> to keep things simple. This should also be easy to do from the framework
>> side and then the sink doesn't need to do any custom state handling.
>>
>> Best,
>> Aljoscha
>>
>> On 22.09.20 16:03, Steven Wu wrote:
>> > Previous APIs discussed have been trying to do more in the framework.
>> If we
>> > take a different approach to a lighter framework, these sets of
>> > minimal APIs are probably good enough. Sink can handle the bookkeeping,
>> > merge, retry logics.
>> >
>> > /**
>> >   * CommT is the DataFile in Iceberg
>> >   * GlobalCommT is the checkpoint data type, like ManifestFile in
>> Iceberg
>> > */
>> > interface GlobalCommitter<CommT, GlobalCommT> {
>> >
>> >    void collect(CommT);
>> >
>> >    void commit();
>> >
>> >    List<GlobalCommT> snapshotState();
>> >
>> >    // this is just a callback to sink so that it can perform filter and
>> > retain the uncommitted GlobalCommT in the internal bookkeeping
>> >    void recoveredCommittables(List<GlobalCommT>) ;
>> > }
>> >
>> > The most important need from the framework is to run GlobalCommitter in
>> the
>> > jobmanager. It involves the topology creation, checkpoint handling,
>> > serializing the executions of commit() calls etc.
>> >
>> > Thanks,
>> > Steven
>> >
>> > On Tue, Sep 22, 2020 at 6:39 AM Steven Wu <st...@gmail.com> wrote:
>> >
>> >> It is fine to leave the CommitResult/RETRY outside the scope of
>> framework.
>> >> Then the framework might need to provide some hooks in the
>> >> checkpoint/restore logic. because the commit happened in the post
>> >> checkpoint completion step, sink needs to update the internal state
>> when
>> >> the commit is successful so that the next checkpoint won't include the
>> >> committed GlobalCommT.
>> >>
>> >> Maybe GlobalCommitter can have an API like this?
>> >>> List<GlobalCommT> snapshotState();
>> >>
>> >> But then we still need the recover API if we don't let sink directly
>> >> manage the state.
>> >>> List<GlobalCommT> recoverCommittables(List<GlobalCommT>)
>> >>
>> >> Thanks,
>> >> Steven
>> >>
>> >> On Tue, Sep 22, 2020 at 6:33 AM Aljoscha Krettek <al...@apache.org>
>> >> wrote:
>> >>
>> >>> On 22.09.20 13:26, Guowei Ma wrote:
>> >>>> Actually I am not sure adding `isAvailable` is enough. Maybe it is
>> not.
>> >>>> But for the initial version I hope we could make the sink api sync
>> >>> because
>> >>>> there is already a lot of stuff that has to finish. :--)
>> >>>
>> >>> I agree, for the first version we should stick to a simpler
>> synchronous
>> >>> interface.
>> >>>
>> >>> Aljoscha
>> >>>
>> >>
>> >
>>
>>

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

Posted by Guowei Ma <gu...@gmail.com>.
>> I think we should go with something like

>> List<GlobalCommitT> filterRecoveredCommittables(List<>)

>> to keep things simple. This should also be easy to do from the framework
>> side and then the sink doesn't need to do any custom state handling.

I second Aljoscha's  proposal. For the first version there is already much
stuff to do.
For now I think it would be satisfied with IceBerg Sink.

Best,
Guowei


On Tue, Sep 22, 2020 at 10:54 PM Aljoscha Krettek <al...@apache.org>
wrote:

> I think we should go with something like
>
> List<GlobalCommitT> filterRecoveredCommittables(List<>)
>
> to keep things simple. This should also be easy to do from the framework
> side and then the sink doesn't need to do any custom state handling.
>
> Best,
> Aljoscha
>
> On 22.09.20 16:03, Steven Wu wrote:
> > Previous APIs discussed have been trying to do more in the framework. If
> we
> > take a different approach to a lighter framework, these sets of
> > minimal APIs are probably good enough. Sink can handle the bookkeeping,
> > merge, retry logics.
> >
> > /**
> >   * CommT is the DataFile in Iceberg
> >   * GlobalCommT is the checkpoint data type, like ManifestFile in Iceberg
> > */
> > interface GlobalCommitter<CommT, GlobalCommT> {
> >
> >    void collect(CommT);
> >
> >    void commit();
> >
> >    List<GlobalCommT> snapshotState();
> >
> >    // this is just a callback to sink so that it can perform filter and
> > retain the uncommitted GlobalCommT in the internal bookkeeping
> >    void recoveredCommittables(List<GlobalCommT>) ;
> > }
> >
> > The most important need from the framework is to run GlobalCommitter in
> the
> > jobmanager. It involves the topology creation, checkpoint handling,
> > serializing the executions of commit() calls etc.
> >
> > Thanks,
> > Steven
> >
> > On Tue, Sep 22, 2020 at 6:39 AM Steven Wu <st...@gmail.com> wrote:
> >
> >> It is fine to leave the CommitResult/RETRY outside the scope of
> framework.
> >> Then the framework might need to provide some hooks in the
> >> checkpoint/restore logic. because the commit happened in the post
> >> checkpoint completion step, sink needs to update the internal state when
> >> the commit is successful so that the next checkpoint won't include the
> >> committed GlobalCommT.
> >>
> >> Maybe GlobalCommitter can have an API like this?
> >>> List<GlobalCommT> snapshotState();
> >>
> >> But then we still need the recover API if we don't let sink directly
> >> manage the state.
> >>> List<GlobalCommT> recoverCommittables(List<GlobalCommT>)
> >>
> >> Thanks,
> >> Steven
> >>
> >> On Tue, Sep 22, 2020 at 6:33 AM Aljoscha Krettek <al...@apache.org>
> >> wrote:
> >>
> >>> On 22.09.20 13:26, Guowei Ma wrote:
> >>>> Actually I am not sure adding `isAvailable` is enough. Maybe it is
> not.
> >>>> But for the initial version I hope we could make the sink api sync
> >>> because
> >>>> there is already a lot of stuff that has to finish. :--)
> >>>
> >>> I agree, for the first version we should stick to a simpler synchronous
> >>> interface.
> >>>
> >>> Aljoscha
> >>>
> >>
> >
>
>

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

Posted by Aljoscha Krettek <al...@apache.org>.
I think we should go with something like

List<GlobalCommitT> filterRecoveredCommittables(List<>)

to keep things simple. This should also be easy to do from the framework 
side and then the sink doesn't need to do any custom state handling.

Best,
Aljoscha

On 22.09.20 16:03, Steven Wu wrote:
> Previous APIs discussed have been trying to do more in the framework. If we
> take a different approach to a lighter framework, these sets of
> minimal APIs are probably good enough. Sink can handle the bookkeeping,
> merge, retry logics.
> 
> /**
>   * CommT is the DataFile in Iceberg
>   * GlobalCommT is the checkpoint data type, like ManifestFile in Iceberg
> */
> interface GlobalCommitter<CommT, GlobalCommT> {
> 
>    void collect(CommT);
> 
>    void commit();
> 
>    List<GlobalCommT> snapshotState();
> 
>    // this is just a callback to sink so that it can perform filter and
> retain the uncommitted GlobalCommT in the internal bookkeeping
>    void recoveredCommittables(List<GlobalCommT>) ;
> }
> 
> The most important need from the framework is to run GlobalCommitter in the
> jobmanager. It involves the topology creation, checkpoint handling,
> serializing the executions of commit() calls etc.
> 
> Thanks,
> Steven
> 
> On Tue, Sep 22, 2020 at 6:39 AM Steven Wu <st...@gmail.com> wrote:
> 
>> It is fine to leave the CommitResult/RETRY outside the scope of framework.
>> Then the framework might need to provide some hooks in the
>> checkpoint/restore logic. because the commit happened in the post
>> checkpoint completion step, sink needs to update the internal state when
>> the commit is successful so that the next checkpoint won't include the
>> committed GlobalCommT.
>>
>> Maybe GlobalCommitter can have an API like this?
>>> List<GlobalCommT> snapshotState();
>>
>> But then we still need the recover API if we don't let sink directly
>> manage the state.
>>> List<GlobalCommT> recoverCommittables(List<GlobalCommT>)
>>
>> Thanks,
>> Steven
>>
>> On Tue, Sep 22, 2020 at 6:33 AM Aljoscha Krettek <al...@apache.org>
>> wrote:
>>
>>> On 22.09.20 13:26, Guowei Ma wrote:
>>>> Actually I am not sure adding `isAvailable` is enough. Maybe it is not.
>>>> But for the initial version I hope we could make the sink api sync
>>> because
>>>> there is already a lot of stuff that has to finish. :--)
>>>
>>> I agree, for the first version we should stick to a simpler synchronous
>>> interface.
>>>
>>> Aljoscha
>>>
>>
> 


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

Posted by Steven Wu <st...@gmail.com>.
Previous APIs discussed have been trying to do more in the framework. If we
take a different approach to a lighter framework, these sets of
minimal APIs are probably good enough. Sink can handle the bookkeeping,
merge, retry logics.

/**
 * CommT is the DataFile in Iceberg
 * GlobalCommT is the checkpoint data type, like ManifestFile in Iceberg
*/
interface GlobalCommitter<CommT, GlobalCommT> {

  void collect(CommT);

  void commit();

  List<GlobalCommT> snapshotState();

  // this is just a callback to sink so that it can perform filter and
retain the uncommitted GlobalCommT in the internal bookkeeping
  void recoveredCommittables(List<GlobalCommT>) ;
}

The most important need from the framework is to run GlobalCommitter in the
jobmanager. It involves the topology creation, checkpoint handling,
serializing the executions of commit() calls etc.

Thanks,
Steven

On Tue, Sep 22, 2020 at 6:39 AM Steven Wu <st...@gmail.com> wrote:

> It is fine to leave the CommitResult/RETRY outside the scope of framework.
> Then the framework might need to provide some hooks in the
> checkpoint/restore logic. because the commit happened in the post
> checkpoint completion step, sink needs to update the internal state when
> the commit is successful so that the next checkpoint won't include the
> committed GlobalCommT.
>
> Maybe GlobalCommitter can have an API like this?
> > List<GlobalCommT> snapshotState();
>
> But then we still need the recover API if we don't let sink directly
> manage the state.
> > List<GlobalCommT> recoverCommittables(List<GlobalCommT>)
>
> Thanks,
> Steven
>
> On Tue, Sep 22, 2020 at 6:33 AM Aljoscha Krettek <al...@apache.org>
> wrote:
>
>> On 22.09.20 13:26, Guowei Ma wrote:
>> > Actually I am not sure adding `isAvailable` is enough. Maybe it is not.
>> > But for the initial version I hope we could make the sink api sync
>> because
>> > there is already a lot of stuff that has to finish. :--)
>>
>> I agree, for the first version we should stick to a simpler synchronous
>> interface.
>>
>> Aljoscha
>>
>

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

Posted by Steven Wu <st...@gmail.com>.
It is fine to leave the CommitResult/RETRY outside the scope of framework.
Then the framework might need to provide some hooks in the
checkpoint/restore logic. because the commit happened in the post
checkpoint completion step, sink needs to update the internal state when
the commit is successful so that the next checkpoint won't include the
committed GlobalCommT.

Maybe GlobalCommitter can have an API like this?
> List<GlobalCommT> snapshotState();

But then we still need the recover API if we don't let sink directly manage
the state.
> List<GlobalCommT> recoverCommittables(List<GlobalCommT>)

Thanks,
Steven

On Tue, Sep 22, 2020 at 6:33 AM Aljoscha Krettek <al...@apache.org>
wrote:

> On 22.09.20 13:26, Guowei Ma wrote:
> > Actually I am not sure adding `isAvailable` is enough. Maybe it is not.
> > But for the initial version I hope we could make the sink api sync
> because
> > there is already a lot of stuff that has to finish. :--)
>
> I agree, for the first version we should stick to a simpler synchronous
> interface.
>
> Aljoscha
>

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

Posted by Aljoscha Krettek <al...@apache.org>.
On 22.09.20 13:26, Guowei Ma wrote:
> Actually I am not sure adding `isAvailable` is enough. Maybe it is not.
> But for the initial version I hope we could make the sink api sync because
> there is already a lot of stuff that has to finish. :--)

I agree, for the first version we should stick to a simpler synchronous 
interface.

Aljoscha

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

Posted by Guowei Ma <gu...@gmail.com>.
>> 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?

Hi @piotr <pi...@ververica.com>

Actually I am not sure adding `isAvailable` is enough. Maybe it is not.
But for the initial version I hope we could make the sink api sync because
there is already a lot of stuff that has to finish. :--)

What do you think?

Best,
Guowei


On Tue, Sep 22, 2020 at 5:25 PM Aljoscha Krettek <al...@apache.org>
wrote:

> On 22.09.20 11:10, Guowei Ma wrote:
> > 1. I think maybe we could add a EOI interface to the `GlobalCommit`. So
> > that we could make `write success file` be available in both batch and
> > stream execution mode.
>
> We could, yes. I'm now hesitant because we're adding more things but I
> think it should be fine.
>
> > 2.  If we choose to let the two types of committer appear at the same
> time
> > in the API we have to figure out how to express the relation between the
> > two committers. I think the Sink API may look like the following: What do
> > you think?
> > Sink<T, CommT, CommR, ShareStateT...> {
> >          Writer<T, CommT, ShareStateT.....> createWriter();
> >          Optional<Committer<CommT>> createCommitter();
> >          Optional<GlobalComiitter<CommT, GlobalCommT>>
> > createGlobalCommitter();
> > }
>
> Yes, I think this is what we should do. Though I think that we should
> initially not support shared state. The FileSink only uses this to
> create unique file names and I think we can do without it. If we see
> that we need it later we can add it but I would like to keep things
> minimal initially. It's always easy to add things later but it's hard to
> take things away once you added them.
>
> > 3. Maybe a silly question: Why do we need `commit` return
> `CommitResult`? I
> > think the sink developer could rety himself. Why need the framework to do
> > the retry?
>
> It's not a silly question at all! I think we need the retry to support
> such problems as Steven mentioned. If a commit fails a RETRY tells the
> framework that it should keep the commits in state and retry them on the
> next checkpoint. When the committer returns FAILURE we should just fail
> the job. It's to support temporary outages of the external metastore.
>
> I'm open to leaving it out of the initial version for the same reasons I
> mentioned above but I think it could be valuable.
>
> Best,
> Aljoscha
>
>

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

Posted by Aljoscha Krettek <al...@apache.org>.
On 22.09.20 11:10, Guowei Ma wrote:
> 1. I think maybe we could add a EOI interface to the `GlobalCommit`. So
> that we could make `write success file` be available in both batch and
> stream execution mode.

We could, yes. I'm now hesitant because we're adding more things but I 
think it should be fine.

> 2.  If we choose to let the two types of committer appear at the same time
> in the API we have to figure out how to express the relation between the
> two committers. I think the Sink API may look like the following: What do
> you think?
> Sink<T, CommT, CommR, ShareStateT...> {
>          Writer<T, CommT, ShareStateT.....> createWriter();
>          Optional<Committer<CommT>> createCommitter();
>          Optional<GlobalComiitter<CommT, GlobalCommT>>
> createGlobalCommitter();
> }

Yes, I think this is what we should do. Though I think that we should 
initially not support shared state. The FileSink only uses this to 
create unique file names and I think we can do without it. If we see 
that we need it later we can add it but I would like to keep things 
minimal initially. It's always easy to add things later but it's hard to 
take things away once you added them.

> 3. Maybe a silly question: Why do we need `commit` return `CommitResult`? I
> think the sink developer could rety himself. Why need the framework to do
> the retry?

It's not a silly question at all! I think we need the retry to support 
such problems as Steven mentioned. If a commit fails a RETRY tells the 
framework that it should keep the commits in state and retry them on the 
next checkpoint. When the committer returns FAILURE we should just fail 
the job. It's to support temporary outages of the external metastore.

I'm open to leaving it out of the initial version for the same reasons I 
mentioned above but I think it could be valuable.

Best,
Aljoscha


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

Posted by Guowei Ma <gu...@gmail.com>.
Thanks @aljoscha summary. I agree we should postpone the discussion of the
sink topology first and focus on the normal file sink and IcebergSink in
the Flink 1.12.

I have three little questions:

1. I think maybe we could add a EOI interface to the `GlobalCommit`. So
that we could make `write success file` be available in both batch and
stream execution mode.
2.  If we choose to let the two types of committer appear at the same time
in the API we have to figure out how to express the relation between the
two committers. I think the Sink API may look like the following: What do
you think?
Sink<T, CommT, CommR, ShareStateT...> {
        Writer<T, CommT, ShareStateT.....> createWriter();
        Optional<Committer<CommT>> createCommitter();
        Optional<GlobalComiitter<CommT, GlobalCommT>>
createGlobalCommitter();
}
3. Maybe a silly question: Why do we need `commit` return `CommitResult`? I
think the sink developer could rety himself. Why need the framework to do
the retry?

Best,
Guowei


On Tue, Sep 22, 2020 at 4:47 PM Aljoscha Krettek <al...@apache.org>
wrote:

> Ah sorry, I think I now see what you mean. I think it's ok to add a
> `List<GlobalCommittableT> recoverCommittables(List<GlobalCommittableT>)`
> method.
>
>
> On 22.09.20 09:42, Aljoscha Krettek wrote:
> > On 22.09.20 06:06, Steven Wu wrote:
> >> 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.
> >
> > I think we don't need the `recoverGlobalCommittables()` hook. The sink
> > implementation would have to do the filtering once, so it can either do
> > it in the recover hook or it could do it in the next `commit()` call.
> > Both of these would mean we only have to do one pass through the list
> > and connect to Iceberg. Doing the check in `commit()` would mean the
> > interface of GlobalCommittable is simpler and to me it seems natural
> > that we do the check in the commit() method to ensure that commits are
> > idempotent.
> >
> > What do you think?
> >
> > Aljoscha
>
>

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

Posted by Aljoscha Krettek <al...@apache.org>.
Ah sorry, I think I now see what you mean. I think it's ok to add a 
`List<GlobalCommittableT> recoverCommittables(List<GlobalCommittableT>)` 
method.


On 22.09.20 09:42, Aljoscha Krettek wrote:
> On 22.09.20 06:06, Steven Wu wrote:
>> 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.
> 
> I think we don't need the `recoverGlobalCommittables()` hook. The sink 
> implementation would have to do the filtering once, so it can either do 
> it in the recover hook or it could do it in the next `commit()` call. 
> Both of these would mean we only have to do one pass through the list 
> and connect to Iceberg. Doing the check in `commit()` would mean the 
> interface of GlobalCommittable is simpler and to me it seems natural 
> that we do the check in the commit() method to ensure that commits are 
> idempotent.
> 
> What do you think?
> 
> Aljoscha


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

Posted by Aljoscha Krettek <al...@apache.org>.
On 22.09.20 06:06, Steven Wu wrote:
> 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.

I think we don't need the `recoverGlobalCommittables()` hook. The sink 
implementation would have to do the filtering once, so it can either do 
it in the recover hook or it could do it in the next `commit()` call. 
Both of these would mean we only have to do one pass through the list 
and connect to Iceberg. Doing the check in `commit()` would mean the 
interface of GlobalCommittable is simpler and to me it seems natural 
that we do the check in the commit() method to ensure that commits are 
idempotent.

What do you think?

Aljoscha

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

Posted by Steven Wu <st...@gmail.com>.
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 <al...@apache.org>
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 <st...@gmail.com> 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 <gu...@gmail.com>  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 <gu...@gmail.com> 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 <gu...@gmail.com>
> 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 <st...@gmail.com>
> >> 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 <gu...@gmail.com> 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 <gu...@gmail.com>
> >> 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 <st...@gmail.com>
> >>>>>> 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 <
> >> aljoscha@apache.org
> >>>>>>>
> >>>>>>> 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
> >>>>>>>>
> >>>>>>>
> >>>>>>
> >>>>>
> >>
> >
>
>

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

Posted by Aljoscha Krettek <al...@apache.org>.
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 <st...@gmail.com> 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 <gu...@gmail.com>  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 <gu...@gmail.com> 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 <gu...@gmail.com> 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 <st...@gmail.com>
>> 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 <gu...@gmail.com> 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 <gu...@gmail.com>
>> 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 <st...@gmail.com>
>>>>>> 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 <
>> aljoscha@apache.org
>>>>>>>
>>>>>>> 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
>>>>>>>>
>>>>>>>
>>>>>>
>>>>>
>>
> 


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

Posted by Piotr Nowojski <pn...@apache.org>.
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 <st...@gmail.com> 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 <gu...@gmail.com>  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 <gu...@gmail.com> 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 <gu...@gmail.com> 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 <st...@gmail.com>
> 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 <gu...@gmail.com> 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 <gu...@gmail.com>
> 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 <st...@gmail.com>
> >>>> 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 <
> aljoscha@apache.org
> >>>> >
> >>>> > 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
> >>>> > >
> >>>> >
> >>>>
> >>>
>

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

Posted by Steven Wu <st...@gmail.com>.
> I think Iceberg sink needs to do the dedup in the `commit` call. The
`recoveredGlobalCommittables` is just for restoring the ids.


@Guowei Ma <gu...@gmail.com>  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 <gu...@gmail.com> 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 <gu...@gmail.com> 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 <st...@gmail.com> 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 <gu...@gmail.com> 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 <gu...@gmail.com> 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 <st...@gmail.com>
>>>> 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 <aljoscha@apache.org
>>>> >
>>>> > 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
>>>> > >
>>>> >
>>>>
>>>

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

Posted by Guowei Ma <gu...@gmail.com>.
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 <gu...@gmail.com> 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 <st...@gmail.com> 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 <gu...@gmail.com> 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 <gu...@gmail.com> 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 <st...@gmail.com> 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 <al...@apache.org>
>>> > 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
>>> > >
>>> >
>>>
>>

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

Posted by Guowei Ma <gu...@gmail.com>.
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 <st...@gmail.com> 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 <gu...@gmail.com> 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 <gu...@gmail.com> 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 <st...@gmail.com> 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 <al...@apache.org>
>> > 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
>> > >
>> >
>>
>

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

Posted by Steven Wu <st...@gmail.com>.
> 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 <gu...@gmail.com> 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 <gu...@gmail.com> 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 <st...@gmail.com> 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 <al...@apache.org>
> > 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
> > >
> >
>

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

Posted by Guowei Ma <gu...@gmail.com>.
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 <st...@gmail.com> 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 <al...@apache.org>
> 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
> >
>

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

Posted by Steven Wu <st...@gmail.com>.
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 <al...@apache.org>
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
>

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

Posted by Aljoscha Krettek <al...@apache.org>.
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

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

Posted by Steven Wu <st...@gmail.com>.
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 <al...@apache.org>
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.
>
>
>
>
>

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

Posted by Aljoscha Krettek <al...@apache.org>.
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.





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

Posted by Guowei Ma <gu...@gmail.com>.
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 <gu...@gmail.com> 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 <st...@gmail.com> wrote:
>
>> > AFAIK the committer would not see the file-1-2 when ck1 happens in the
>> ExactlyOnce mode.
>>
>> @Guowei Ma <gu...@gmail.com> 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 <st...@gmail.com> 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 <gu...@gmail.com> 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 <zh...@alibaba-inc.com> 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 <dw...@apache.org>  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 <pi...@ververica.com>  Please correct me if  I
>>>> misunderstand you. thanks.
>>>>
>>>> Best,
>>>> Guowei
>>>>
>>>>
>>>> On Tue, Sep 15, 2020 at 3:55 PM Dawid Wysakowicz <
>>>> dwysakowicz@apache.org>
>>>> 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 <
>>>> dwysakowicz@apache.org> <dw...@apache.org>
>>>> > 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
>>>> >
>>>> >
>>>> >
>>>>
>>>

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

Posted by Guowei Ma <gu...@gmail.com>.
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 <st...@gmail.com> wrote:

> > AFAIK the committer would not see the file-1-2 when ck1 happens in the
> ExactlyOnce mode.
>
> @Guowei Ma <gu...@gmail.com> 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 <st...@gmail.com> 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 <gu...@gmail.com> 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 <zh...@alibaba-inc.com> 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 <dw...@apache.org>  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 <pi...@ververica.com>  Please correct me if  I
>>> misunderstand you. thanks.
>>>
>>> Best,
>>> Guowei
>>>
>>>
>>> On Tue, Sep 15, 2020 at 3:55 PM Dawid Wysakowicz <dwysakowicz@apache.org
>>> >
>>> 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 <
>>> dwysakowicz@apache.org> <dw...@apache.org>
>>> > 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
>>> >
>>> >
>>> >
>>>
>>

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

Posted by Steven Wu <st...@gmail.com>.
> AFAIK the committer would not see the file-1-2 when ck1 happens in the
ExactlyOnce mode.

@Guowei Ma <gu...@gmail.com> 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 <st...@gmail.com> 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 <gu...@gmail.com> 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 <zh...@alibaba-inc.com> 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 <dw...@apache.org>  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 <pi...@ververica.com>  Please correct me if  I
>> misunderstand you. thanks.
>>
>> Best,
>> Guowei
>>
>>
>> On Tue, Sep 15, 2020 at 3:55 PM Dawid Wysakowicz <dw...@apache.org>
>> 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 <
>> dwysakowicz@apache.org> <dw...@apache.org>
>> > 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
>> >
>> >
>> >
>>
>

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

Posted by Steven Wu <st...@gmail.com>.
> 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 <gu...@gmail.com> 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 <zh...@alibaba-inc.com> 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 <dw...@apache.org>  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 <pi...@ververica.com>  Please correct me if  I
> misunderstand you. thanks.
>
> Best,
> Guowei
>
>
> On Tue, Sep 15, 2020 at 3:55 PM Dawid Wysakowicz <dw...@apache.org>
> 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 <
> dwysakowicz@apache.org> <dw...@apache.org>
> > 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
> >
> >
> >
>

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

Posted by Guowei Ma <gu...@gmail.com>.
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 <zh...@alibaba-inc.com> 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 <dw...@apache.org>  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 <pi...@ververica.com>  Please correct me if  I
misunderstand you. thanks.

Best,
Guowei


On Tue, Sep 15, 2020 at 3:55 PM Dawid Wysakowicz <dw...@apache.org>
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 <dw...@apache.org> <dw...@apache.org>
> 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
>
>
>

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

Posted by Dawid Wysakowicz <dw...@apache.org>.
> 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 <dw...@apache.org>
> 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
>>>
>>

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

Posted by Guowei Ma <gu...@gmail.com>.
>> 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 <dw...@apache.org>
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
> >
>
>

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

Posted by Dawid Wysakowicz <dw...@apache.org>.
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
>


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

Posted by Aljoscha Krettek <al...@apache.org>.
I thought about this some more. One of the important parts of the 
Iceberg sink is to know whether we have already committed some 
DataFiles. Currently, this is implemented by writing a (JobId, 
MaxCheckpointId) tuple to the Iceberg table when committing. When 
restoring from a failure we check this and discard committables 
(DataFile) that we know to already be committed.

I think this can have some problems, for example when checkpoint ids are 
not strictly sequential, when we wrap around, or when the JobID changes. 
This will happen when doing a stop/start-from-savepoint cycle, for example.

I think we could fix this by having Flink provide a nonce to the 
GlobalCommitter where Flink guarantees that this nonce is unique and 
will not change for repeated invocations of the GlobalCommitter with the 
same set of committables. The GlobalCommitter could use this to 
determine whether a set of committables has already been committed to 
the Iceberg table.

It's seems very tailor-made for Iceberg for now but other systems should 
suffer from the same problem.

Best,
Aljoscha

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

Posted by Aljoscha Krettek <al...@apache.org>.
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


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

Posted by Steven Wu <st...@gmail.com>.
Aljoscha, thanks a lot for the detailed response. Now I have a better
understanding of the initial scope.

To me, there are possibly three different committer behaviors. For the lack
of better names, let's call them
* No/NoopCommitter
* Committer / LocalCommitter (file sink?)
* GlobalCommitter (Iceberg)

## 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.
-----------------
// before barrier emitted to downstream
void prepareSnapshot(long checkpointId) throws Exception;

// or will flush be called automatically before the barrier emitted
downstream?
// if yes, we need the checkpointId arg for the reason listed in [1]
void flush(WriterOutput<CommT> output) throws IOException;

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.
For that reason, writers need to know the checkpointId where the restore
happened. Can we add a RestoreContext interface to the restoreWriter method?
---------------
Writer<IN, CommT, WriterS, SharedS> restoreWriter(InitContext context,
RestoreContext restoreContext, List<WriterS> state, List<SharedS> share);

interface RestoreContext {
  long getCheckpointId();
}


## Committer interface

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.
------------------
void commit(CommT committable) throws Exception;
      ==>
void collect(CommT committable) throws Exception;
void commit() throws Exception;

As discussed in [1] and mentioned above, the Iceberg committer needs to
know which checkpointId is the commit for. So can we add checkpiontId arg
to the commit API. However, I don't know how this would affect the batch
execution where checkpoints are disabled.
------------------
void commit(long checkpointId) throws Exception;

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


Thanks,
Steven

[1] https://github.com/apache/iceberg/pull/1185#discussion_r479589663
[2] https://github.com/apache/iceberg/pull/1185#discussion_r479457104



On Fri, Sep 11, 2020 at 3:27 AM Aljoscha Krettek <al...@apache.org>
wrote:

> Regarding the FLIP itself, I like the motivation section and the
> What/How/When/Where section a lot!
>
> I don't understand why we need the "Drain and Snapshot" section. It
> seems to be some details about stop-with-savepoint and drain, and the
> relation to BATCH execution but I don't know if it is needed to
> understand the rest of the document. I'm happy to be wrong here, though,
> if there's good reasons for the section.
>
> On the question of Alternative 1 and 2, I have a strong preference for
> Alternative 1 because we could avoid strong coupling to other modules.
> With Alternative 2 we would depend on `flink-streaming-java` and even
> `flink-runtime`. For the new source API (FLIP-27) we managed to keep the
> dependencies slim and the code is in flink-core. I'd be very happy if we
> can manage the same for the new sink API.
>
> Best,
> Aljoscha
>
> On 11.09.20 12:02, Aljoscha Krettek wrote:
> > Hi Everyone,
> >
> > thanks to Guowei for publishing the FLIP, and thanks Steven for the very
> > thoughtful email!
> >
> > We thought a lot internally about some of the questions you posted but
> > left a lot (almost all) of the implementation details out of the FLIP
> > for now because we wanted to focus on semantics and API. I'll try and
> > address the points below.
> >
> > ## Initial Scope of the new Sink API
> >
> > We need to accept some initial scope that we want to achieve for Flink
> > 1.12. I don't think we can try and find the solution that will work for
> > all current and future external systems. For me, the initial goal would
> > be to produce a Sink API and implementations for systems where you can
> > prepare "committables" in one process and commit those from another
> > process. Those are systems that support "real" transactions as you need
> > them in a two-phase commit protocol. This includes:
> >
> >   - File Sink, including HDFS, S3 via special part-file uploads
> >   - Iceberg
> >   - HDFS
> >
> > The work should include runtime support for both BATCH and STREAMING
> > execution as outlined in https://s.apache.org/FLIP-134.
> >
> > Supporting Kafka already becomes difficult but I'll go into that below.
> >
> > ## Where to run the Committer
> >
> > Guowei hinted at this in the FLIP: the idea is that the Framework
> > decides where to run the committer based on requirements and based on
> > the execution mode (STREAMING or BATCH).
> >
> > Something that is not in the FLIP but which we thought about is that we
> > need to allow different types of committers. I'm currently thinking we
> > need at least a normal "Committer" and a "GlobalCommiter" (name TBD).
> >
> > 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.
> >
> > An interesting read on this topic is the discussion on
> > https://issues.apache.org/jira/browse/MAPREDUCE-4815. About the Hadoop
> > FileOutputCommitter and the two different available algorithms for
> > committing final job/task results.
> >
> > These interfaces untie the sink implementation from the Runtime and we
> > could, for example, have a runtime like this:
> >
> > ### BATCH
> >
> >   - Collect all committables and store them in a fault tolerant way
> > until the job finishes
> >   - For a normal Commit function, call it on the individual commits. We
> > can potentially distribute this if it becomes a bottleneck
> >   - For GlobalCommit function, call it will all the commits. This cannot
> > be distributed
> >
> > We can collect the committables in an OperatorCoordinator or potentially
> > somehow in a task. Though I prefer an OperatorCoordinator right now. The
> > operator coordinator needs to keep the commits in a fault-tolerant way.
> >
> > ### STREAMING
> >
> >   - For normal Commit, keep the committables in state on the individual
> > tasks, commit them when a checkpoint completes
> >   - For global CommitFunction we have options: collect them in a DOP-1
> > operator in the topology or send them to an OperatorCoordinator to do
> > the commit there. This is where the source/sink duality that Steven
> > mentions becomes visible.
> >
> > ## Kafka
> >
> > Kafka is a problematic case because it doesn't really support
> > transactions as outlined above. Our current Sink implementations works
> > around that with hacks but that only gets us so far.
> >
> > The problem with Kafka is that we need to aggressively clean up pending
> > transactions in case a failure happens. Otherwise stale transactions
> > would block downstream consumers. See here for details:
> > http://kafka.apache.org/documentation/#isolation.level.
> >
> > The way we solve this in the current Kafka sink is by using a fixed pool
> > of transactional IDs and then cancelling all outstanding transactions
> > for the IDs when we restore from a savepoint. In order for this to work
> > we need to recycle the IDs, so there needs to be a back-channel from the
> > Committer to the Writter, or they need to share internal state.
> >
> > I don't get see a satisfying solution for this so I think we should
> > exclude this from the initial version.
> >
> > ## On Write-Ahead-Log Sinks
> >
> > Some sinks, like ES or Cassandra would require that we keep a WAL in
> > Flink and then ship the contents to the external system on checkpoint.
> > The reason is that these systems don't support real transactions where
> > you can prepare them in one process and commit them from another process.
> >
> > Best,
> > Aljoscha
> >
> >
> > On 11.09.20 02:23, Steven Wu wrote:
> >> Guowei,
> >>
> >> Thanks a lot for the proposal and starting the discussion thread. Very
> >> excited.
> >>
> >> For the big question of "Is the sink an operator or a topology?", I
> >> have a
> >> few related sub questions.
> >> * Where should we run the committers?
> >> * Is the committer parallel or single parallelism?
> >> * Can a single choice satisfy all sinks?
> >>
> >> Trying to envision how some sinks can be implemented with this new
> >> unified
> >> sink interface.
> >>
> >> 1. Kafka sink
> >>
> >> Kafka supports non-transactional and transactional writes
> >> * Non-transaction writes don't need commit action. we can have *parallel
> >> writers and no/no-op committers*. This is probably true for other
> >> non-transactional message queues.
> >> * Transaction writes can be implemented as *parallel writers and
> parallel
> >> committers*. In this case, I don't know if it makes sense to separate
> >> writers and committers into two separate operators, because they
> probably
> >> need to share the same KafkaProducer object.
> >>
> >> Either way, both writers and committers probably should *run inside task
> >> managers*.
> >>
> >> 2. ES sink
> >>
> >> ES sink typically buffers the data up to a certain size or time
> threshold
> >> and then uploads/commits a batch to ES. Writers buffer data and flush
> >> when
> >> needed, and committer does the HTTP bulk upload to commit. To avoid
> >> serialization/deserialization cost, we should run *parallel writers and
> >> parallel committers* and they *should be* *chained or bundled together*
> >> while *running inside task managers*.
> >>
> >> It can also be implemented as *parallel writers and no/no-op
> committers*,
> >> where all logics (batching and upload) are put inside the writers.
> >>
> >> 3. Iceberg [1] sink
> >>
> >> It is currently implemented as two-stage operators with *parallel
> writers
> >> and single-parallelism committers*.
> >> * *parallel writers* that write records into data files. Upon
> checkpoint,
> >> writers flush and upload the files, and send the metadata/location of
> the
> >> data files to the downstream committer. Writers need to do the flush
> >> inside
> >> the "prepareSnapshotPreBarrier" method (NOT "snapshotState" method)
> >> before
> >> forwarding the checkpoint barrier to the committer
> >> * single-parallelism committer operator. It collects data files from
> >> upstream writers. During "snapshotState", it saves collected data
> >> files (or
> >> an uber metadata file) into state. When the checkpoint is completed,
> >> inside
> >> "notifyCheckpointComplete" it commits those data files to Iceberg
> >> tables. *The
> >> committer has to be single parallelism*, because we don't want
> >> hundreds or
> >> thousands of parallel committers to compete for commit operations with
> >> opportunistic concurrency control. It will be very inefficient and
> >> probably
> >> infeasible if the parallelism is high. Too many tiny
> commits/transactions
> >> can also slow down both the table write and read paths due to too many
> >> manifest files.
> >>
> >> Right now, both Iceberg writer and committer operators run inside task
> >> managers. It has one major drawback. With Iceberg sink, embarrassingly
> >> parallel jobs won't be embarrassingly parallel anymore. That breaks the
> >> benefit of region recovery for embarrassingly parallel DAG.
> Conceptually,
> >> the Writer-Committer sink pattern is like the mirroring of the FLIP-27
> >> Enumerator-Reader source pattern. It will be better *if the committer
> can
> >> run inside the job manager* like the SplitEnumerator for the FLIP-27
> >> source.
> >>
> >> -----------------------
> >> Additional questions regarding the doc/API
> >> * Any example for the writer shared state (Writer#snapshotSharedState)?
> >> * We allow the case where the writer has no state, right? Meaning
> WriterS
> >> can be Void.
> >>
> >> [1] https://iceberg.apache.org/
> >>
> >> Thanks,
> >> Steven
> >>
> >> On Thu, Sep 10, 2020 at 6:44 AM Guowei Ma <gu...@gmail.com> wrote:
> >>
> >>> Hi, devs & users
> >>>
> >>> As discussed in FLIP-131[1], Flink will deprecate the DataSet API in
> >>> favor
> >>> of DataStream API and Table API. Users should be able to use
> >>> DataStream API
> >>> to write jobs that support both bounded and unbounded execution modes.
> >>> However, Flink does not provide a sink API to guarantee the
> Exactly-once
> >>> semantics in both bounded and unbounded scenarios, which blocks the
> >>> unification.
> >>>
> >>> So we want to introduce a new unified sink API which could let the user
> >>> develop the sink once and run it everywhere. You could find more
> >>> details in
> >>> FLIP-143[2].
> >>>
> >>> The FLIP contains some open questions that I'd really appreciate inputs
> >>> from the community. Some of the open questions include:
> >>>
> >>>     1. We provide two alternative Sink API in the FLIP. The only
> >>>     difference between the two versions is how to expose the state to
> >>> the user.
> >>>     We want to know which one is your preference?
> >>>     2. How does the sink API support to write to the Hive?
> >>>     3. Is the sink an operator or a topology?
> >>>
> >>> [1]
> >>>
> https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=158866741
> >>>
> >>> [2]
> >>>
> https://cwiki.apache.org/confluence/display/FLINK/FLIP-143%3A+Unified+Sink+API
> >>>
> >>>
> >>> Best,
> >>> Guowei
> >>>
> >>
> >
>
>

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

Posted by Aljoscha Krettek <al...@apache.org>.
Regarding the FLIP itself, I like the motivation section and the 
What/How/When/Where section a lot!

I don't understand why we need the "Drain and Snapshot" section. It 
seems to be some details about stop-with-savepoint and drain, and the 
relation to BATCH execution but I don't know if it is needed to 
understand the rest of the document. I'm happy to be wrong here, though, 
if there's good reasons for the section.

On the question of Alternative 1 and 2, I have a strong preference for 
Alternative 1 because we could avoid strong coupling to other modules. 
With Alternative 2 we would depend on `flink-streaming-java` and even 
`flink-runtime`. For the new source API (FLIP-27) we managed to keep the 
dependencies slim and the code is in flink-core. I'd be very happy if we 
can manage the same for the new sink API.

Best,
Aljoscha

On 11.09.20 12:02, Aljoscha Krettek wrote:
> Hi Everyone,
> 
> thanks to Guowei for publishing the FLIP, and thanks Steven for the very 
> thoughtful email!
> 
> We thought a lot internally about some of the questions you posted but 
> left a lot (almost all) of the implementation details out of the FLIP 
> for now because we wanted to focus on semantics and API. I'll try and 
> address the points below.
> 
> ## Initial Scope of the new Sink API
> 
> We need to accept some initial scope that we want to achieve for Flink 
> 1.12. I don't think we can try and find the solution that will work for 
> all current and future external systems. For me, the initial goal would 
> be to produce a Sink API and implementations for systems where you can 
> prepare "committables" in one process and commit those from another 
> process. Those are systems that support "real" transactions as you need 
> them in a two-phase commit protocol. This includes:
> 
>   - File Sink, including HDFS, S3 via special part-file uploads
>   - Iceberg
>   - HDFS
> 
> The work should include runtime support for both BATCH and STREAMING 
> execution as outlined in https://s.apache.org/FLIP-134.
> 
> Supporting Kafka already becomes difficult but I'll go into that below.
> 
> ## Where to run the Committer
> 
> Guowei hinted at this in the FLIP: the idea is that the Framework 
> decides where to run the committer based on requirements and based on 
> the execution mode (STREAMING or BATCH).
> 
> Something that is not in the FLIP but which we thought about is that we 
> need to allow different types of committers. I'm currently thinking we 
> need at least a normal "Committer" and a "GlobalCommiter" (name TBD).
> 
> 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.
> 
> An interesting read on this topic is the discussion on 
> https://issues.apache.org/jira/browse/MAPREDUCE-4815. About the Hadoop 
> FileOutputCommitter and the two different available algorithms for 
> committing final job/task results.
> 
> These interfaces untie the sink implementation from the Runtime and we 
> could, for example, have a runtime like this:
> 
> ### BATCH
> 
>   - Collect all committables and store them in a fault tolerant way 
> until the job finishes
>   - For a normal Commit function, call it on the individual commits. We 
> can potentially distribute this if it becomes a bottleneck
>   - For GlobalCommit function, call it will all the commits. This cannot 
> be distributed
> 
> We can collect the committables in an OperatorCoordinator or potentially 
> somehow in a task. Though I prefer an OperatorCoordinator right now. The 
> operator coordinator needs to keep the commits in a fault-tolerant way.
> 
> ### STREAMING
> 
>   - For normal Commit, keep the committables in state on the individual 
> tasks, commit them when a checkpoint completes
>   - For global CommitFunction we have options: collect them in a DOP-1 
> operator in the topology or send them to an OperatorCoordinator to do 
> the commit there. This is where the source/sink duality that Steven 
> mentions becomes visible.
> 
> ## Kafka
> 
> Kafka is a problematic case because it doesn't really support 
> transactions as outlined above. Our current Sink implementations works 
> around that with hacks but that only gets us so far.
> 
> The problem with Kafka is that we need to aggressively clean up pending 
> transactions in case a failure happens. Otherwise stale transactions 
> would block downstream consumers. See here for details: 
> http://kafka.apache.org/documentation/#isolation.level.
> 
> The way we solve this in the current Kafka sink is by using a fixed pool 
> of transactional IDs and then cancelling all outstanding transactions 
> for the IDs when we restore from a savepoint. In order for this to work 
> we need to recycle the IDs, so there needs to be a back-channel from the 
> Committer to the Writter, or they need to share internal state.
> 
> I don't get see a satisfying solution for this so I think we should 
> exclude this from the initial version.
> 
> ## On Write-Ahead-Log Sinks
> 
> Some sinks, like ES or Cassandra would require that we keep a WAL in 
> Flink and then ship the contents to the external system on checkpoint. 
> The reason is that these systems don't support real transactions where 
> you can prepare them in one process and commit them from another process.
> 
> Best,
> Aljoscha
> 
> 
> On 11.09.20 02:23, Steven Wu wrote:
>> Guowei,
>>
>> Thanks a lot for the proposal and starting the discussion thread. Very
>> excited.
>>
>> For the big question of "Is the sink an operator or a topology?", I 
>> have a
>> few related sub questions.
>> * Where should we run the committers?
>> * Is the committer parallel or single parallelism?
>> * Can a single choice satisfy all sinks?
>>
>> Trying to envision how some sinks can be implemented with this new 
>> unified
>> sink interface.
>>
>> 1. Kafka sink
>>
>> Kafka supports non-transactional and transactional writes
>> * Non-transaction writes don't need commit action. we can have *parallel
>> writers and no/no-op committers*. This is probably true for other
>> non-transactional message queues.
>> * Transaction writes can be implemented as *parallel writers and parallel
>> committers*. In this case, I don't know if it makes sense to separate
>> writers and committers into two separate operators, because they probably
>> need to share the same KafkaProducer object.
>>
>> Either way, both writers and committers probably should *run inside task
>> managers*.
>>
>> 2. ES sink
>>
>> ES sink typically buffers the data up to a certain size or time threshold
>> and then uploads/commits a batch to ES. Writers buffer data and flush 
>> when
>> needed, and committer does the HTTP bulk upload to commit. To avoid
>> serialization/deserialization cost, we should run *parallel writers and
>> parallel committers* and they *should be* *chained or bundled together*
>> while *running inside task managers*.
>>
>> It can also be implemented as *parallel writers and no/no-op committers*,
>> where all logics (batching and upload) are put inside the writers.
>>
>> 3. Iceberg [1] sink
>>
>> It is currently implemented as two-stage operators with *parallel writers
>> and single-parallelism committers*.
>> * *parallel writers* that write records into data files. Upon checkpoint,
>> writers flush and upload the files, and send the metadata/location of the
>> data files to the downstream committer. Writers need to do the flush 
>> inside
>> the "prepareSnapshotPreBarrier" method (NOT "snapshotState" method) 
>> before
>> forwarding the checkpoint barrier to the committer
>> * single-parallelism committer operator. It collects data files from
>> upstream writers. During "snapshotState", it saves collected data 
>> files (or
>> an uber metadata file) into state. When the checkpoint is completed, 
>> inside
>> "notifyCheckpointComplete" it commits those data files to Iceberg 
>> tables. *The
>> committer has to be single parallelism*, because we don't want 
>> hundreds or
>> thousands of parallel committers to compete for commit operations with
>> opportunistic concurrency control. It will be very inefficient and 
>> probably
>> infeasible if the parallelism is high. Too many tiny commits/transactions
>> can also slow down both the table write and read paths due to too many
>> manifest files.
>>
>> Right now, both Iceberg writer and committer operators run inside task
>> managers. It has one major drawback. With Iceberg sink, embarrassingly
>> parallel jobs won't be embarrassingly parallel anymore. That breaks the
>> benefit of region recovery for embarrassingly parallel DAG. Conceptually,
>> the Writer-Committer sink pattern is like the mirroring of the FLIP-27
>> Enumerator-Reader source pattern. It will be better *if the committer can
>> run inside the job manager* like the SplitEnumerator for the FLIP-27
>> source.
>>
>> -----------------------
>> Additional questions regarding the doc/API
>> * Any example for the writer shared state (Writer#snapshotSharedState)?
>> * We allow the case where the writer has no state, right? Meaning WriterS
>> can be Void.
>>
>> [1] https://iceberg.apache.org/
>>
>> Thanks,
>> Steven
>>
>> On Thu, Sep 10, 2020 at 6:44 AM Guowei Ma <gu...@gmail.com> wrote:
>>
>>> Hi, devs & users
>>>
>>> As discussed in FLIP-131[1], Flink will deprecate the DataSet API in 
>>> favor
>>> of DataStream API and Table API. Users should be able to use 
>>> DataStream API
>>> to write jobs that support both bounded and unbounded execution modes.
>>> However, Flink does not provide a sink API to guarantee the Exactly-once
>>> semantics in both bounded and unbounded scenarios, which blocks the
>>> unification.
>>>
>>> So we want to introduce a new unified sink API which could let the user
>>> develop the sink once and run it everywhere. You could find more 
>>> details in
>>> FLIP-143[2].
>>>
>>> The FLIP contains some open questions that I'd really appreciate inputs
>>> from the community. Some of the open questions include:
>>>
>>>     1. We provide two alternative Sink API in the FLIP. The only
>>>     difference between the two versions is how to expose the state to 
>>> the user.
>>>     We want to know which one is your preference?
>>>     2. How does the sink API support to write to the Hive?
>>>     3. Is the sink an operator or a topology?
>>>
>>> [1]
>>> https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=158866741 
>>>
>>> [2]
>>> https://cwiki.apache.org/confluence/display/FLINK/FLIP-143%3A+Unified+Sink+API 
>>>
>>>
>>> Best,
>>> Guowei
>>>
>>
> 


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

Posted by Aljoscha Krettek <al...@apache.org>.
Hi Everyone,

thanks to Guowei for publishing the FLIP, and thanks Steven for the very 
thoughtful email!

We thought a lot internally about some of the questions you posted but 
left a lot (almost all) of the implementation details out of the FLIP 
for now because we wanted to focus on semantics and API. I'll try and 
address the points below.

## Initial Scope of the new Sink API

We need to accept some initial scope that we want to achieve for Flink 
1.12. I don't think we can try and find the solution that will work for 
all current and future external systems. For me, the initial goal would 
be to produce a Sink API and implementations for systems where you can 
prepare "committables" in one process and commit those from another 
process. Those are systems that support "real" transactions as you need 
them in a two-phase commit protocol. This includes:

  - File Sink, including HDFS, S3 via special part-file uploads
  - Iceberg
  - HDFS

The work should include runtime support for both BATCH and STREAMING 
execution as outlined in https://s.apache.org/FLIP-134.

Supporting Kafka already becomes difficult but I'll go into that below.

## Where to run the Committer

Guowei hinted at this in the FLIP: the idea is that the Framework 
decides where to run the committer based on requirements and based on 
the execution mode (STREAMING or BATCH).

Something that is not in the FLIP but which we thought about is that we 
need to allow different types of committers. I'm currently thinking we 
need at least a normal "Committer" and a "GlobalCommiter" (name TBD).

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.

An interesting read on this topic is the discussion on 
https://issues.apache.org/jira/browse/MAPREDUCE-4815. About the Hadoop 
FileOutputCommitter and the two different available algorithms for 
committing final job/task results.

These interfaces untie the sink implementation from the Runtime and we 
could, for example, have a runtime like this:

### BATCH

  - Collect all committables and store them in a fault tolerant way 
until the job finishes
  - For a normal Commit function, call it on the individual commits. We 
can potentially distribute this if it becomes a bottleneck
  - For GlobalCommit function, call it will all the commits. This cannot 
be distributed

We can collect the committables in an OperatorCoordinator or potentially 
somehow in a task. Though I prefer an OperatorCoordinator right now. The 
operator coordinator needs to keep the commits in a fault-tolerant way.

### STREAMING

  - For normal Commit, keep the committables in state on the individual 
tasks, commit them when a checkpoint completes
  - For global CommitFunction we have options: collect them in a DOP-1 
operator in the topology or send them to an OperatorCoordinator to do 
the commit there. This is where the source/sink duality that Steven 
mentions becomes visible.

## Kafka

Kafka is a problematic case because it doesn't really support 
transactions as outlined above. Our current Sink implementations works 
around that with hacks but that only gets us so far.

The problem with Kafka is that we need to aggressively clean up pending 
transactions in case a failure happens. Otherwise stale transactions 
would block downstream consumers. See here for details: 
http://kafka.apache.org/documentation/#isolation.level.

The way we solve this in the current Kafka sink is by using a fixed pool 
of transactional IDs and then cancelling all outstanding transactions 
for the IDs when we restore from a savepoint. In order for this to work 
we need to recycle the IDs, so there needs to be a back-channel from the 
Committer to the Writter, or they need to share internal state.

I don't get see a satisfying solution for this so I think we should 
exclude this from the initial version.

## On Write-Ahead-Log Sinks

Some sinks, like ES or Cassandra would require that we keep a WAL in 
Flink and then ship the contents to the external system on checkpoint. 
The reason is that these systems don't support real transactions where 
you can prepare them in one process and commit them from another process.

Best,
Aljoscha


On 11.09.20 02:23, Steven Wu wrote:
> Guowei,
> 
> Thanks a lot for the proposal and starting the discussion thread. Very
> excited.
> 
> For the big question of "Is the sink an operator or a topology?", I have a
> few related sub questions.
> * Where should we run the committers?
> * Is the committer parallel or single parallelism?
> * Can a single choice satisfy all sinks?
> 
> Trying to envision how some sinks can be implemented with this new unified
> sink interface.
> 
> 1. Kafka sink
> 
> Kafka supports non-transactional and transactional writes
> * Non-transaction writes don't need commit action. we can have *parallel
> writers and no/no-op committers*. This is probably true for other
> non-transactional message queues.
> * Transaction writes can be implemented as *parallel writers and parallel
> committers*. In this case, I don't know if it makes sense to separate
> writers and committers into two separate operators, because they probably
> need to share the same KafkaProducer object.
> 
> Either way, both writers and committers probably should *run inside task
> managers*.
> 
> 2. ES sink
> 
> ES sink typically buffers the data up to a certain size or time threshold
> and then uploads/commits a batch to ES. Writers buffer data and flush when
> needed, and committer does the HTTP bulk upload to commit. To avoid
> serialization/deserialization cost, we should run *parallel writers and
> parallel committers* and they *should be* *chained or bundled together*
> while *running inside task managers*.
> 
> It can also be implemented as *parallel writers and no/no-op committers*,
> where all logics (batching and upload) are put inside the writers.
> 
> 3. Iceberg [1] sink
> 
> It is currently implemented as two-stage operators with *parallel writers
> and single-parallelism committers*.
> * *parallel writers* that write records into data files. Upon checkpoint,
> writers flush and upload the files, and send the metadata/location of the
> data files to the downstream committer. Writers need to do the flush inside
> the "prepareSnapshotPreBarrier" method (NOT "snapshotState" method) before
> forwarding the checkpoint barrier to the committer
> * single-parallelism committer operator. It collects data files from
> upstream writers. During "snapshotState", it saves collected data files (or
> an uber metadata file) into state. When the checkpoint is completed, inside
> "notifyCheckpointComplete" it commits those data files to Iceberg tables. *The
> committer has to be single parallelism*, because we don't want hundreds or
> thousands of parallel committers to compete for commit operations with
> opportunistic concurrency control. It will be very inefficient and probably
> infeasible if the parallelism is high. Too many tiny commits/transactions
> can also slow down both the table write and read paths due to too many
> manifest files.
> 
> Right now, both Iceberg writer and committer operators run inside task
> managers. It has one major drawback. With Iceberg sink, embarrassingly
> parallel jobs won't be embarrassingly parallel anymore. That breaks the
> benefit of region recovery for embarrassingly parallel DAG. Conceptually,
> the Writer-Committer sink pattern is like the mirroring of the FLIP-27
> Enumerator-Reader source pattern. It will be better *if the committer can
> run inside the job manager* like the SplitEnumerator for the FLIP-27
> source.
> 
> -----------------------
> Additional questions regarding the doc/API
> * Any example for the writer shared state (Writer#snapshotSharedState)?
> * We allow the case where the writer has no state, right? Meaning WriterS
> can be Void.
> 
> [1] https://iceberg.apache.org/
> 
> Thanks,
> Steven
> 
> On Thu, Sep 10, 2020 at 6:44 AM Guowei Ma <gu...@gmail.com> wrote:
> 
>> Hi, devs & users
>>
>> As discussed in FLIP-131[1], Flink will deprecate the DataSet API in favor
>> of DataStream API and Table API. Users should be able to use DataStream API
>> to write jobs that support both bounded and unbounded execution modes.
>> However, Flink does not provide a sink API to guarantee the Exactly-once
>> semantics in both bounded and unbounded scenarios, which blocks the
>> unification.
>>
>> So we want to introduce a new unified sink API which could let the user
>> develop the sink once and run it everywhere. You could find more details in
>> FLIP-143[2].
>>
>> The FLIP contains some open questions that I'd really appreciate inputs
>> from the community. Some of the open questions include:
>>
>>     1. We provide two alternative Sink API in the FLIP. The only
>>     difference between the two versions is how to expose the state to the user.
>>     We want to know which one is your preference?
>>     2. How does the sink API support to write to the Hive?
>>     3. Is the sink an operator or a topology?
>>
>> [1]
>> https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=158866741
>> [2]
>> https://cwiki.apache.org/confluence/display/FLINK/FLIP-143%3A+Unified+Sink+API
>>
>> Best,
>> Guowei
>>
> 


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

Posted by Steven Wu <st...@gmail.com>.
Guowei,

Thanks a lot for the proposal and starting the discussion thread. Very
excited.

For the big question of "Is the sink an operator or a topology?", I have a
few related sub questions.
* Where should we run the committers?
* Is the committer parallel or single parallelism?
* Can a single choice satisfy all sinks?

Trying to envision how some sinks can be implemented with this new unified
sink interface.

1. Kafka sink

Kafka supports non-transactional and transactional writes
* Non-transaction writes don't need commit action. we can have *parallel
writers and no/no-op committers*. This is probably true for other
non-transactional message queues.
* Transaction writes can be implemented as *parallel writers and parallel
committers*. In this case, I don't know if it makes sense to separate
writers and committers into two separate operators, because they probably
need to share the same KafkaProducer object.

Either way, both writers and committers probably should *run inside task
managers*.

2. ES sink

ES sink typically buffers the data up to a certain size or time threshold
and then uploads/commits a batch to ES. Writers buffer data and flush when
needed, and committer does the HTTP bulk upload to commit. To avoid
serialization/deserialization cost, we should run *parallel writers and
parallel committers* and they *should be* *chained or bundled together*
while *running inside task managers*.

It can also be implemented as *parallel writers and no/no-op committers*,
where all logics (batching and upload) are put inside the writers.

3. Iceberg [1] sink

It is currently implemented as two-stage operators with *parallel writers
and single-parallelism committers*.
* *parallel writers* that write records into data files. Upon checkpoint,
writers flush and upload the files, and send the metadata/location of the
data files to the downstream committer. Writers need to do the flush inside
the "prepareSnapshotPreBarrier" method (NOT "snapshotState" method) before
forwarding the checkpoint barrier to the committer
* single-parallelism committer operator. It collects data files from
upstream writers. During "snapshotState", it saves collected data files (or
an uber metadata file) into state. When the checkpoint is completed, inside
"notifyCheckpointComplete" it commits those data files to Iceberg tables. *The
committer has to be single parallelism*, because we don't want hundreds or
thousands of parallel committers to compete for commit operations with
opportunistic concurrency control. It will be very inefficient and probably
infeasible if the parallelism is high. Too many tiny commits/transactions
can also slow down both the table write and read paths due to too many
manifest files.

Right now, both Iceberg writer and committer operators run inside task
managers. It has one major drawback. With Iceberg sink, embarrassingly
parallel jobs won't be embarrassingly parallel anymore. That breaks the
benefit of region recovery for embarrassingly parallel DAG. Conceptually,
the Writer-Committer sink pattern is like the mirroring of the FLIP-27
Enumerator-Reader source pattern. It will be better *if the committer can
run inside the job manager* like the SplitEnumerator for the FLIP-27
source.

-----------------------
Additional questions regarding the doc/API
* Any example for the writer shared state (Writer#snapshotSharedState)?
* We allow the case where the writer has no state, right? Meaning WriterS
can be Void.

[1] https://iceberg.apache.org/

Thanks,
Steven

On Thu, Sep 10, 2020 at 6:44 AM Guowei Ma <gu...@gmail.com> wrote:

> Hi, devs & users
>
> As discussed in FLIP-131[1], Flink will deprecate the DataSet API in favor
> of DataStream API and Table API. Users should be able to use DataStream API
> to write jobs that support both bounded and unbounded execution modes.
> However, Flink does not provide a sink API to guarantee the Exactly-once
> semantics in both bounded and unbounded scenarios, which blocks the
> unification.
>
> So we want to introduce a new unified sink API which could let the user
> develop the sink once and run it everywhere. You could find more details in
> FLIP-143[2].
>
> The FLIP contains some open questions that I'd really appreciate inputs
> from the community. Some of the open questions include:
>
>    1. We provide two alternative Sink API in the FLIP. The only
>    difference between the two versions is how to expose the state to the user.
>    We want to know which one is your preference?
>    2. How does the sink API support to write to the Hive?
>    3. Is the sink an operator or a topology?
>
> [1]
> https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=158866741
> [2]
> https://cwiki.apache.org/confluence/display/FLINK/FLIP-143%3A+Unified+Sink+API
>
> Best,
> Guowei
>

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

Posted by Steven Wu <st...@gmail.com>.
Guowei,

Thanks a lot for the proposal and starting the discussion thread. Very
excited.

For the big question of "Is the sink an operator or a topology?", I have a
few related sub questions.
* Where should we run the committers?
* Is the committer parallel or single parallelism?
* Can a single choice satisfy all sinks?

Trying to envision how some sinks can be implemented with this new unified
sink interface.

1. Kafka sink

Kafka supports non-transactional and transactional writes
* Non-transaction writes don't need commit action. we can have *parallel
writers and no/no-op committers*. This is probably true for other
non-transactional message queues.
* Transaction writes can be implemented as *parallel writers and parallel
committers*. In this case, I don't know if it makes sense to separate
writers and committers into two separate operators, because they probably
need to share the same KafkaProducer object.

Either way, both writers and committers probably should *run inside task
managers*.

2. ES sink

ES sink typically buffers the data up to a certain size or time threshold
and then uploads/commits a batch to ES. Writers buffer data and flush when
needed, and committer does the HTTP bulk upload to commit. To avoid
serialization/deserialization cost, we should run *parallel writers and
parallel committers* and they *should be* *chained or bundled together*
while *running inside task managers*.

It can also be implemented as *parallel writers and no/no-op committers*,
where all logics (batching and upload) are put inside the writers.

3. Iceberg [1] sink

It is currently implemented as two-stage operators with *parallel writers
and single-parallelism committers*.
* *parallel writers* that write records into data files. Upon checkpoint,
writers flush and upload the files, and send the metadata/location of the
data files to the downstream committer. Writers need to do the flush inside
the "prepareSnapshotPreBarrier" method (NOT "snapshotState" method) before
forwarding the checkpoint barrier to the committer
* single-parallelism committer operator. It collects data files from
upstream writers. During "snapshotState", it saves collected data files (or
an uber metadata file) into state. When the checkpoint is completed, inside
"notifyCheckpointComplete" it commits those data files to Iceberg tables. *The
committer has to be single parallelism*, because we don't want hundreds or
thousands of parallel committers to compete for commit operations with
opportunistic concurrency control. It will be very inefficient and probably
infeasible if the parallelism is high. Too many tiny commits/transactions
can also slow down both the table write and read paths due to too many
manifest files.

Right now, both Iceberg writer and committer operators run inside task
managers. It has one major drawback. With Iceberg sink, embarrassingly
parallel jobs won't be embarrassingly parallel anymore. That breaks the
benefit of region recovery for embarrassingly parallel DAG. Conceptually,
the Writer-Committer sink pattern is like the mirroring of the FLIP-27
Enumerator-Reader source pattern. It will be better *if the committer can
run inside the job manager* like the SplitEnumerator for the FLIP-27
source.

-----------------------
Additional questions regarding the doc/API
* Any example for the writer shared state (Writer#snapshotSharedState)?
* We allow the case where the writer has no state, right? Meaning WriterS
can be Void.

[1] https://iceberg.apache.org/

Thanks,
Steven

On Thu, Sep 10, 2020 at 6:44 AM Guowei Ma <gu...@gmail.com> wrote:

> Hi, devs & users
>
> As discussed in FLIP-131[1], Flink will deprecate the DataSet API in favor
> of DataStream API and Table API. Users should be able to use DataStream API
> to write jobs that support both bounded and unbounded execution modes.
> However, Flink does not provide a sink API to guarantee the Exactly-once
> semantics in both bounded and unbounded scenarios, which blocks the
> unification.
>
> So we want to introduce a new unified sink API which could let the user
> develop the sink once and run it everywhere. You could find more details in
> FLIP-143[2].
>
> The FLIP contains some open questions that I'd really appreciate inputs
> from the community. Some of the open questions include:
>
>    1. We provide two alternative Sink API in the FLIP. The only
>    difference between the two versions is how to expose the state to the user.
>    We want to know which one is your preference?
>    2. How does the sink API support to write to the Hive?
>    3. Is the sink an operator or a topology?
>
> [1]
> https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=158866741
> [2]
> https://cwiki.apache.org/confluence/display/FLINK/FLIP-143%3A+Unified+Sink+API
>
> Best,
> Guowei
>