You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@flink.apache.org by Roman Khachatryan <ro...@apache.org> on 2021/12/02 09:15:09 UTC

Re: [DISCUSS] FLIP-191: Extend unified Sink interface to support small file compaction

Thanks for clarifying (I was initially confused by merging state files
rather than output files).

> At some point, Flink will definitely have some WAL adapter that can turn any sink into an exactly-once sink (with some caveats). For now, we keep that as an orthogonal solution as it has a rather high price (bursty workload with high latency). Ideally, we can keep the compaction asynchronously...

Yes, that would be something like a WAL. I agree that it would have a
different set of trade-offs.


Regards,
Roman

On Mon, Nov 29, 2021 at 3:33 PM Arvid Heise <ar...@apache.org> wrote:
>>
>> > One way to avoid write-read-merge is by wrapping SinkWriter with
>> > another one, which would buffer input elements in a temporary storage
>> > (e.g. local file) until a threshold is reached; after that, it would
>> > invoke the original SinkWriter. And if a checkpoint barrier comes in
>> > earlier, it would send written data to some aggregator.
>>
>> I think perhaps this seems to be a kind of WAL method? Namely we first
>> write the elements to some WAL logs and persist them on checkpoint
>> (in snapshot or remote FS), or we directly write WAL logs to the remote
>> FS eagerly.
>>
> At some point, Flink will definitely have some WAL adapter that can turn any sink into an exactly-once sink (with some caveats). For now, we keep that as an orthogonal solution as it has a rather high price (bursty workload with high latency). Ideally, we can keep the compaction asynchronously...
>
> On Mon, Nov 29, 2021 at 8:52 AM Yun Gao <yu...@aliyun.com.invalid> wrote:
>>
>> Hi,
>>
>> @Roman very sorry for the late response for a long time,
>>
>> > Merging artifacts from multiple checkpoints would apparently
>> require multiple concurrent checkpoints
>>
>> I think it might not need concurrent checkpoints: suppose some
>> operators (like the committer aggregator in the option 2) maintains
>> the list of files to merge, it could stores the lists of files to merge
>> in the states, then after several checkpoints are done and we have
>> enough files, we could merge all the files in the list.
>>
>> > Asynchronous merging in an aggregator would require some resolution
>> > logic on recovery, so that a merged artifact can be used if the
>> > original one was deleted. Otherwise, wouldn't recovery fail because
>> > some artifacts are missing?
>> > We could also defer deletion until the "compacted" checkpoint is
>> > subsumed - but isn't it too late, as it will be deleted anyways once
>> > subsumed?
>>
>> I think logically we could delete the original files once the "compacted" checkpoint
>> (which finish merging the compacted files and record it in the checkpoint) is completed
>> in all the options. If there are failover before we it, we could restart the merging and if
>> there are failover after it, we could have already recorded the files in the checkpoint.
>>
>> > One way to avoid write-read-merge is by wrapping SinkWriter with
>> > another one, which would buffer input elements in a temporary storage
>> > (e.g. local file) until a threshold is reached; after that, it would
>> > invoke the original SinkWriter. And if a checkpoint barrier comes in
>> > earlier, it would send written data to some aggregator.
>>
>> I think perhaps this seems to be a kind of WAL method? Namely we first
>> write the elements to some WAL logs and persist them on checkpoint
>> (in snapshot or remote FS), or we directly write WAL logs to the remote
>> FS eagerly.
>>
>> Sorry if I do not understand correctly somewhere.
>>
>> Best,
>> Yun
>>
>>
>> ------------------------------------------------------------------
>> From:Roman Khachatryan <ro...@apache.org>
>> Send Time:2021 Nov. 9 (Tue.) 22:03
>> To:dev <de...@flink.apache.org>
>> Subject:Re: [DISCUSS] FLIP-191: Extend unified Sink interface to support small file compaction
>>
>> Hi everyone,
>>
>> Thanks for the proposal and the discussion, I have some remarks:
>> (I'm not very familiar with the new Sink API but I thought about the
>> same problem in context of the changelog state backend)
>>
>> 1. Merging artifacts from multiple checkpoints would apparently
>> require multiple concurrent checkpoints (otherwise, a new checkpoint
>> won't be started before completing the previous one; and the previous
>> one can't be completed before durably storing the artifacts). However,
>> concurrent checkpoints are currently not supported with Unaligned
>> checkpoints (this is besides increasing e2e-latency).
>>
>> 2. Asynchronous merging in an aggregator would require some resolution
>> logic on recovery, so that a merged artifact can be used if the
>> original one was deleted. Otherwise, wouldn't recovery fail because
>> some artifacts are missing?
>> We could also defer deletion until the "compacted" checkpoint is
>> subsumed - but isn't it too late, as it will be deleted anyways once
>> subsumed?
>>
>> 3. Writing small files, then reading and merging them for *every*
>> checkpoint seems worse than only reading them on recovery. I guess I'm
>> missing some cases of reading, so to me it would make sense to mention
>> these cases explicitly in the FLIP motivation section.
>>
>> 4. One way to avoid write-read-merge is by wrapping SinkWriter with
>> another one, which would buffer input elements in a temporary storage
>> (e.g. local file) until a threshold is reached; after that, it would
>> invoke the original SinkWriter. And if a checkpoint barrier comes in
>> earlier, it would send written data to some aggregator. It will
>> increase checkpoint delay (async phase) compared to the current Flink;
>> but not compared to the write-read-merge solution, IIUC.
>> Then such "BufferingSinkWriters" could aggregate input elements from
>> each other, potentially recursively (I mean something like
>> https://cwiki.apache.org/confluence/download/attachments/173082889/DSTL-DFS-DAG.png
>> )
>>
>> 5. Reducing the number of files by reducing aggregator parallelism as
>> opposed to merging on reaching size threshold will likely be less
>> optimal and more difficult to configure. OTH, thresholds might be more
>> difficult to implement and (with recursive merging) would incur higher
>> latency. Maybe that's also something to decide explicitly or at least
>> mention in the FLIP.
>>
>>
>>
>> Regards,
>> Roman
>>
>>
>> On Tue, Nov 9, 2021 at 5:23 AM Reo Lei <le...@gmail.com> wrote:
>> >
>> > Hi Fabian,
>> >
>> > Thanks for drafting the FLIP and trying to support small file compaction. I
>> > think this feature is very urgent and valuable for users(at least for me).
>> >
>> > Currently I am trying to support streaming rewrite(compact) for Iceberg on
>> > PR#3323 <https://github.com/apache/iceberg/pull/3323>. As Steven mentioned,
>> > Iceberg sink and compact data through the following steps:
>> > Step-1: Some parallel data writer(sinker) to write streaming data as files.
>> > Step-2: A single parallelism data files committer to commit the completed
>> > files as soon as possible to make them available.
>> > Step-3: Some parallel file rewriter(compactor) to collect committed files
>> > from multiple checkpoints, and rewriter(compact) them together once the
>> > total file size or number of files reach the threshold.
>> > Step-4: A single parallelism rewrite(compact) result committer to commit
>> > the rewritten(compacted) files to replace the old files and make them
>> > available.
>> >
>> >
>> > If Flink want to support small file compaction, some key point I think is
>> > necessary:
>> >
>> > 1, Compact files from multiple checkpoints.
>> > I totally agree with Jingsong, because completed file size usually could
>> > not reach the threshold in a single checkpoint. Especially for partitioned
>> > table, we need to compact the files of each partition, but usually the file
>> > size of each partition will be different and may not reach the merge
>> > threshold. If we compact these files, in a single checkpoint, regardless of
>> > whether the total file size reaches the threshold, then the value of
>> > compacting will be diminished and we will still get small files because
>> > these compacted files are not reach to target size. So we need the
>> > compactor to collect committed files from multiple checkpoints and compact
>> > them until they reach the threshold.
>> >
>> > 2, Separate write phase and compact phase.
>> > Users usually hope the data becomes available as soon as possible, and the
>> >  end-to-end latency is very important. I think we need to separate the
>> > write and compact phase. For the write phase, there include the Step-1
>> > and Step-2, we sink data as file and commit it pre checkpoint and regardless
>> > of whether the file size it is. That could ensure the data will be
>> > available ASAP. For the compact phase, there include the Step-3
>> > and Step-4,  the compactor should collect committed files from multiple
>> > checkpoints and compact them asynchronously once they reach the threshold,
>> > and the compact committer will commit the  compaction result in the next
>> > checkpoint. We compact the committed files asynchronously because we don't
>> > want the compaction to affect the data sink or the whole pipeline.
>> >
>> > 3, Exactly once guarantee between write and compact phase.
>> > Once we separate write phase and compact phase, we need to consider
>> > how to guarantee
>> > the exact once semantic between two phases. We should not lose any data or
>> > files on the compactor(Step-3) in any case and cause the compaction result
>> > to be inconsistent with before. I think flink should provide an easy-to-use
>> > interface to make that easier.
>> >
>> > 4, Metadata operation and  compaction result validation.
>> > In the compact phase, there may be not only compact files, but also a lot
>> > of metadata operations, such as the iceberg needing to read/write manifest
>> > and do MOR. And we need some interface to support users to do some
>> > validation of the compaction result. I think these points should be
>> > considered when we design the compaction API.
>> >
>> >
>> > Back to FLIP-191, option 1 looks very complicated while option 2 is
>> > relatively simple, but neither of these two solutions separates the write
>> > phase from the compact phase. So I think we should consider the points I
>> > mentioned above. And if you have any other questions you can always feel
>> > free to reach out to me!
>> >
>> > BR,
>> > Reo
>> >
>> > Fabian Paul <fa...@ververica.com> 于2021年11月8日周一 下午7:59写道:
>> >
>> > > Hi all,
>> > >
>> > > Thanks for the lively discussions. I am really excited to see so many
>> > > people
>> > > participating in this thread. It also underlines the need that many people
>> > > would
>> > > like to see a solution soon.
>> > >
>> > > I have updated the FLIP and removed the parallelism configuration because
>> > > it is
>> > > unnecessary since users can configure a constant exchange key to send all
>> > > committables to only one committable aggregator.
>> > >
>> > >
>> > > 1. Burden for developers w.r.t batch stream unification.
>> > >
>> > > @yun @guowei, from a theoretical point you are right about exposing the
>> > > DataStream
>> > > API in the sink users have the full power to write correct batch and
>> > > streaming
>> > > sinks. I think in reality a lot of users still struggle to build pipelines
>> > > with
>> > > i.e. the operator pipeline which works correct in streaming and batch mode.
>> > > Another problem I see is by exposing more deeper concepts is that we
>> > > cannot do
>> > > any optimization because we cannot reason about how sinks are built in the
>> > > future.
>> > >
>> > > We should also try to steer users towards using only `Functions` to give
>> > > us more
>> > > flexibility to swap the internal operator representation. I agree with
>> > > @yun we
>> > > should try to make the `ProcessFunction` more versatile to work on that
>> > > goal but
>> > > I see this as unrelated to the FLIP.
>> > >
>> > >
>> > > 2. Regarding Commit / Global commit
>> > >
>> > > I envision the global committer to be specific depending on the data lake
>> > > solution you want to write to. However, it is entirely orthogonal to the
>> > > compaction.
>> > > Currently, I do not expect any changes w.r.t the Global commit introduces
>> > > by
>> > > this FLIP.
>> > >
>> > >
>> > > 3. Regarding the case of trans-checkpoints merging
>> > >
>> > > @yun, as user, I would expect that if the committer receives in a
>> > > checkpoint files
>> > > to merge/commit that these are also finished when the checkpoint finishes.
>> > > I think all sinks rely on this principle currently i.e., KafkaSink needs to
>> > > commit all open transactions until the next checkpoint can happen.
>> > >
>> > > Maybe in the future, we can somehow move the Committer#commit call to an
>> > > asynchronous execution, but we should discuss it as a separate thread.
>> > >
>> > > > We probably should first describe the different causes of small files and
>> > > > what problems was this proposal trying to solve. I wrote a data shuffling
>> > > > proposal [1] for Flink Iceberg sink (shared with Iceberg community [2]).
>> > > It
>> > > > can address small files problems due to skewed data distribution across
>> > > > Iceberg table partitions. Streaming shuffling before writers (to files)
>> > > is
>> > > > typically more efficient than post-write file compaction (which involves
>> > > > read-merge-write). It is usually cheaper to prevent a problem (small
>> > > files)
>> > > > than fixing it.
>> > >
>> > >
>> > > @steven you are raising a good point, although I think only using a
>> > > customizable
>> > > shuffle won't address the generation of small files. One assumption is that
>> > > at least the sink generates one file per subtask, which can already be too
>> > > many.
>> > > Another problem is that with low checkpointing intervals, the files do not
>> > > meet
>> > > the required size. The latter point is probably addressable by changing the
>> > > checkpoint interval, which might be inconvenient for some users.
>> > >
>> > > > The sink coordinator checkpoint problem (mentioned in option 1) would be
>> > > > great if Flink can address it. In the spirit of source
>> > > (enumerator-reader)
>> > > > and sink (writer-coordinator) duality, sink coordinator checkpoint should
>> > > > happen after the writer operator. This would be a natural fit to support
>> > > > global committer in FLIP-143. It is probably an orthogonal matter to this
>> > > > proposal.
>> > >
>> > >
>> > > To me the question here is what are the benefits of having a coordinator in
>> > > comparison to a global committer/aggregator operator.
>> > >
>> > > > Personally, I am usually in favor of keeping streaming ingestion (to data
>> > > > lake) relatively simple and stable. Also sometimes compaction and sorting
>> > > > are performed together in data rewrite maintenance jobs to improve read
>> > > > performance. In that case, the value of compacting (in Flink streaming
>> > > > ingestion) diminishes.
>> > >
>> > >
>> > > I agree it is always possible to have scheduled maintenance jobs keeping
>> > > care of
>> > > your data i.e., doing compaction. Unfortunately, the downside is that you
>> > > have to your data after it is already available for other downstream
>> > > consumers.
>> > > I guess this can lead to all kinds of visibility problems. I am also
>> > > surprised that
>> > > you personally are a fan of this approach and, on the other hand, are
>> > > developing
>> > > the Iceberg sink, which goes somewhat against your mentioned principle of
>> > > keeping
>> > > the sink simple.
>> > >
>> > > > Currently, it is unclear from the doc and this thread where the
>> > > compaction
>> > > > is actually happening. Jingsong's reply described one model
>> > > > writer (parallel) -> aggregator (single-parallelism compaction planner)
>> > > ->
>> > > > compactor (parallel) -> global committer (single-parallelism)
>> > >
>> > >
>> > > My idea of the topology is very similar to the one outlined by Jinsong. The
>> > > compaction will happen in the committer operator.
>> > >
>> > > >
>> > > > In the Iceberg community, the following model has been discussed. It is
>> > > > better for Iceberg because it won't delay the data availability.
>> > > > writer (parallel) -> global committer for append (single parallelism) ->
>> > > > compactor (parallel) -> global committer for rewrite commit (single
>> > > > parallelism)
>> > >
>> > >
>> > > From a quick glimpse, it seems that the exact same topology is possible to
>> > > express with the committable aggregator, but this definitely depends on
>> > > the exact
>> > > setup.
>> > >
>> > > Best,
>> > > Fabian
>>

回复: [DISCUSS] FLIP-191: Extend unified Sink interface to support small file compaction

Posted by GALENO <70...@qq.com.INVALID>.
取消订阅




------------------&nbsp;原始邮件&nbsp;------------------
发件人:                                                                                                                        "dev"                                                                                    <fpaul@apache.org&gt;;
发送时间:&nbsp;2021年12月13日(星期一) 晚上11:59
收件人:&nbsp;"dev"<dev@flink.apache.org&gt;;

主题:&nbsp;Re: [DISCUSS] FLIP-191: Extend unified Sink interface to support small file compaction



Hi all,

After a lot of discussions with different, we received very fruitful
feedback and reworked the ideas behind this FLIP. Initially, we had
the impression that the compaction problem is solvable by a single
topology that we can reuse across different sinks. We now have a
better understanding that different external systems require different
compaction mechanism i.e. Hive requires compaction before finally
registering the file in the metastore or Iceberg compacts the files
after they have been registered and just lazily compacts them.

Considering all these different views we came up with a design that
builds upon what @guowei.mgw@gmail.com and @yungao.gy@aliyun.com have
proposed at the beginning. We allow inserting custom topologies before
and after the SinkWriters and Committers. Furthermore, we do not see
it as a downside. The Sink interfaces that will expose the DataStream
to the user reside in flink-streaming-java in contrast to the basic
Sink interfaces that reside fin flink-core deem it to be only used by
expert users.

Moreover, we also wanted to remove the global committer from the
unified Sink interfaces and replace it with a custom post-commit
topology. Unfortunately, we cannot do it without breaking the Sink
interface since the GlobalCommittables are part of the parameterized
Sink interface. Thus, we propose building a new Sink V2 interface
consisting of composable interfaces that do not offer the
GlobalCommitter anymore. We will implement a utility to extend a Sink
with post topology that mimics the behavior of the GlobalCommitter.
The new Sink V2 provides the same sort of methods as the Sink V1
interface, so a migration of sinks that do not use the GlobalCommitter
should be very easy.
We plan to keep the existing Sink V1 interfaces to not break
externally built sinks. As part of this FLIP, we migrate all the
connectors inside of the main repository to the new Sink V2 API.

The FLIP document is also updated and includes the proposed changes.

Looking forward to your feedback,
Fabian

https://cwiki.apache.org/confluence/display/FLINK/FLIP-191%3A+Extend+unified+Sink+interface+to+support+small+file+compaction


On Thu, Dec 2, 2021 at 10:15 AM Roman Khachatryan <roman@apache.org&gt; wrote:
&gt;
&gt; Thanks for clarifying (I was initially confused by merging state files
&gt; rather than output files).
&gt;
&gt; &gt; At some point, Flink will definitely have some WAL adapter that can turn any sink into an exactly-once sink (with some caveats). For now, we keep that as an orthogonal solution as it has a rather high price (bursty workload with high latency). Ideally, we can keep the compaction asynchronously...
&gt;
&gt; Yes, that would be something like a WAL. I agree that it would have a
&gt; different set of trade-offs.
&gt;
&gt;
&gt; Regards,
&gt; Roman
&gt;
&gt; On Mon, Nov 29, 2021 at 3:33 PM Arvid Heise <arvid@apache.org&gt; wrote:
&gt; &gt;&gt;
&gt; &gt;&gt; &gt; One way to avoid write-read-merge is by wrapping SinkWriter with
&gt; &gt;&gt; &gt; another one, which would buffer input elements in a temporary storage
&gt; &gt;&gt; &gt; (e.g. local file) until a threshold is reached; after that, it would
&gt; &gt;&gt; &gt; invoke the original SinkWriter. And if a checkpoint barrier comes in
&gt; &gt;&gt; &gt; earlier, it would send written data to some aggregator.
&gt; &gt;&gt;
&gt; &gt;&gt; I think perhaps this seems to be a kind of WAL method? Namely we first
&gt; &gt;&gt; write the elements to some WAL logs and persist them on checkpoint
&gt; &gt;&gt; (in snapshot or remote FS), or we directly write WAL logs to the remote
&gt; &gt;&gt; FS eagerly.
&gt; &gt;&gt;
&gt; &gt; At some point, Flink will definitely have some WAL adapter that can turn any sink into an exactly-once sink (with some caveats). For now, we keep that as an orthogonal solution as it has a rather high price (bursty workload with high latency). Ideally, we can keep the compaction asynchronously...
&gt; &gt;
&gt; &gt; On Mon, Nov 29, 2021 at 8:52 AM Yun Gao <yungao.gy@aliyun.com.invalid&gt; wrote:
&gt; &gt;&gt;
&gt; &gt;&gt; Hi,
&gt; &gt;&gt;
&gt; &gt;&gt; @Roman very sorry for the late response for a long time,
&gt; &gt;&gt;
&gt; &gt;&gt; &gt; Merging artifacts from multiple checkpoints would apparently
&gt; &gt;&gt; require multiple concurrent checkpoints
&gt; &gt;&gt;
&gt; &gt;&gt; I think it might not need concurrent checkpoints: suppose some
&gt; &gt;&gt; operators (like the committer aggregator in the option 2) maintains
&gt; &gt;&gt; the list of files to merge, it could stores the lists of files to merge
&gt; &gt;&gt; in the states, then after several checkpoints are done and we have
&gt; &gt;&gt; enough files, we could merge all the files in the list.
&gt; &gt;&gt;
&gt; &gt;&gt; &gt; Asynchronous merging in an aggregator would require some resolution
&gt; &gt;&gt; &gt; logic on recovery, so that a merged artifact can be used if the
&gt; &gt;&gt; &gt; original one was deleted. Otherwise, wouldn't recovery fail because
&gt; &gt;&gt; &gt; some artifacts are missing?
&gt; &gt;&gt; &gt; We could also defer deletion until the "compacted" checkpoint is
&gt; &gt;&gt; &gt; subsumed - but isn't it too late, as it will be deleted anyways once
&gt; &gt;&gt; &gt; subsumed?
&gt; &gt;&gt;
&gt; &gt;&gt; I think logically we could delete the original files once the "compacted" checkpoint
&gt; &gt;&gt; (which finish merging the compacted files and record it in the checkpoint) is completed
&gt; &gt;&gt; in all the options. If there are failover before we it, we could restart the merging and if
&gt; &gt;&gt; there are failover after it, we could have already recorded the files in the checkpoint.
&gt; &gt;&gt;
&gt; &gt;&gt; &gt; One way to avoid write-read-merge is by wrapping SinkWriter with
&gt; &gt;&gt; &gt; another one, which would buffer input elements in a temporary storage
&gt; &gt;&gt; &gt; (e.g. local file) until a threshold is reached; after that, it would
&gt; &gt;&gt; &gt; invoke the original SinkWriter. And if a checkpoint barrier comes in
&gt; &gt;&gt; &gt; earlier, it would send written data to some aggregator.
&gt; &gt;&gt;
&gt; &gt;&gt; I think perhaps this seems to be a kind of WAL method? Namely we first
&gt; &gt;&gt; write the elements to some WAL logs and persist them on checkpoint
&gt; &gt;&gt; (in snapshot or remote FS), or we directly write WAL logs to the remote
&gt; &gt;&gt; FS eagerly.
&gt; &gt;&gt;
&gt; &gt;&gt; Sorry if I do not understand correctly somewhere.
&gt; &gt;&gt;
&gt; &gt;&gt; Best,
&gt; &gt;&gt; Yun
&gt; &gt;&gt;
&gt; &gt;&gt;
&gt; &gt;&gt; ------------------------------------------------------------------
&gt; &gt;&gt; From:Roman Khachatryan <roman@apache.org&gt;
&gt; &gt;&gt; Send Time:2021 Nov. 9 (Tue.) 22:03
&gt; &gt;&gt; To:dev <dev@flink.apache.org&gt;
&gt; &gt;&gt; Subject:Re: [DISCUSS] FLIP-191: Extend unified Sink interface to support small file compaction
&gt; &gt;&gt;
&gt; &gt;&gt; Hi everyone,
&gt; &gt;&gt;
&gt; &gt;&gt; Thanks for the proposal and the discussion, I have some remarks:
&gt; &gt;&gt; (I'm not very familiar with the new Sink API but I thought about the
&gt; &gt;&gt; same problem in context of the changelog state backend)
&gt; &gt;&gt;
&gt; &gt;&gt; 1. Merging artifacts from multiple checkpoints would apparently
&gt; &gt;&gt; require multiple concurrent checkpoints (otherwise, a new checkpoint
&gt; &gt;&gt; won't be started before completing the previous one; and the previous
&gt; &gt;&gt; one can't be completed before durably storing the artifacts). However,
&gt; &gt;&gt; concurrent checkpoints are currently not supported with Unaligned
&gt; &gt;&gt; checkpoints (this is besides increasing e2e-latency).
&gt; &gt;&gt;
&gt; &gt;&gt; 2. Asynchronous merging in an aggregator would require some resolution
&gt; &gt;&gt; logic on recovery, so that a merged artifact can be used if the
&gt; &gt;&gt; original one was deleted. Otherwise, wouldn't recovery fail because
&gt; &gt;&gt; some artifacts are missing?
&gt; &gt;&gt; We could also defer deletion until the "compacted" checkpoint is
&gt; &gt;&gt; subsumed - but isn't it too late, as it will be deleted anyways once
&gt; &gt;&gt; subsumed?
&gt; &gt;&gt;
&gt; &gt;&gt; 3. Writing small files, then reading and merging them for *every*
&gt; &gt;&gt; checkpoint seems worse than only reading them on recovery. I guess I'm
&gt; &gt;&gt; missing some cases of reading, so to me it would make sense to mention
&gt; &gt;&gt; these cases explicitly in the FLIP motivation section.
&gt; &gt;&gt;
&gt; &gt;&gt; 4. One way to avoid write-read-merge is by wrapping SinkWriter with
&gt; &gt;&gt; another one, which would buffer input elements in a temporary storage
&gt; &gt;&gt; (e.g. local file) until a threshold is reached; after that, it would
&gt; &gt;&gt; invoke the original SinkWriter. And if a checkpoint barrier comes in
&gt; &gt;&gt; earlier, it would send written data to some aggregator. It will
&gt; &gt;&gt; increase checkpoint delay (async phase) compared to the current Flink;
&gt; &gt;&gt; but not compared to the write-read-merge solution, IIUC.
&gt; &gt;&gt; Then such "BufferingSinkWriters" could aggregate input elements from
&gt; &gt;&gt; each other, potentially recursively (I mean something like
&gt; &gt;&gt; https://cwiki.apache.org/confluence/download/attachments/173082889/DSTL-DFS-DAG.png
&gt; &gt;&gt; )
&gt; &gt;&gt;
&gt; &gt;&gt; 5. Reducing the number of files by reducing aggregator parallelism as
&gt; &gt;&gt; opposed to merging on reaching size threshold will likely be less
&gt; &gt;&gt; optimal and more difficult to configure. OTH, thresholds might be more
&gt; &gt;&gt; difficult to implement and (with recursive merging) would incur higher
&gt; &gt;&gt; latency. Maybe that's also something to decide explicitly or at least
&gt; &gt;&gt; mention in the FLIP.
&gt; &gt;&gt;
&gt; &gt;&gt;
&gt; &gt;&gt;
&gt; &gt;&gt; Regards,
&gt; &gt;&gt; Roman
&gt; &gt;&gt;
&gt; &gt;&gt;
&gt; &gt;&gt; On Tue, Nov 9, 2021 at 5:23 AM Reo Lei <leinuowen@gmail.com&gt; wrote:
&gt; &gt;&gt; &gt;
&gt; &gt;&gt; &gt; Hi Fabian,
&gt; &gt;&gt; &gt;
&gt; &gt;&gt; &gt; Thanks for drafting the FLIP and trying to support small file compaction. I
&gt; &gt;&gt; &gt; think this feature is very urgent and valuable for users(at least for me).
&gt; &gt;&gt; &gt;
&gt; &gt;&gt; &gt; Currently I am trying to support streaming rewrite(compact) for Iceberg on
&gt; &gt;&gt; &gt; PR#3323 <https://github.com/apache/iceberg/pull/3323&gt;. As Steven mentioned,
&gt; &gt;&gt; &gt; Iceberg sink and compact data through the following steps:
&gt; &gt;&gt; &gt; Step-1: Some parallel data writer(sinker) to write streaming data as files.
&gt; &gt;&gt; &gt; Step-2: A single parallelism data files committer to commit the completed
&gt; &gt;&gt; &gt; files as soon as possible to make them available.
&gt; &gt;&gt; &gt; Step-3: Some parallel file rewriter(compactor) to collect committed files
&gt; &gt;&gt; &gt; from multiple checkpoints, and rewriter(compact) them together once the
&gt; &gt;&gt; &gt; total file size or number of files reach the threshold.
&gt; &gt;&gt; &gt; Step-4: A single parallelism rewrite(compact) result committer to commit
&gt; &gt;&gt; &gt; the rewritten(compacted) files to replace the old files and make them
&gt; &gt;&gt; &gt; available.
&gt; &gt;&gt; &gt;
&gt; &gt;&gt; &gt;
&gt; &gt;&gt; &gt; If Flink want to support small file compaction, some key point I think is
&gt; &gt;&gt; &gt; necessary:
&gt; &gt;&gt; &gt;
&gt; &gt;&gt; &gt; 1, Compact files from multiple checkpoints.
&gt; &gt;&gt; &gt; I totally agree with Jingsong, because completed file size usually could
&gt; &gt;&gt; &gt; not reach the threshold in a single checkpoint. Especially for partitioned
&gt; &gt;&gt; &gt; table, we need to compact the files of each partition, but usually the file
&gt; &gt;&gt; &gt; size of each partition will be different and may not reach the merge
&gt; &gt;&gt; &gt; threshold. If we compact these files, in a single checkpoint, regardless of
&gt; &gt;&gt; &gt; whether the total file size reaches the threshold, then the value of
&gt; &gt;&gt; &gt; compacting will be diminished and we will still get small files because
&gt; &gt;&gt; &gt; these compacted files are not reach to target size. So we need the
&gt; &gt;&gt; &gt; compactor to collect committed files from multiple checkpoints and compact
&gt; &gt;&gt; &gt; them until they reach the threshold.
&gt; &gt;&gt; &gt;
&gt; &gt;&gt; &gt; 2, Separate write phase and compact phase.
&gt; &gt;&gt; &gt; Users usually hope the data becomes available as soon as possible, and the
&gt; &gt;&gt; &gt;&nbsp; end-to-end latency is very important. I think we need to separate the
&gt; &gt;&gt; &gt; write and compact phase. For the write phase, there include the Step-1
&gt; &gt;&gt; &gt; and Step-2, we sink data as file and commit it pre checkpoint and regardless
&gt; &gt;&gt; &gt; of whether the file size it is. That could ensure the data will be
&gt; &gt;&gt; &gt; available ASAP. For the compact phase, there include the Step-3
&gt; &gt;&gt; &gt; and Step-4,&nbsp; the compactor should collect committed files from multiple
&gt; &gt;&gt; &gt; checkpoints and compact them asynchronously once they reach the threshold,
&gt; &gt;&gt; &gt; and the compact committer will commit the&nbsp; compaction result in the next
&gt; &gt;&gt; &gt; checkpoint. We compact the committed files asynchronously because we don't
&gt; &gt;&gt; &gt; want the compaction to affect the data sink or the whole pipeline.
&gt; &gt;&gt; &gt;
&gt; &gt;&gt; &gt; 3, Exactly once guarantee between write and compact phase.
&gt; &gt;&gt; &gt; Once we separate write phase and compact phase, we need to consider
&gt; &gt;&gt; &gt; how to guarantee
&gt; &gt;&gt; &gt; the exact once semantic between two phases. We should not lose any data or
&gt; &gt;&gt; &gt; files on the compactor(Step-3) in any case and cause the compaction result
&gt; &gt;&gt; &gt; to be inconsistent with before. I think flink should provide an easy-to-use
&gt; &gt;&gt; &gt; interface to make that easier.
&gt; &gt;&gt; &gt;
&gt; &gt;&gt; &gt; 4, Metadata operation and&nbsp; compaction result validation.
&gt; &gt;&gt; &gt; In the compact phase, there may be not only compact files, but also a lot
&gt; &gt;&gt; &gt; of metadata operations, such as the iceberg needing to read/write manifest
&gt; &gt;&gt; &gt; and do MOR. And we need some interface to support users to do some
&gt; &gt;&gt; &gt; validation of the compaction result. I think these points should be
&gt; &gt;&gt; &gt; considered when we design the compaction API.
&gt; &gt;&gt; &gt;
&gt; &gt;&gt; &gt;
&gt; &gt;&gt; &gt; Back to FLIP-191, option 1 looks very complicated while option 2 is
&gt; &gt;&gt; &gt; relatively simple, but neither of these two solutions separates the write
&gt; &gt;&gt; &gt; phase from the compact phase. So I think we should consider the points I
&gt; &gt;&gt; &gt; mentioned above. And if you have any other questions you can always feel
&gt; &gt;&gt; &gt; free to reach out to me!
&gt; &gt;&gt; &gt;
&gt; &gt;&gt; &gt; BR,
&gt; &gt;&gt; &gt; Reo
&gt; &gt;&gt; &gt;
&gt; &gt;&gt; &gt; Fabian Paul <fabianpaul@ververica.com&gt; 于2021年11月8日周一 下午7:59写道:
&gt; &gt;&gt; &gt;
&gt; &gt;&gt; &gt; &gt; Hi all,
&gt; &gt;&gt; &gt; &gt;
&gt; &gt;&gt; &gt; &gt; Thanks for the lively discussions. I am really excited to see so many
&gt; &gt;&gt; &gt; &gt; people
&gt; &gt;&gt; &gt; &gt; participating in this thread. It also underlines the need that many people
&gt; &gt;&gt; &gt; &gt; would
&gt; &gt;&gt; &gt; &gt; like to see a solution soon.
&gt; &gt;&gt; &gt; &gt;
&gt; &gt;&gt; &gt; &gt; I have updated the FLIP and removed the parallelism configuration because
&gt; &gt;&gt; &gt; &gt; it is
&gt; &gt;&gt; &gt; &gt; unnecessary since users can configure a constant exchange key to send all
&gt; &gt;&gt; &gt; &gt; committables to only one committable aggregator.
&gt; &gt;&gt; &gt; &gt;
&gt; &gt;&gt; &gt; &gt;
&gt; &gt;&gt; &gt; &gt; 1. Burden for developers w.r.t batch stream unification.
&gt; &gt;&gt; &gt; &gt;
&gt; &gt;&gt; &gt; &gt; @yun @guowei, from a theoretical point you are right about exposing the
&gt; &gt;&gt; &gt; &gt; DataStream
&gt; &gt;&gt; &gt; &gt; API in the sink users have the full power to write correct batch and
&gt; &gt;&gt; &gt; &gt; streaming
&gt; &gt;&gt; &gt; &gt; sinks. I think in reality a lot of users still struggle to build pipelines
&gt; &gt;&gt; &gt; &gt; with
&gt; &gt;&gt; &gt; &gt; i.e. the operator pipeline which works correct in streaming and batch mode.
&gt; &gt;&gt; &gt; &gt; Another problem I see is by exposing more deeper concepts is that we
&gt; &gt;&gt; &gt; &gt; cannot do
&gt; &gt;&gt; &gt; &gt; any optimization because we cannot reason about how sinks are built in the
&gt; &gt;&gt; &gt; &gt; future.
&gt; &gt;&gt; &gt; &gt;
&gt; &gt;&gt; &gt; &gt; We should also try to steer users towards using only `Functions` to give
&gt; &gt;&gt; &gt; &gt; us more
&gt; &gt;&gt; &gt; &gt; flexibility to swap the internal operator representation. I agree with
&gt; &gt;&gt; &gt; &gt; @yun we
&gt; &gt;&gt; &gt; &gt; should try to make the `ProcessFunction` more versatile to work on that
&gt; &gt;&gt; &gt; &gt; goal but
&gt; &gt;&gt; &gt; &gt; I see this as unrelated to the FLIP.
&gt; &gt;&gt; &gt; &gt;
&gt; &gt;&gt; &gt; &gt;
&gt; &gt;&gt; &gt; &gt; 2. Regarding Commit / Global commit
&gt; &gt;&gt; &gt; &gt;
&gt; &gt;&gt; &gt; &gt; I envision the global committer to be specific depending on the data lake
&gt; &gt;&gt; &gt; &gt; solution you want to write to. However, it is entirely orthogonal to the
&gt; &gt;&gt; &gt; &gt; compaction.
&gt; &gt;&gt; &gt; &gt; Currently, I do not expect any changes w.r.t the Global commit introduces
&gt; &gt;&gt; &gt; &gt; by
&gt; &gt;&gt; &gt; &gt; this FLIP.
&gt; &gt;&gt; &gt; &gt;
&gt; &gt;&gt; &gt; &gt;
&gt; &gt;&gt; &gt; &gt; 3. Regarding the case of trans-checkpoints merging
&gt; &gt;&gt; &gt; &gt;
&gt; &gt;&gt; &gt; &gt; @yun, as user, I would expect that if the committer receives in a
&gt; &gt;&gt; &gt; &gt; checkpoint files
&gt; &gt;&gt; &gt; &gt; to merge/commit that these are also finished when the checkpoint finishes.
&gt; &gt;&gt; &gt; &gt; I think all sinks rely on this principle currently i.e., KafkaSink needs to
&gt; &gt;&gt; &gt; &gt; commit all open transactions until the next checkpoint can happen.
&gt; &gt;&gt; &gt; &gt;
&gt; &gt;&gt; &gt; &gt; Maybe in the future, we can somehow move the Committer#commit call to an
&gt; &gt;&gt; &gt; &gt; asynchronous execution, but we should discuss it as a separate thread.
&gt; &gt;&gt; &gt; &gt;
&gt; &gt;&gt; &gt; &gt; &gt; We probably should first describe the different causes of small files and
&gt; &gt;&gt; &gt; &gt; &gt; what problems was this proposal trying to solve. I wrote a data shuffling
&gt; &gt;&gt; &gt; &gt; &gt; proposal [1] for Flink Iceberg sink (shared with Iceberg community [2]).
&gt; &gt;&gt; &gt; &gt; It
&gt; &gt;&gt; &gt; &gt; &gt; can address small files problems due to skewed data distribution across
&gt; &gt;&gt; &gt; &gt; &gt; Iceberg table partitions. Streaming shuffling before writers (to files)
&gt; &gt;&gt; &gt; &gt; is
&gt; &gt;&gt; &gt; &gt; &gt; typically more efficient than post-write file compaction (which involves
&gt; &gt;&gt; &gt; &gt; &gt; read-merge-write). It is usually cheaper to prevent a problem (small
&gt; &gt;&gt; &gt; &gt; files)
&gt; &gt;&gt; &gt; &gt; &gt; than fixing it.
&gt; &gt;&gt; &gt; &gt;
&gt; &gt;&gt; &gt; &gt;
&gt; &gt;&gt; &gt; &gt; @steven you are raising a good point, although I think only using a
&gt; &gt;&gt; &gt; &gt; customizable
&gt; &gt;&gt; &gt; &gt; shuffle won't address the generation of small files. One assumption is that
&gt; &gt;&gt; &gt; &gt; at least the sink generates one file per subtask, which can already be too
&gt; &gt;&gt; &gt; &gt; many.
&gt; &gt;&gt; &gt; &gt; Another problem is that with low checkpointing intervals, the files do not
&gt; &gt;&gt; &gt; &gt; meet
&gt; &gt;&gt; &gt; &gt; the required size. The latter point is probably addressable by changing the
&gt; &gt;&gt; &gt; &gt; checkpoint interval, which might be inconvenient for some users.
&gt; &gt;&gt; &gt; &gt;
&gt; &gt;&gt; &gt; &gt; &gt; The sink coordinator checkpoint problem (mentioned in option 1) would be
&gt; &gt;&gt; &gt; &gt; &gt; great if Flink can address it. In the spirit of source
&gt; &gt;&gt; &gt; &gt; (enumerator-reader)
&gt; &gt;&gt; &gt; &gt; &gt; and sink (writer-coordinator) duality, sink coordinator checkpoint should
&gt; &gt;&gt; &gt; &gt; &gt; happen after the writer operator. This would be a natural fit to support
&gt; &gt;&gt; &gt; &gt; &gt; global committer in FLIP-143. It is probably an orthogonal matter to this
&gt; &gt;&gt; &gt; &gt; &gt; proposal.
&gt; &gt;&gt; &gt; &gt;
&gt; &gt;&gt; &gt; &gt;
&gt; &gt;&gt; &gt; &gt; To me the question here is what are the benefits of having a coordinator in
&gt; &gt;&gt; &gt; &gt; comparison to a global committer/aggregator operator.
&gt; &gt;&gt; &gt; &gt;
&gt; &gt;&gt; &gt; &gt; &gt; Personally, I am usually in favor of keeping streaming ingestion (to data
&gt; &gt;&gt; &gt; &gt; &gt; lake) relatively simple and stable. Also sometimes compaction and sorting
&gt; &gt;&gt; &gt; &gt; &gt; are performed together in data rewrite maintenance jobs to improve read
&gt; &gt;&gt; &gt; &gt; &gt; performance. In that case, the value of compacting (in Flink streaming
&gt; &gt;&gt; &gt; &gt; &gt; ingestion) diminishes.
&gt; &gt;&gt; &gt; &gt;
&gt; &gt;&gt; &gt; &gt;
&gt; &gt;&gt; &gt; &gt; I agree it is always possible to have scheduled maintenance jobs keeping
&gt; &gt;&gt; &gt; &gt; care of
&gt; &gt;&gt; &gt; &gt; your data i.e., doing compaction. Unfortunately, the downside is that you
&gt; &gt;&gt; &gt; &gt; have to your data after it is already available for other downstream
&gt; &gt;&gt; &gt; &gt; consumers.
&gt; &gt;&gt; &gt; &gt; I guess this can lead to all kinds of visibility problems. I am also
&gt; &gt;&gt; &gt; &gt; surprised that
&gt; &gt;&gt; &gt; &gt; you personally are a fan of this approach and, on the other hand, are
&gt; &gt;&gt; &gt; &gt; developing
&gt; &gt;&gt; &gt; &gt; the Iceberg sink, which goes somewhat against your mentioned principle of
&gt; &gt;&gt; &gt; &gt; keeping
&gt; &gt;&gt; &gt; &gt; the sink simple.
&gt; &gt;&gt; &gt; &gt;
&gt; &gt;&gt; &gt; &gt; &gt; Currently, it is unclear from the doc and this thread where the
&gt; &gt;&gt; &gt; &gt; compaction
&gt; &gt;&gt; &gt; &gt; &gt; is actually happening. Jingsong's reply described one model
&gt; &gt;&gt; &gt; &gt; &gt; writer (parallel) -&gt; aggregator (single-parallelism compaction planner)
&gt; &gt;&gt; &gt; &gt; -&gt;
&gt; &gt;&gt; &gt; &gt; &gt; compactor (parallel) -&gt; global committer (single-parallelism)
&gt; &gt;&gt; &gt; &gt;
&gt; &gt;&gt; &gt; &gt;
&gt; &gt;&gt; &gt; &gt; My idea of the topology is very similar to the one outlined by Jinsong. The
&gt; &gt;&gt; &gt; &gt; compaction will happen in the committer operator.
&gt; &gt;&gt; &gt; &gt;
&gt; &gt;&gt; &gt; &gt; &gt;
&gt; &gt;&gt; &gt; &gt; &gt; In the Iceberg community, the following model has been discussed. It is
&gt; &gt;&gt; &gt; &gt; &gt; better for Iceberg because it won't delay the data availability.
&gt; &gt;&gt; &gt; &gt; &gt; writer (parallel) -&gt; global committer for append (single parallelism) -&gt;
&gt; &gt;&gt; &gt; &gt; &gt; compactor (parallel) -&gt; global committer for rewrite commit (single
&gt; &gt;&gt; &gt; &gt; &gt; parallelism)
&gt; &gt;&gt; &gt; &gt;
&gt; &gt;&gt; &gt; &gt;
&gt; &gt;&gt; &gt; &gt; From a quick glimpse, it seems that the exact same topology is possible to
&gt; &gt;&gt; &gt; &gt; express with the committable aggregator, but this definitely depends on
&gt; &gt;&gt; &gt; &gt; the exact
&gt; &gt;&gt; &gt; &gt; setup.
&gt; &gt;&gt; &gt; &gt;
&gt; &gt;&gt; &gt; &gt; Best,
&gt; &gt;&gt; &gt; &gt; Fabian
&gt; &gt;&gt;

Re: Re: [DISCUSS] FLIP-191: Extend unified Sink interface to support small file compaction

Posted by Arvid Heise <ar...@apache.org>.
I just noticed offline with Yun that I have some misconception on how how
blocking data exchange work in Flink. Apparently, a subtask of the stage 2
is already started when all connected subtasks of stage 1 are finished. I
was assuming that it works similarly to Hadoop where all tasks of stage 1
have to be finished before any task of stage 2 is started.
So to amend the proposal, we would introduce a full shuffle before
committer in batch. That would simulate the behavior of Hadoop.

We also need to verify what happens in case of multiple sinks. In that
case, we may have started committer of sink 1 while sink 2 is still
writing. That would be fine as long as sink 1 writers are not restarted. If
not then it seems as if we may improve scheduler to actually not restart
"finished" tasks. It seems like an orthogonal issue though.

Note that all discussed issues apply to Sink V1 as well, so the FLIP is not
making anything worse.


On Thu, Dec 16, 2021 at 1:53 PM Arvid Heise <ar...@apache.org> wrote:

> Hi Yun,
>
> the basic idea is to use different regions for the different stages.
>
> So you have your stages:
> Stage 1: <job> -> pre-writer topology -> writer -> committables
> (materialized)
> Stage 2: committables (materialized) -> pre-commit topology -> committer
> -> succesful committables (materialized)
> Stage 3: succesful committables (materialized) -> post-commit topology
>
> If any instance of a given stages fails, the whole stage is restarted.
> So, in case of the main pipeline (stage 1) fails, no data will be
> committed at all. On a rerun, we start fresh (or from the previous stage).
> Only, when all data has been written, we start with committing the data.
> An issue during committing, will retrigger the commit stage (stage 2) and
> only that stage. Thus, all committables are stable and remain stable.
> When we are done committing all committables, we run the post-commit
> topology similarly "atomically".
>
> So now the cavaets:
> - If committer is rerun, all committables are restarted. So the committer
> needs to be idempotent. That is the same with STREAMING mode now and afaik
> there is no way around it.
> - If we lose a TM during commit phase, we will run into the original
> issues of inconstent data as we need to rerun the whole job. That would be
> solved with HA storage and we haven't found any solution that doesn't
> require some kind of external storage. However, the commit phase should be
> rather fast and errors are improbable (low volume).
>
> I'd still like to have an HA storage but that issue is also in Sink V1 and
> kind of orthogonal. It's also nothing that we can solve without involving
> more folks (I'm trying to kick start that in the background).
>
> On Thu, Dec 16, 2021 at 1:31 PM Yun Gao <yu...@aliyun.com> wrote:
>
>> Hi,
>>
>> Very thanks Fabian for the explanation and it solves most of the issues.
>> There is one left issue I want to have a double confirmation is that for
>> the edges between writer and committer and in the post-committer
>> topology,
>> perhaps the result partition with HA storage is not enough solve all the
>> issues
>> directly ? It is due to after the committer and post-committer topology
>> is finished
>> and the data is committed, it might still be restarted due to JM failover
>> and the
>> deterministic problem (namely the example of  (A -> [B1, B2], A, B1 have
>> finished and
>> B2 failed, if -> is rebalance / random / rescale,  all of A, B1, B2 would
>> restarted). Then
>> the records would be produced and created for the second times.
>>
>> We might let the writers to skip producing the new records, but if we
>> have multiple sinks like
>> OP1 -> (writer 1 -> committer 1)
>>      |--> (writer 2 -> committer 2)
>>
>> and the failover happens after writer 1 & committer 1 get finished but
>> writer 2 is running,
>> if op1 produced different records across the two runs, then the two sinks
>> would produces
>> different data, which might be not suitable in some cases. Perhaps we
>> need some support
>> from the scheduler side?
>>
>> But I also agree this could be a separate issue and we could solve it separately
>> in some future
>> as long as we know how to solve it~
>>
>> Best,
>> Yun
>>
>>
>> ------------------------------------------------------------------
>> From:Arvid Heise <ar...@apache.org>
>> Send Time:2021 Dec. 16 (Thu.) 19:54
>> To:dev <de...@flink.apache.org>
>> Cc:Yun Gao <yu...@aliyun.com>
>> Subject:Re: Re: [DISCUSS] FLIP-191: Extend unified Sink interface to
>> support small file compaction
>>
>> Just a quick amend: There will be no blocking exchange in the pre-writer
>> exchange for performance reasons.
>> After the writer, we have tiny data volume and are free to add as many as
>> we see necessary.
>>
>> On Thu, Dec 16, 2021 at 11:18 AM Fabian Paul <fp...@apache.org> wrote:
>> Hi Till,
>>
>> Good point, in the scenario with the blocking keyed exchange between
>> the writer and committer my idea is to make the committer effectively
>> the global committer. With Sink V2 there is no real difference anymore
>> between the committer and global committer.
>> You are right that everything after the committer would be part of the
>> same failover region but we plan to insert a blocking exchange by
>> default before all of the custom topologies.
>>
>> Best,
>> Fabian
>>
>> On Thu, Dec 16, 2021 at 11:08 AM Till Rohrmann <tr...@apache.org>
>> wrote:
>> >
>> > Hi Fabian,
>> >
>> > quick question on your comment 3. If there is a pipelined data exchange
>> > with a keyBy between the writers/committers and the component that does
>> the
>> > global commit, then there will only be a single failover region. So is
>> it
>> > correct that you assumed blocking data exchanges for the scenario you
>> > described?
>> >
>> > Cheers,
>> > Till
>> >
>> > On Thu, Dec 16, 2021 at 9:23 AM Fabian Paul <fp...@apache.org> wrote:
>> >
>> > > Hi Yun,
>> > >
>> > > Thanks for your fast feedback. Let me clarify your points.
>> > >
>> > > 1. We solve it by using StreamExchangeMode.BATCH before any exchange.
>> > > That obviously doesn’t help with lost TM but we would need to employ
>> > > HA storage for that. Same issue as now and orthogonal.
>> > >
>> > > 2. Extending V1 with V2 or vice versa would require renames of methods
>> > > (since return types are non-optional) and is making API changes. Even
>> > > though Experimental, we want to give connector developers the
>> > > opportunity to provide 1 implementation for all of Flink 1.X. We will
>> > > offer an internal adapter from V1 to V2, 2 sinkTo , and internally
>> > > just have one code-path.
>> > >
>> > > 3. DataStreamSink would act as a unified view on all the operators and
>> > > update them all at once when using setParallelism and so on (setName
>> > > and setUid will receive suffixes per operator).
>> > > Iceberg actually has a different requirement: They want to have a
>> > > committer with parallelism 1 but as a coordinator such that
>> > > embarrassingly parallel pipelines have different fail-over regions. I
>> > > was thinking that in this case, they need to implement a no-op
>> > > committer (that just forwards the committables) and use a post-commit
>> > > topology that achieves that.
>> > > Another option is that they use the preCommit topology and insert a
>> > > constant key-by that forwards all committables to a single committer.
>> > > We are planning to provide building blocks for such pipelines as we
>> > > go.
>> > >
>> > > Best,
>> > > Fabian
>> > >
>> > > On Thu, Dec 16, 2021 at 5:50 AM Yun Gao <yu...@aliyun.com> wrote:
>> > > >
>> > > > Hi Fabian,
>> > > >
>> > > > Very thanks for the update! I think the latest version in general
>> looks
>> > > good from my side
>> > > > and I think using separate feature interface would be much more
>> easy to
>> > > understand
>> > > > and extend in the future. I have some pending issues on the details
>> > > though:
>> > > >
>> > > > 1. The first one is if we could support end-to-end exactly-once with
>> > > post-committing
>> > > > topology in the batch mode ? Since for the batch mode, currently we
>> > > could only commit
>> > > >  all the transactions after the whole job is finished, otherwise if
>> > > there are JM failover or the
>> > > > writer / committer get restarted due to indeterminstic (A -> [B1,
>> B2],
>> > > A, B1 have finished and
>> > > >  B2 failed, if -> is rebalance / random / rescale,  all of A, B1, B2
>> > > would restarted), there might
>> > > > be repeat records. Previously one possible thought is to move
>> committer
>> > > and global committer
>> > > >  to the operator coordinator, but if it is a topology, we might need
>> > > some other kind of solutions?
>> > > >
>> > > > 2. I also want to have a dobule confirmation with the compatibility:
>> > > since the old sink is also named
>> > > > with Sink, do we want to put the Sink v2 in a new package ? Besides,
>> > > since we might want to keep
>> > > > only have one `sinkTo(Sink<?> sink)` , perhaps we also need to make
>> the
>> > > Sink v1 to be a subclass of
>> > > > Sink v2 and extends the stateful and two-phase-commit sinks, right?
>> > > >
>> > > > 3. I'd like also have a confirmation on ours thoughts with the
>> > > `DataStreamSink` returned by the sinkTo method:
>> > > > The main issue is how do we implement the method like
>> `setParallelism`
>> > > or `setMaxParallelism` since now the sink
>> > > > would be translated to multiple transformations? perhaps we could
>> make
>> > > it the default values for all the transformations
>> > > > for the sink? A related issue would be for iceberg sink, I think it
>> > > would need to have only one committer to avoid the
>> > > > competition of the optimistic locks (which would cause performance
>> > > degradation), then it might need to have N writers
>> > > > with 1 committers, to build such topology, perhaps we might need to
>> add
>> > > new methods to specify the parallelism of
>> > > > the writers and committers separately?
>> > > >
>> > > > Best,
>> > > > Yun
>> > > >
>> > > >
>> > > > ------------------Original Mail ------------------
>> > > > Sender:Fabian Paul <fp...@apache.org>
>> > > > Send Date:Mon Dec 13 23:59:43 2021
>> > > > Recipients:dev <de...@flink.apache.org>
>> > > > Subject:Re: [DISCUSS] FLIP-191: Extend unified Sink interface to
>> support
>> > > small file compaction
>> > > >>
>> > > >> Hi all,
>> > > >>
>> > > >>
>> > > >>
>> > > >> After a lot of discussions with different, we received very
>> fruitful
>> > > >>
>> > > >> feedback and reworked the ideas behind this FLIP. Initially, we had
>> > > >>
>> > > >> the impression that the compaction problem is solvable by a single
>> > > >>
>> > > >> topology that we can reuse across different sinks. We now have a
>> > > >>
>> > > >> better understanding that different external systems require
>> different
>> > > >>
>> > > >> compaction mechanism i.e. Hive requires compaction before finally
>> > > >>
>> > > >> registering the file in the metastore or Iceberg compacts the files
>> > > >>
>> > > >> after they have been registered and just lazily compacts them.
>> > > >>
>> > > >>
>> > > >>
>> > > >> Considering all these different views we came up with a design that
>> > > >>
>> > > >> builds upon what @guowei.mgw@gmail.com and @yungao.gy@aliyun.com
>> have
>> > > >>
>> > > >> proposed at the beginning. We allow inserting custom topologies
>> before
>> > > >>
>> > > >> and after the SinkWriters and Committers. Furthermore, we do not
>> see
>> > > >>
>> > > >> it as a downside. The Sink interfaces that will expose the
>> DataStream
>> > > >>
>> > > >> to the user reside in flink-streaming-java in contrast to the basic
>> > > >>
>> > > >> Sink interfaces that reside fin flink-core deem it to be only used
>> by
>> > > >>
>> > > >> expert users.
>> > > >>
>> > > >>
>> > > >>
>> > > >> Moreover, we also wanted to remove the global committer from the
>> > > >>
>> > > >> unified Sink interfaces and replace it with a custom post-commit
>> > > >>
>> > > >> topology. Unfortunately, we cannot do it without breaking the Sink
>> > > >>
>> > > >> interface since the GlobalCommittables are part of the
>> parameterized
>> > > >>
>> > > >> Sink interface. Thus, we propose building a new Sink V2 interface
>> > > >>
>> > > >> consisting of composable interfaces that do not offer the
>> > > >>
>> > > >> GlobalCommitter anymore. We will implement a utility to extend a
>> Sink
>> > > >>
>> > > >> with post topology that mimics the behavior of the GlobalCommitter.
>> > > >>
>> > > >> The new Sink V2 provides the same sort of methods as the Sink V1
>> > > >>
>> > > >> interface, so a migration of sinks that do not use the
>> GlobalCommitter
>> > > >>
>> > > >> should be very easy.
>> > > >>
>> > > >> We plan to keep the existing Sink V1 interfaces to not break
>> > > >>
>> > > >> externally built sinks. As part of this FLIP, we migrate all the
>> > > >>
>> > > >> connectors inside of the main repository to the new Sink V2 API.
>> > > >>
>> > > >>
>> > > >>
>> > > >> The FLIP document is also updated and includes the proposed
>> changes.
>> > > >>
>> > > >>
>> > > >>
>> > > >> Looking forward to your feedback,
>> > > >>
>> > > >> Fabian
>> > > >>
>> > > >>
>> > > >>
>> > > >>
>> > >
>> https://cwiki.apache.org/confluence/display/FLINK/FLIP-191%3A+Extend+unified+Sink+interface+to+support+small+file+compaction
>> > > >>
>> > > >>
>> > > >>
>> > > >>
>> > > >>
>> > > >> On Thu, Dec 2, 2021 at 10:15 AM Roman Khachatryan wrote:
>> > > >>
>> > > >> >
>> > > >>
>> > > >> > Thanks for clarifying (I was initially confused by merging state
>> files
>> > > >>
>> > > >> > rather than output files).
>> > > >>
>> > > >> >
>> > > >>
>> > > >> > > At some point, Flink will definitely have some WAL adapter
>> that can
>> > > turn any sink into an exactly-once sink (with some caveats). For now,
>> we
>> > > keep that as an orthogonal solution as it has a rather high price
>> (bursty
>> > > workload with high latency). Ideally, we can keep the compaction
>> > > asynchronously...
>> > > >>
>> > > >> >
>> > > >>
>> > > >> > Yes, that would be something like a WAL. I agree that it would
>> have a
>> > > >>
>> > > >> > different set of trade-offs.
>> > > >>
>> > > >> >
>> > > >>
>> > > >> >
>> > > >>
>> > > >> > Regards,
>> > > >>
>> > > >> > Roman
>> > > >>
>> > > >> >
>> > > >>
>> > > >> > On Mon, Nov 29, 2021 at 3:33 PM Arvid Heise wrote:
>> > > >>
>> > > >> > >>
>> > > >>
>> > > >> > >> > One way to avoid write-read-merge is by wrapping SinkWriter
>> with
>> > > >>
>> > > >> > >> > another one, which would buffer input elements in a
>> temporary
>> > > storage
>> > > >>
>> > > >> > >> > (e.g. local file) until a threshold is reached; after that,
>> it
>> > > would
>> > > >>
>> > > >> > >> > invoke the original SinkWriter. And if a checkpoint barrier
>> > > comes in
>> > > >>
>> > > >> > >> > earlier, it would send written data to some aggregator.
>> > > >>
>> > > >> > >>
>> > > >>
>> > > >> > >> I think perhaps this seems to be a kind of WAL method? Namely
>> we
>> > > first
>> > > >>
>> > > >> > >> write the elements to some WAL logs and persist them on
>> checkpoint
>> > > >>
>> > > >> > >> (in snapshot or remote FS), or we directly write WAL logs to
>> the
>> > > remote
>> > > >>
>> > > >> > >> FS eagerly.
>> > > >>
>> > > >> > >>
>> > > >>
>> > > >> > > At some point, Flink will definitely have some WAL adapter
>> that can
>> > > turn any sink into an exactly-once sink (with some caveats). For now,
>> we
>> > > keep that as an orthogonal solution as it has a rather high price
>> (bursty
>> > > workload with high latency). Ideally, we can keep the compaction
>> > > asynchronously...
>> > > >>
>> > > >> > >
>> > > >>
>> > > >> > > On Mon, Nov 29, 2021 at 8:52 AM Yun Gao wrote:
>> > > >>
>> > > >> > >>
>> > > >>
>> > > >> > >> Hi,
>> > > >>
>> > > >> > >>
>> > > >>
>> > > >> > >> @Roman very sorry for the late response for a long time,
>> > > >>
>> > > >> > >>
>> > > >>
>> > > >> > >> > Merging artifacts from multiple checkpoints would apparently
>> > > >>
>> > > >> > >> require multiple concurrent checkpoints
>> > > >>
>> > > >> > >>
>> > > >>
>> > > >> > >> I think it might not need concurrent checkpoints: suppose some
>> > > >>
>> > > >> > >> operators (like the committer aggregator in the option 2)
>> maintains
>> > > >>
>> > > >> > >> the list of files to merge, it could stores the lists of
>> files to
>> > > merge
>> > > >>
>> > > >> > >> in the states, then after several checkpoints are done and we
>> have
>> > > >>
>> > > >> > >> enough files, we could merge all the files in the list.
>> > > >>
>> > > >> > >>
>> > > >>
>> > > >> > >> > Asynchronous merging in an aggregator would require some
>> > > resolution
>> > > >>
>> > > >> > >> > logic on recovery, so that a merged artifact can be used if
>> the
>> > > >>
>> > > >> > >> > original one was deleted. Otherwise, wouldn't recovery fail
>> > > because
>> > > >>
>> > > >> > >> > some artifacts are missing?
>> > > >>
>> > > >> > >> > We could also defer deletion until the "compacted"
>> checkpoint is
>> > > >>
>> > > >> > >> > subsumed - but isn't it too late, as it will be deleted
>> anyways
>> > > once
>> > > >>
>> > > >> > >> > subsumed?
>> > > >>
>> > > >> > >>
>> > > >>
>> > > >> > >> I think logically we could delete the original files once the
>> > > "compacted" checkpoint
>> > > >>
>> > > >> > >> (which finish merging the compacted files and record it in the
>> > > checkpoint) is completed
>> > > >>
>> > > >> > >> in all the options. If there are failover before we it, we
>> could
>> > > restart the merging and if
>> > > >>
>> > > >> > >> there are failover after it, we could have already recorded
>> the
>> > > files in the checkpoint.
>> > > >>
>> > > >> > >>
>> > > >>
>> > > >> > >> > One way to avoid write-read-merge is by wrapping SinkWriter
>> with
>> > > >>
>> > > >> > >> > another one, which would buffer input elements in a
>> temporary
>> > > storage
>> > > >>
>> > > >> > >> > (e.g. local file) until a threshold is reached; after that,
>> it
>> > > would
>> > > >>
>> > > >> > >> > invoke the original SinkWriter. And if a checkpoint barrier
>> > > comes in
>> > > >>
>> > > >> > >> > earlier, it would send written data to some aggregator.
>> > > >>
>> > > >> > >>
>> > > >>
>> > > >> > >> I think perhaps this seems to be a kind of WAL method? Namely
>> we
>> > > first
>> > > >>
>> > > >> > >> write the elements to some WAL logs and persist them on
>> checkpoint
>> > > >>
>> > > >> > >> (in snapshot or remote FS), or we directly write WAL logs to
>> the
>> > > remote
>> > > >>
>> > > >> > >> FS eagerly.
>> > > >>
>> > > >> > >>
>> > > >>
>> > > >> > >> Sorry if I do not understand correctly somewhere.
>> > > >>
>> > > >> > >>
>> > > >>
>> > > >> > >> Best,
>> > > >>
>> > > >> > >> Yun
>> > > >>
>> > > >> > >>
>> > > >>
>> > > >> > >>
>> > > >>
>> > > >> > >>
>> ------------------------------------------------------------------
>> > > >>
>> > > >> > >> From:Roman Khachatryan
>> > > >>
>> > > >> > >> Send Time:2021 Nov. 9 (Tue.) 22:03
>> > > >>
>> > > >> > >> To:dev
>> > > >>
>> > > >> > >> Subject:Re: [DISCUSS] FLIP-191: Extend unified Sink interface
>> to
>> > > support small file compaction
>> > > >>
>> > > >> > >>
>> > > >>
>> > > >> > >> Hi everyone,
>> > > >>
>> > > >> > >>
>> > > >>
>> > > >> > >> Thanks for the proposal and the discussion, I have some
>> remarks:
>> > > >>
>> > > >> > >> (I'm not very familiar with the new Sink API but I thought
>> about
>> > > the
>> > > >>
>> > > >> > >> same problem in context of the changelog state backend)
>> > > >>
>> > > >> > >>
>> > > >>
>> > > >> > >> 1. Merging artifacts from multiple checkpoints would
>> apparently
>> > > >>
>> > > >> > >> require multiple concurrent checkpoints (otherwise, a new
>> > > checkpoint
>> > > >>
>> > > >> > >> won't be started before completing the previous one; and the
>> > > previous
>> > > >>
>> > > >> > >> one can't be completed before durably storing the artifacts).
>> > > However,
>> > > >>
>> > > >> > >> concurrent checkpoints are currently not supported with
>> Unaligned
>> > > >>
>> > > >> > >> checkpoints (this is besides increasing e2e-latency).
>> > > >>
>> > > >> > >>
>> > > >>
>> > > >> > >> 2. Asynchronous merging in an aggregator would require some
>> > > resolution
>> > > >>
>> > > >> > >> logic on recovery, so that a merged artifact can be used if
>> the
>> > > >>
>> > > >> > >> original one was deleted. Otherwise, wouldn't recovery fail
>> because
>> > > >>
>> > > >> > >> some artifacts are missing?
>> > > >>
>> > > >> > >> We could also defer deletion until the "compacted" checkpoint
>> is
>> > > >>
>> > > >> > >> subsumed - but isn't it too late, as it will be deleted
>> anyways
>> > > once
>> > > >>
>> > > >> > >> subsumed?
>> > > >>
>> > > >> > >>
>> > > >>
>> > > >> > >> 3. Writing small files, then reading and merging them for
>> *every*
>> > > >>
>> > > >> > >> checkpoint seems worse than only reading them on recovery. I
>> guess
>> > > I'm
>> > > >>
>> > > >> > >> missing some cases of reading, so to me it would make sense to
>> > > mention
>> > > >>
>> > > >> > >> these cases explicitly in the FLIP motivation section.
>> > > >>
>> > > >> > >>
>> > > >>
>> > > >> > >> 4. One way to avoid write-read-merge is by wrapping
>> SinkWriter with
>> > > >>
>> > > >> > >> another one, which would buffer input elements in a temporary
>> > > storage
>> > > >>
>> > > >> > >> (e.g. local file) until a threshold is reached; after that, it
>> > > would
>> > > >>
>> > > >> > >> invoke the original SinkWriter. And if a checkpoint barrier
>> comes
>> > > in
>> > > >>
>> > > >> > >> earlier, it would send written data to some aggregator. It
>> will
>> > > >>
>> > > >> > >> increase checkpoint delay (async phase) compared to the
>> current
>> > > Flink;
>> > > >>
>> > > >> > >> but not compared to the write-read-merge solution, IIUC.
>> > > >>
>> > > >> > >> Then such "BufferingSinkWriters" could aggregate input
>> elements
>> > > from
>> > > >>
>> > > >> > >> each other, potentially recursively (I mean something like
>> > > >>
>> > > >> > >>
>> > >
>> https://cwiki.apache.org/confluence/download/attachments/173082889/DSTL-DFS-DAG.png
>> > > >>
>> > > >> > >> )
>> > > >>
>> > > >> > >>
>> > > >>
>> > > >> > >> 5. Reducing the number of files by reducing aggregator
>> parallelism
>> > > as
>> > > >>
>> > > >> > >> opposed to merging on reaching size threshold will likely be
>> less
>> > > >>
>> > > >> > >> optimal and more difficult to configure. OTH, thresholds
>> might be
>> > > more
>> > > >>
>> > > >> > >> difficult to implement and (with recursive merging) would
>> incur
>> > > higher
>> > > >>
>> > > >> > >> latency. Maybe that's also something to decide explicitly or
>> at
>> > > least
>> > > >>
>> > > >> > >> mention in the FLIP.
>> > > >>
>> > > >> > >>
>> > > >>
>> > > >> > >>
>> > > >>
>> > > >> > >>
>> > > >>
>> > > >> > >> Regards,
>> > > >>
>> > > >> > >> Roman
>> > > >>
>> > > >> > >>
>> > > >>
>> > > >> > >>
>> > > >>
>> > > >> > >> On Tue, Nov 9, 2021 at 5:23 AM Reo Lei wrote:
>> > > >>
>> > > >> > >> >
>> > > >>
>> > > >> > >> > Hi Fabian,
>> > > >>
>> > > >> > >> >
>> > > >>
>> > > >> > >> > Thanks for drafting the FLIP and trying to support small
>> file
>> > > compaction. I
>> > > >>
>> > > >> > >> > think this feature is very urgent and valuable for users(at
>> > > least for me).
>> > > >>
>> > > >> > >> >
>> > > >>
>> > > >> > >> > Currently I am trying to support streaming rewrite(compact)
>> for
>> > > Iceberg on
>> > > >>
>> > > >> > >> > PR#3323 . As Steven mentioned,
>> > > >>
>> > > >> > >> > Iceberg sink and compact data through the following steps:
>> > > >>
>> > > >> > >> > Step-1: Some parallel data writer(sinker) to write streaming
>> > > data as files.
>> > > >>
>> > > >> > >> > Step-2: A single parallelism data files committer to commit
>> the
>> > > completed
>> > > >>
>> > > >> > >> > files as soon as possible to make them available.
>> > > >>
>> > > >> > >> > Step-3: Some parallel file rewriter(compactor) to collect
>> > > committed files
>> > > >>
>> > > >> > >> > from multiple checkpoints, and rewriter(compact) them
>> together
>> > > once the
>> > > >>
>> > > >> > >> > total file size or number of files reach the threshold.
>> > > >>
>> > > >> > >> > Step-4: A single parallelism rewrite(compact) result
>> committer
>> > > to commit
>> > > >>
>> > > >> > >> > the rewritten(compacted) files to replace the old files and
>> make
>> > > them
>> > > >>
>> > > >> > >> > available.
>> > > >>
>> > > >> > >> >
>> > > >>
>> > > >> > >> >
>> > > >>
>> > > >> > >> > If Flink want to support small file compaction, some key
>> point I
>> > > think is
>> > > >>
>> > > >> > >> > necessary:
>> > > >>
>> > > >> > >> >
>> > > >>
>> > > >> > >> > 1, Compact files from multiple checkpoints.
>> > > >>
>> > > >> > >> > I totally agree with Jingsong, because completed file size
>> > > usually could
>> > > >>
>> > > >> > >> > not reach the threshold in a single checkpoint. Especially
>> for
>> > > partitioned
>> > > >>
>> > > >> > >> > table, we need to compact the files of each partition, but
>> > > usually the file
>> > > >>
>> > > >> > >> > size of each partition will be different and may not reach
>> the
>> > > merge
>> > > >>
>> > > >> > >> > threshold. If we compact these files, in a single
>> checkpoint,
>> > > regardless of
>> > > >>
>> > > >> > >> > whether the total file size reaches the threshold, then the
>> > > value of
>> > > >>
>> > > >> > >> > compacting will be diminished and we will still get small
>> files
>> > > because
>> > > >>
>> > > >> > >> > these compacted files are not reach to target size. So we
>> need
>> > > the
>> > > >>
>> > > >> > >> > compactor to collect committed files from multiple
>> checkpoints
>> > > and compact
>> > > >>
>> > > >> > >> > them until they reach the threshold.
>> > > >>
>> > > >> > >> >
>> > > >>
>> > > >> > >> > 2, Separate write phase and compact phase.
>> > > >>
>> > > >> > >> > Users usually hope the data becomes available as soon as
>> > > possible, and the
>> > > >>
>> > > >> > >> > end-to-end latency is very important. I think we need to
>> > > separate the
>> > > >>
>> > > >> > >> > write and compact phase. For the write phase, there include
>> the
>> > > Step-1
>> > > >>
>> > > >> > >> > and Step-2, we sink data as file and commit it pre
>> checkpoint
>> > > and regardless
>> > > >>
>> > > >> > >> > of whether the file size it is. That could ensure the data
>> will
>> > > be
>> > > >>
>> > > >> > >> > available ASAP. For the compact phase, there include the
>> Step-3
>> > > >>
>> > > >> > >> > and Step-4, the compactor should collect committed files
>> from
>> > > multiple
>> > > >>
>> > > >> > >> > checkpoints and compact them asynchronously once they reach
>> the
>> > > threshold,
>> > > >>
>> > > >> > >> > and the compact committer will commit the compaction result
>> in
>> > > the next
>> > > >>
>> > > >> > >> > checkpoint. We compact the committed files asynchronously
>> > > because we don't
>> > > >>
>> > > >> > >> > want the compaction to affect the data sink or the whole
>> > > pipeline.
>> > > >>
>> > > >> > >> >
>> > > >>
>> > > >> > >> > 3, Exactly once guarantee between write and compact phase.
>> > > >>
>> > > >> > >> > Once we separate write phase and compact phase, we need to
>> > > consider
>> > > >>
>> > > >> > >> > how to guarantee
>> > > >>
>> > > >> > >> > the exact once semantic between two phases. We should not
>> lose
>> > > any data or
>> > > >>
>> > > >> > >> > files on the compactor(Step-3) in any case and cause the
>> > > compaction result
>> > > >>
>> > > >> > >> > to be inconsistent with before. I think flink should
>> provide an
>> > > easy-to-use
>> > > >>
>> > > >> > >> > interface to make that easier.
>> > > >>
>> > > >> > >> >
>> > > >>
>> > > >> > >> > 4, Metadata operation and compaction result validation.
>> > > >>
>> > > >> > >> > In the compact phase, there may be not only compact files,
>> but
>> > > also a lot
>> > > >>
>> > > >> > >> > of metadata operations, such as the iceberg needing to
>> > > read/write manifest
>> > > >>
>> > > >> > >> > and do MOR. And we need some interface to support users to
>> do
>> > > some
>> > > >>
>> > > >> > >> > validation of the compaction result. I think these points
>> should
>> > > be
>> > > >>
>> > > >> > >> > considered when we design the compaction API.
>> > > >>
>> > > >> > >> >
>> > > >>
>> > > >> > >> >
>> > > >>
>> > > >> > >> > Back to FLIP-191, option 1 looks very complicated while
>> option 2
>> > > is
>> > > >>
>> > > >> > >> > relatively simple, but neither of these two solutions
>> separates
>> > > the write
>> > > >>
>> > > >> > >> > phase from the compact phase. So I think we should consider
>> the
>> > > points I
>> > > >>
>> > > >> > >> > mentioned above. And if you have any other questions you can
>> > > always feel
>> > > >>
>> > > >> > >> > free to reach out to me!
>> > > >>
>> > > >> > >> >
>> > > >>
>> > > >> > >> > BR,
>> > > >>
>> > > >> > >> > Reo
>> > > >>
>> > > >> > >> >
>> > > >>
>> > > >> > >> > Fabian Paul 于2021年11月8日周一 下午7:59写道:
>> > > >>
>> > > >> > >> >
>> > > >>
>> > > >> > >> > > Hi all,
>> > > >>
>> > > >> > >> > >
>> > > >>
>> > > >> > >> > > Thanks for the lively discussions. I am really excited to
>> see
>> > > so many
>> > > >>
>> > > >> > >> > > people
>> > > >>
>> > > >> > >> > > participating in this thread. It also underlines the need
>> that
>> > > many people
>> > > >>
>> > > >> > >> > > would
>> > > >>
>> > > >> > >> > > like to see a solution soon.
>> > > >>
>> > > >> > >> > >
>> > > >>
>> > > >> > >> > > I have updated the FLIP and removed the parallelism
>> > > configuration because
>> > > >>
>> > > >> > >> > > it is
>> > > >>
>> > > >> > >> > > unnecessary since users can configure a constant exchange
>> key
>> > > to send all
>> > > >>
>> > > >> > >> > > committables to only one committable aggregator.
>> > > >>
>> > > >> > >> > >
>> > > >>
>> > > >> > >> > >
>> > > >>
>> > > >> > >> > > 1. Burden for developers w.r.t batch stream unification.
>> > > >>
>> > > >> > >> > >
>> > > >>
>> > > >> > >> > > @yun @guowei, from a theoretical point you are right about
>> > > exposing the
>> > > >>
>> > > >> > >> > > DataStream
>> > > >>
>> > > >> > >> > > API in the sink users have the full power to write correct
>> > > batch and
>> > > >>
>> > > >> > >> > > streaming
>> > > >>
>> > > >> > >> > > sinks. I think in reality a lot of users still struggle to
>> > > build pipelines
>> > > >>
>> > > >> > >> > > with
>> > > >>
>> > > >> > >> > > i.e. the operator pipeline which works correct in
>> streaming
>> > > and batch mode.
>> > > >>
>> > > >> > >> > > Another problem I see is by exposing more deeper concepts
>> is
>> > > that we
>> > > >>
>> > > >> > >> > > cannot do
>> > > >>
>> > > >> > >> > > any optimization because we cannot reason about how sinks
>> are
>> > > built in the
>> > > >>
>> > > >> > >> > > future.
>> > > >>
>> > > >> > >> > >
>> > > >>
>> > > >> > >> > > We should also try to steer users towards using only
>> > > `Functions` to give
>> > > >>
>> > > >> > >> > > us more
>> > > >>
>> > > >> > >> > > flexibility to swap the internal operator representation.
>> I
>> > > agree with
>> > > >>
>> > > >> > >> > > @yun we
>> > > >>
>> > > >> > >> > > should try to make the `ProcessFunction` more versatile to
>> > > work on that
>> > > >>
>> > > >> > >> > > goal but
>> > > >>
>> > > >> > >> > > I see this as unrelated to the FLIP.
>> > > >>
>> > > >> > >> > >
>> > > >>
>> > > >> > >> > >
>> > > >>
>> > > >> > >> > > 2. Regarding Commit / Global commit
>> > > >>
>> > > >> > >> > >
>> > > >>
>> > > >> > >> > > I envision the global committer to be specific depending
>> on
>> > > the data lake
>> > > >>
>> > > >> > >> > > solution you want to write to. However, it is entirely
>> > > orthogonal to the
>> > > >>
>> > > >> > >> > > compaction.
>> > > >>
>> > > >> > >> > > Currently, I do not expect any changes w.r.t the Global
>> commit
>> > > introduces
>> > > >>
>> > > >> > >> > > by
>> > > >>
>> > > >> > >> > > this FLIP.
>> > > >>
>> > > >> > >> > >
>> > > >>
>> > > >> > >> > >
>> > > >>
>> > > >> > >> > > 3. Regarding the case of trans-checkpoints merging
>> > > >>
>> > > >> > >> > >
>> > > >>
>> > > >> > >> > > @yun, as user, I would expect that if the committer
>> receives
>> > > in a
>> > > >>
>> > > >> > >> > > checkpoint files
>> > > >>
>> > > >> > >> > > to merge/commit that these are also finished when the
>> > > checkpoint finishes.
>> > > >>
>> > > >> > >> > > I think all sinks rely on this principle currently i.e.,
>> > > KafkaSink needs to
>> > > >>
>> > > >> > >> > > commit all open transactions until the next checkpoint can
>> > > happen.
>> > > >>
>> > > >> > >> > >
>> > > >>
>> > > >> > >> > > Maybe in the future, we can somehow move the
>> Committer#commit
>> > > call to an
>> > > >>
>> > > >> > >> > > asynchronous execution, but we should discuss it as a
>> separate
>> > > thread.
>> > > >>
>> > > >> > >> > >
>> > > >>
>> > > >> > >> > > > We probably should first describe the different causes
>> of
>> > > small files and
>> > > >>
>> > > >> > >> > > > what problems was this proposal trying to solve. I
>> wrote a
>> > > data shuffling
>> > > >>
>> > > >> > >> > > > proposal [1] for Flink Iceberg sink (shared with Iceberg
>> > > community [2]).
>> > > >>
>> > > >> > >> > > It
>> > > >>
>> > > >> > >> > > > can address small files problems due to skewed data
>> > > distribution across
>> > > >>
>> > > >> > >> > > > Iceberg table partitions. Streaming shuffling before
>> writers
>> > > (to files)
>> > > >>
>> > > >> > >> > > is
>> > > >>
>> > > >> > >> > > > typically more efficient than post-write file compaction
>> > > (which involves
>> > > >>
>> > > >> > >> > > > read-merge-write). It is usually cheaper to prevent a
>> > > problem (small
>> > > >>
>> > > >> > >> > > files)
>> > > >>
>> > > >> > >> > > > than fixing it.
>> > > >>
>> > > >> > >> > >
>> > > >>
>> > > >> > >> > >
>> > > >>
>> > > >> > >> > > @steven you are raising a good point, although I think
>> only
>> > > using a
>> > > >>
>> > > >> > >> > > customizable
>> > > >>
>> > > >> > >> > > shuffle won't address the generation of small files. One
>> > > assumption is that
>> > > >>
>> > > >> > >> > > at least the sink generates one file per subtask, which
>> can
>> > > already be too
>> > > >>
>> > > >> > >> > > many.
>> > > >>
>> > > >> > >> > > Another problem is that with low checkpointing intervals,
>> the
>> > > files do not
>> > > >>
>> > > >> > >> > > meet
>> > > >>
>> > > >> > >> > > the required size. The latter point is probably
>> addressable by
>> > > changing the
>> > > >>
>> > > >> > >> > > checkpoint interval, which might be inconvenient for some
>> > > users.
>> > > >>
>> > > >> > >> > >
>> > > >>
>> > > >> > >> > > > The sink coordinator checkpoint problem (mentioned in
>> option
>> > > 1) would be
>> > > >>
>> > > >> > >> > > > great if Flink can address it. In the spirit of source
>> > > >>
>> > > >> > >> > > (enumerator-reader)
>> > > >>
>> > > >> > >> > > > and sink (writer-coordinator) duality, sink coordinator
>> > > checkpoint should
>> > > >>
>> > > >> > >> > > > happen after the writer operator. This would be a
>> natural
>> > > fit to support
>> > > >>
>> > > >> > >> > > > global committer in FLIP-143. It is probably an
>> orthogonal
>> > > matter to this
>> > > >>
>> > > >> > >> > > > proposal.
>> > > >>
>> > > >> > >> > >
>> > > >>
>> > > >> > >> > >
>> > > >>
>> > > >> > >> > > To me the question here is what are the benefits of
>> having a
>> > > coordinator in
>> > > >>
>> > > >> > >> > > comparison to a global committer/aggregator operator.
>> > > >>
>> > > >> > >> > >
>> > > >>
>> > > >> > >> > > > Personally, I am usually in favor of keeping streaming
>> > > ingestion (to data
>> > > >>
>> > > >> > >> > > > lake) relatively simple and stable. Also sometimes
>> > > compaction and sorting
>> > > >>
>> > > >> > >> > > > are performed together in data rewrite maintenance jobs
>> to
>> > > improve read
>> > > >>
>> > > >> > >> > > > performance. In that case, the value of compacting (in
>> Flink
>> > > streaming
>> > > >>
>> > > >> > >> > > > ingestion) diminishes.
>> > > >>
>> > > >> > >> > >
>> > > >>
>> > > >> > >> > >
>> > > >>
>> > > >> > >> > > I agree it is always possible to have scheduled
>> maintenance
>> > > jobs keeping
>> > > >>
>> > > >> > >> > > care of
>> > > >>
>> > > >> > >> > > your data i.e., doing compaction. Unfortunately, the
>> downside
>> > > is that you
>> > > >>
>> > > >> > >> > > have to your data after it is already available for other
>> > > downstream
>> > > >>
>> > > >> > >> > > consumers.
>> > > >>
>> > > >> > >> > > I guess this can lead to all kinds of visibility
>> problems. I
>> > > am also
>> > > >>
>> > > >> > >> > > surprised that
>> > > >>
>> > > >> > >> > > you personally are a fan of this approach and, on the
>> other
>> > > hand, are
>> > > >>
>> > > >> > >> > > developing
>> > > >>
>> > > >> > >> > > the Iceberg sink, which goes somewhat against your
>> mentioned
>> > > principle of
>> > > >>
>> > > >> > >> > > keeping
>> > > >>
>> > > >> > >> > > the sink simple.
>> > > >>
>> > > >> > >> > >
>> > > >>
>> > > >> > >> > > > Currently, it is unclear from the doc and this thread
>> where
>> > > the
>> > > >>
>> > > >> > >> > > compaction
>> > > >>
>> > > >> > >> > > > is actually happening. Jingsong's reply described one
>> model
>> > > >>
>> > > >> > >> > > > writer (parallel) -> aggregator (single-parallelism
>> > > compaction planner)
>> > > >>
>> > > >> > >> > > ->
>> > > >>
>> > > >> > >> > > > compactor (parallel) -> global committer
>> (single-parallelism)
>> > > >>
>> > > >> > >> > >
>> > > >>
>> > > >> > >> > >
>> > > >>
>> > > >> > >> > > My idea of the topology is very similar to the one
>> outlined by
>> > > Jinsong. The
>> > > >>
>> > > >> > >> > > compaction will happen in the committer operator.
>> > > >>
>> > > >> > >> > >
>> > > >>
>> > > >> > >> > > >
>> > > >>
>> > > >> > >> > > > In the Iceberg community, the following model has been
>> > > discussed. It is
>> > > >>
>> > > >> > >> > > > better for Iceberg because it won't delay the data
>> > > availability.
>> > > >>
>> > > >> > >> > > > writer (parallel) -> global committer for append (single
>> > > parallelism) ->
>> > > >>
>> > > >> > >> > > > compactor (parallel) -> global committer for rewrite
>> commit
>> > > (single
>> > > >>
>> > > >> > >> > > > parallelism)
>> > > >>
>> > > >> > >> > >
>> > > >>
>> > > >> > >> > >
>> > > >>
>> > > >> > >> > > From a quick glimpse, it seems that the exact same
>> topology is
>> > > possible to
>> > > >>
>> > > >> > >> > > express with the committable aggregator, but this
>> definitely
>> > > depends on
>> > > >>
>> > > >> > >> > > the exact
>> > > >>
>> > > >> > >> > > setup.
>> > > >>
>> > > >> > >> > >
>> > > >>
>> > > >> > >> > > Best,
>> > > >>
>> > > >> > >> > > Fabian
>> > > >>
>> > > >> > >>
>> > >
>>
>>
>>

Re: Re: Re: [DISCUSS] FLIP-191: Extend unified Sink interface to support small file compaction

Posted by Yun Gao <yu...@aliyun.com.INVALID>.
Hi Fabian, 

Very thanks for the explanation! Sorry that cascading commits might not 
be accurate and my initial concern is that if we have cases that the post-committer
topology wants to process the committables after they get committed. But since
we seems indeed have walkaround for that, thus I also totally agree we could postpone 
supporting for this case in the future. 

I have no other concerns from my side~ Very thanks for drafting the FLIP and 
bringing up the discussion!

Best,
Yun



 ------------------Original Mail ------------------
Sender:Fabian Paul <fp...@apache.org>
Send Date:Mon Jan 3 16:45:21 2022
Recipients:dev <de...@flink.apache.org>, Yun Gao <yu...@aliyun.com>
Subject:Re: Re: [DISCUSS] FLIP-191: Extend unified Sink interface to support small file compaction
Hi Yun,



Thanks for summarizing the two issues.



1. Losing intermediate shuffle data in batch mode



I fully agree with your analysis. We will start to mitigate the

problem by introducing the blocking exchanges and documenting that it

will not prevent duplication in case a complete taskmanager is lost.

In the future, we may introduce some persistent HA storage to write

these intermediate results to but I think this should not be part of

this FLIP. One other option is to use a remote shuffle service that

can persist the results. In the end, probably the final solution

requires some kind of persistent storage.



2. Cascading checkpoints in the post-commit topology for the final checkpoint



One thing I'd like to mention is that this problem already exists with

the current implementation of the global committer so we do not worsen

the situation. I do not think we need multiple cascading commits but

only a single additional checkpoint for the post-commit topology after

the final checkpoint. Our idea is to trigger such a checkpoint and

merge it with the previously taken result so it only appears to be one

checkpoint.

Overall, I want to fix this problem as soon as possible but I see the

other API changes are more critical and we can treat this problem as a

bug fix because it also affects the Sink V1 global committer.

For only taking a savepoint I do not see a problem that requires

additional logic because when the job is recovered the previous

committables are emitted again on the next checkpoint.



Hopefully, this answers your question. I will start the vote if there

are no other concerns since we are moving towards the feature freeze

deadline and the FLIP is targeted for Flink 1.15.



Best,

Fabian



On Tue, Dec 21, 2021 at 11:29 AM Yun Gao  wrote:

>

> Hi,

>

> As Arvid appointed out, with some more checks, if a job vertex have multiple downstream vertices and

> they are all connected via blocking edges, they should not affect each other, as long as we could

> ensure the intermediate shuffle data do not get lost, the finished precedent job vertex not restarted.

> Thus as a whole, for 2pc under batch mode, we should be able to ensure exactly-once with

> JM Failover + HA result partition + all-to-all blocking edge before the committer. For the first step,

> we could first have all-to-all blocking edge to reduce the probability of repeat data.

>

> And for the post-committer topology, sorry it reminds me another issue we ever discussed

> before: for the streaming mode, we should not be able to support cascade commit since now

> emit records in notifyCheckpointComplete is not supported after FLIP-147. Thus I think the

> committables we emit from the committer to the post-committer topology should be before

> they actually committed, right? If it is the case, perhaps we either document the situation clearly

> and if one job requires taking actions after the committables get committed, he could further check

> if the committables are committed manully(like scanning the filesystem to see if the files are renamed),

> or we might postpone supporting the post-committer topology until we have further determined the

> semantics and solutions ?

>

> Best,

> Yun

>

>

>

> ------------------------------------------------------------------

> From:Arvid Heise 

> Send Time:2021 Dec. 16 (Thu.) 21:48

> To:Yun Gao 

> Cc:dev 

> Subject:Re: Re: [DISCUSS] FLIP-191: Extend unified Sink interface to support small file compaction

>

> I just noticed offline with Yun that I have some misconception on how how blocking data exchange work in Flink. Apparently, a subtask of the stage 2 is already started when all connected subtasks of stage 1 are finished. I was assuming that it works similarly to Hadoop where all tasks of stage 1 have to be finished before any task of stage 2 is started.

> So to amend the proposal, we would introduce a full shuffle before committer in batch. That would simulate the behavior of Hadoop.

>

> We also need to verify what happens in case of multiple sinks. In that case, we may have started committer of sink 1 while sink 2 is still writing. That would be fine as long as sink 1 writers are not restarted. If not then it seems as if we may improve scheduler to actually not restart "finished" tasks. It seems like an orthogonal issue though.

>

> Note that all discussed issues apply to Sink V1 as well, so the FLIP is not making anything worse.

>

>

> On Thu, Dec 16, 2021 at 1:53 PM Arvid Heise  wrote:

> Hi Yun,

>

> the basic idea is to use different regions for the different stages.

>

> So you have your stages:

> Stage 1:  -> pre-writer topology -> writer -> committables (materialized)

> Stage 2: committables (materialized) -> pre-commit topology -> committer -> succesful committables (materialized)

> Stage 3: succesful committables (materialized) -> post-commit topology

>

> If any instance of a given stages fails, the whole stage is restarted.

> So, in case of the main pipeline (stage 1) fails, no data will be committed at all. On a rerun, we start fresh (or from the previous stage).

> Only, when all data has been written, we start with committing the data. An issue during committing, will retrigger the commit stage (stage 2) and only that stage. Thus, all committables are stable and remain stable.

> When we are done committing all committables, we run the post-commit topology similarly "atomically".

>

> So now the cavaets:

> - If committer is rerun, all committables are restarted. So the committer needs to be idempotent. That is the same with STREAMING mode now and afaik there is no way around it.

> - If we lose a TM during commit phase, we will run into the original issues of inconstent data as we need to rerun the whole job. That would be solved with HA storage and we haven't found any solution that doesn't require some kind of external storage. However, the commit phase should be rather fast and errors are improbable (low volume).

>

> I'd still like to have an HA storage but that issue is also in Sink V1 and kind of orthogonal. It's also nothing that we can solve without involving more folks (I'm trying to kick start that in the background).

>

> On Thu, Dec 16, 2021 at 1:31 PM Yun Gao  wrote:

> Hi,

>

> Very thanks Fabian for the explanation and it solves most of the issues.

> There is one left issue I want to have a double confirmation is that for

> the edges between writer and committer and in the post-committer topology,

> perhaps the result partition with HA storage is not enough solve all the issues

> directly ? It is due to after the committer and post-committer topology is finished

> and the data is committed, it might still be restarted due to JM failover and the

> deterministic problem (namely the example of (A -> [B1, B2], A, B1 have finished and

> B2 failed, if -> is rebalance / random / rescale, all of A, B1, B2 would restarted). Then

> the records would be produced and created for the second times.

>

> We might let the writers to skip producing the new records, but if we have multiple sinks like

> OP1 -> (writer 1 -> committer 1)

> |--> (writer 2 -> committer 2)

>

> and the failover happens after writer 1 & committer 1 get finished but writer 2 is running,

> if op1 produced different records across the two runs, then the two sinks would produces

> different data, which might be not suitable in some cases. Perhaps we need some support

> from the scheduler side?

>

> But I also agree this could be a separate issue and we could solve it separately in some future

> as long as we know how to solve it~

>

> Best,

> Yun

>

>

> ------------------------------------------------------------------

> From:Arvid Heise 

> Send Time:2021 Dec. 16 (Thu.) 19:54

> To:dev 

> Cc:Yun Gao 

> Subject:Re: Re: [DISCUSS] FLIP-191: Extend unified Sink interface to support small file compaction

>

> Just a quick amend: There will be no blocking exchange in the pre-writer exchange for performance reasons.

> After the writer, we have tiny data volume and are free to add as many as we see necessary.

>

> On Thu, Dec 16, 2021 at 11:18 AM Fabian Paul  wrote:

> Hi Till,

>

> Good point, in the scenario with the blocking keyed exchange between

> the writer and committer my idea is to make the committer effectively

> the global committer. With Sink V2 there is no real difference anymore

> between the committer and global committer.

> You are right that everything after the committer would be part of the

> same failover region but we plan to insert a blocking exchange by

> default before all of the custom topologies.

>

> Best,

> Fabian

>

> On Thu, Dec 16, 2021 at 11:08 AM Till Rohrmann  wrote:

> >

> > Hi Fabian,

> >

> > quick question on your comment 3. If there is a pipelined data exchange

> > with a keyBy between the writers/committers and the component that does the

> > global commit, then there will only be a single failover region. So is it

> > correct that you assumed blocking data exchanges for the scenario you

> > described?

> >

> > Cheers,

> > Till

> >

> > On Thu, Dec 16, 2021 at 9:23 AM Fabian Paul  wrote:

> >

> > > Hi Yun,

> > >

> > > Thanks for your fast feedback. Let me clarify your points.

> > >

> > > 1. We solve it by using StreamExchangeMode.BATCH before any exchange.

> > > That obviously doesn’t help with lost TM but we would need to employ

> > > HA storage for that. Same issue as now and orthogonal.

> > >

> > > 2. Extending V1 with V2 or vice versa would require renames of methods

> > > (since return types are non-optional) and is making API changes. Even

> > > though Experimental, we want to give connector developers the

> > > opportunity to provide 1 implementation for all of Flink 1.X. We will

> > > offer an internal adapter from V1 to V2, 2 sinkTo , and internally

> > > just have one code-path.

> > >

> > > 3. DataStreamSink would act as a unified view on all the operators and

> > > update them all at once when using setParallelism and so on (setName

> > > and setUid will receive suffixes per operator).

> > > Iceberg actually has a different requirement: They want to have a

> > > committer with parallelism 1 but as a coordinator such that

> > > embarrassingly parallel pipelines have different fail-over regions. I

> > > was thinking that in this case, they need to implement a no-op

> > > committer (that just forwards the committables) and use a post-commit

> > > topology that achieves that.

> > > Another option is that they use the preCommit topology and insert a

> > > constant key-by that forwards all committables to a single committer.

> > > We are planning to provide building blocks for such pipelines as we

> > > go.

> > >

> > > Best,

> > > Fabian

> > >

> > > On Thu, Dec 16, 2021 at 5:50 AM Yun Gao  wrote:

> > > >

> > > > Hi Fabian,

> > > >

> > > > Very thanks for the update! I think the latest version in general looks

> > > good from my side

> > > > and I think using separate feature interface would be much more easy to

> > > understand

> > > > and extend in the future. I have some pending issues on the details

> > > though:

> > > >

> > > > 1. The first one is if we could support end-to-end exactly-once with

> > > post-committing

> > > > topology in the batch mode ? Since for the batch mode, currently we

> > > could only commit

> > > > all the transactions after the whole job is finished, otherwise if

> > > there are JM failover or the

> > > > writer / committer get restarted due to indeterminstic (A -> [B1, B2],

> > > A, B1 have finished and

> > > > B2 failed, if -> is rebalance / random / rescale, all of A, B1, B2

> > > would restarted), there might

> > > > be repeat records. Previously one possible thought is to move committer

> > > and global committer

> > > > to the operator coordinator, but if it is a topology, we might need

> > > some other kind of solutions?

> > > >

> > > > 2. I also want to have a dobule confirmation with the compatibility:

> > > since the old sink is also named

> > > > with Sink, do we want to put the Sink v2 in a new package ? Besides,

> > > since we might want to keep

> > > > only have one `sinkTo(Sink sink)` , perhaps we also need to make the

> > > Sink v1 to be a subclass of

> > > > Sink v2 and extends the stateful and two-phase-commit sinks, right?

> > > >

> > > > 3. I'd like also have a confirmation on ours thoughts with the

> > > `DataStreamSink` returned by the sinkTo method:

> > > > The main issue is how do we implement the method like `setParallelism`

> > > or `setMaxParallelism` since now the sink

> > > > would be translated to multiple transformations? perhaps we could make

> > > it the default values for all the transformations

> > > > for the sink? A related issue would be for iceberg sink, I think it

> > > would need to have only one committer to avoid the

> > > > competition of the optimistic locks (which would cause performance

> > > degradation), then it might need to have N writers

> > > > with 1 committers, to build such topology, perhaps we might need to add

> > > new methods to specify the parallelism of

> > > > the writers and committers separately?

> > > >

> > > > Best,

> > > > Yun

> > > >

> > > >

> > > > ------------------Original Mail ------------------

> > > > Sender:Fabian Paul 

> > > > Send Date:Mon Dec 13 23:59:43 2021

> > > > Recipients:dev 

> > > > Subject:Re: [DISCUSS] FLIP-191: Extend unified Sink interface to support

> > > small file compaction

> > > >>

> > > >> Hi all,

> > > >>

> > > >>

> > > >>

> > > >> After a lot of discussions with different, we received very fruitful

> > > >>

> > > >> feedback and reworked the ideas behind this FLIP. Initially, we had

> > > >>

> > > >> the impression that the compaction problem is solvable by a single

> > > >>

> > > >> topology that we can reuse across different sinks. We now have a

> > > >>

> > > >> better understanding that different external systems require different

> > > >>

> > > >> compaction mechanism i.e. Hive requires compaction before finally

> > > >>

> > > >> registering the file in the metastore or Iceberg compacts the files

> > > >>

> > > >> after they have been registered and just lazily compacts them.

> > > >>

> > > >>

> > > >>

> > > >> Considering all these different views we came up with a design that

> > > >>

> > > >> builds upon what @guowei.mgw@gmail.com and @yungao.gy@aliyun.com have

> > > >>

> > > >> proposed at the beginning. We allow inserting custom topologies before

> > > >>

> > > >> and after the SinkWriters and Committers. Furthermore, we do not see

> > > >>

> > > >> it as a downside. The Sink interfaces that will expose the DataStream

> > > >>

> > > >> to the user reside in flink-streaming-java in contrast to the basic

> > > >>

> > > >> Sink interfaces that reside fin flink-core deem it to be only used by

> > > >>

> > > >> expert users.

> > > >>

> > > >>

> > > >>

> > > >> Moreover, we also wanted to remove the global committer from the

> > > >>

> > > >> unified Sink interfaces and replace it with a custom post-commit

> > > >>

> > > >> topology. Unfortunately, we cannot do it without breaking the Sink

> > > >>

> > > >> interface since the GlobalCommittables are part of the parameterized

> > > >>

> > > >> Sink interface. Thus, we propose building a new Sink V2 interface

> > > >>

> > > >> consisting of composable interfaces that do not offer the

> > > >>

> > > >> GlobalCommitter anymore. We will implement a utility to extend a Sink

> > > >>

> > > >> with post topology that mimics the behavior of the GlobalCommitter.

> > > >>

> > > >> The new Sink V2 provides the same sort of methods as the Sink V1

> > > >>

> > > >> interface, so a migration of sinks that do not use the GlobalCommitter

> > > >>

> > > >> should be very easy.

> > > >>

> > > >> We plan to keep the existing Sink V1 interfaces to not break

> > > >>

> > > >> externally built sinks. As part of this FLIP, we migrate all the

> > > >>

> > > >> connectors inside of the main repository to the new Sink V2 API.

> > > >>

> > > >>

> > > >>

> > > >> The FLIP document is also updated and includes the proposed changes.

> > > >>

> > > >>

> > > >>

> > > >> Looking forward to your feedback,

> > > >>

> > > >> Fabian

> > > >>

> > > >>

> > > >>

> > > >>

> > > https://cwiki.apache.org/confluence/display/FLINK/FLIP-191%3A+Extend+unified+Sink+interface+to+support+small+file+compaction

> > > >>

> > > >>

> > > >>

> > > >>

> > > >>

> > > >> On Thu, Dec 2, 2021 at 10:15 AM Roman Khachatryan wrote:

> > > >>

> > > >> >

> > > >>

> > > >> > Thanks for clarifying (I was initially confused by merging state files

> > > >>

> > > >> > rather than output files).

> > > >>

> > > >> >

> > > >>

> > > >> > > At some point, Flink will definitely have some WAL adapter that can

> > > turn any sink into an exactly-once sink (with some caveats). For now, we

> > > keep that as an orthogonal solution as it has a rather high price (bursty

> > > workload with high latency). Ideally, we can keep the compaction

> > > asynchronously...

> > > >>

> > > >> >

> > > >>

> > > >> > Yes, that would be something like a WAL. I agree that it would have a

> > > >>

> > > >> > different set of trade-offs.

> > > >>

> > > >> >

> > > >>

> > > >> >

> > > >>

> > > >> > Regards,

> > > >>

> > > >> > Roman

> > > >>

> > > >> >

> > > >>

> > > >> > On Mon, Nov 29, 2021 at 3:33 PM Arvid Heise wrote:

> > > >>

> > > >> > >>

> > > >>

> > > >> > >> > One way to avoid write-read-merge is by wrapping SinkWriter with

> > > >>

> > > >> > >> > another one, which would buffer input elements in a temporary

> > > storage

> > > >>

> > > >> > >> > (e.g. local file) until a threshold is reached; after that, it

> > > would

> > > >>

> > > >> > >> > invoke the original SinkWriter. And if a checkpoint barrier

> > > comes in

> > > >>

> > > >> > >> > earlier, it would send written data to some aggregator.

> > > >>

> > > >> > >>

> > > >>

> > > >> > >> I think perhaps this seems to be a kind of WAL method? Namely we

> > > first

> > > >>

> > > >> > >> write the elements to some WAL logs and persist them on checkpoint

> > > >>

> > > >> > >> (in snapshot or remote FS), or we directly write WAL logs to the

> > > remote

> > > >>

> > > >> > >> FS eagerly.

> > > >>

> > > >> > >>

> > > >>

> > > >> > > At some point, Flink will definitely have some WAL adapter that can

> > > turn any sink into an exactly-once sink (with some caveats). For now, we

> > > keep that as an orthogonal solution as it has a rather high price (bursty

> > > workload with high latency). Ideally, we can keep the compaction

> > > asynchronously...

> > > >>

> > > >> > >

> > > >>

> > > >> > > On Mon, Nov 29, 2021 at 8:52 AM Yun Gao wrote:

> > > >>

> > > >> > >>

> > > >>

> > > >> > >> Hi,

> > > >>

> > > >> > >>

> > > >>

> > > >> > >> @Roman very sorry for the late response for a long time,

> > > >>

> > > >> > >>

> > > >>

> > > >> > >> > Merging artifacts from multiple checkpoints would apparently

> > > >>

> > > >> > >> require multiple concurrent checkpoints

> > > >>

> > > >> > >>

> > > >>

> > > >> > >> I think it might not need concurrent checkpoints: suppose some

> > > >>

> > > >> > >> operators (like the committer aggregator in the option 2) maintains

> > > >>

> > > >> > >> the list of files to merge, it could stores the lists of files to

> > > merge

> > > >>

> > > >> > >> in the states, then after several checkpoints are done and we have

> > > >>

> > > >> > >> enough files, we could merge all the files in the list.

> > > >>

> > > >> > >>

> > > >>

> > > >> > >> > Asynchronous merging in an aggregator would require some

> > > resolution

> > > >>

> > > >> > >> > logic on recovery, so that a merged artifact can be used if the

> > > >>

> > > >> > >> > original one was deleted. Otherwise, wouldn't recovery fail

> > > because

> > > >>

> > > >> > >> > some artifacts are missing?

> > > >>

> > > >> > >> > We could also defer deletion until the "compacted" checkpoint is

> > > >>

> > > >> > >> > subsumed - but isn't it too late, as it will be deleted anyways

> > > once

> > > >>

> > > >> > >> > subsumed?

> > > >>

> > > >> > >>

> > > >>

> > > >> > >> I think logically we could delete the original files once the

> > > "compacted" checkpoint

> > > >>

> > > >> > >> (which finish merging the compacted files and record it in the

> > > checkpoint) is completed

> > > >>

> > > >> > >> in all the options. If there are failover before we it, we could

> > > restart the merging and if

> > > >>

> > > >> > >> there are failover after it, we could have already recorded the

> > > files in the checkpoint.

> > > >>

> > > >> > >>

> > > >>

> > > >> > >> > One way to avoid write-read-merge is by wrapping SinkWriter with

> > > >>

> > > >> > >> > another one, which would buffer input elements in a temporary

> > > storage

> > > >>

> > > >> > >> > (e.g. local file) until a threshold is reached; after that, it

> > > would

> > > >>

> > > >> > >> > invoke the original SinkWriter. And if a checkpoint barrier

> > > comes in

> > > >>

> > > >> > >> > earlier, it would send written data to some aggregator.

> > > >>

> > > >> > >>

> > > >>

> > > >> > >> I think perhaps this seems to be a kind of WAL method? Namely we

> > > first

> > > >>

> > > >> > >> write the elements to some WAL logs and persist them on checkpoint

> > > >>

> > > >> > >> (in snapshot or remote FS), or we directly write WAL logs to the

> > > remote

> > > >>

> > > >> > >> FS eagerly.

> > > >>

> > > >> > >>

> > > >>

> > > >> > >> Sorry if I do not understand correctly somewhere.

> > > >>

> > > >> > >>

> > > >>

> > > >> > >> Best,

> > > >>

> > > >> > >> Yun

> > > >>

> > > >> > >>

> > > >>

> > > >> > >>

> > > >>

> > > >> > >> ------------------------------------------------------------------

> > > >>

> > > >> > >> From:Roman Khachatryan

> > > >>

> > > >> > >> Send Time:2021 Nov. 9 (Tue.) 22:03

> > > >>

> > > >> > >> To:dev

> > > >>

> > > >> > >> Subject:Re: [DISCUSS] FLIP-191: Extend unified Sink interface to

> > > support small file compaction

> > > >>

> > > >> > >>

> > > >>

> > > >> > >> Hi everyone,

> > > >>

> > > >> > >>

> > > >>

> > > >> > >> Thanks for the proposal and the discussion, I have some remarks:

> > > >>

> > > >> > >> (I'm not very familiar with the new Sink API but I thought about

> > > the

> > > >>

> > > >> > >> same problem in context of the changelog state backend)

> > > >>

> > > >> > >>

> > > >>

> > > >> > >> 1. Merging artifacts from multiple checkpoints would apparently

> > > >>

> > > >> > >> require multiple concurrent checkpoints (otherwise, a new

> > > checkpoint

> > > >>

> > > >> > >> won't be started before completing the previous one; and the

> > > previous

> > > >>

> > > >> > >> one can't be completed before durably storing the artifacts).

> > > However,

> > > >>

> > > >> > >> concurrent checkpoints are currently not supported with Unaligned

> > > >>

> > > >> > >> checkpoints (this is besides increasing e2e-latency).

> > > >>

> > > >> > >>

> > > >>

> > > >> > >> 2. Asynchronous merging in an aggregator would require some

> > > resolution

> > > >>

> > > >> > >> logic on recovery, so that a merged artifact can be used if the

> > > >>

> > > >> > >> original one was deleted. Otherwise, wouldn't recovery fail because

> > > >>

> > > >> > >> some artifacts are missing?

> > > >>

> > > >> > >> We could also defer deletion until the "compacted" checkpoint is

> > > >>

> > > >> > >> subsumed - but isn't it too late, as it will be deleted anyways

> > > once

> > > >>

> > > >> > >> subsumed?

> > > >>

> > > >> > >>

> > > >>

> > > >> > >> 3. Writing small files, then reading and merging them for *every*

> > > >>

> > > >> > >> checkpoint seems worse than only reading them on recovery. I guess

> > > I'm

> > > >>

> > > >> > >> missing some cases of reading, so to me it would make sense to

> > > mention

> > > >>

> > > >> > >> these cases explicitly in the FLIP motivation section.

> > > >>

> > > >> > >>

> > > >>

> > > >> > >> 4. One way to avoid write-read-merge is by wrapping SinkWriter with

> > > >>

> > > >> > >> another one, which would buffer input elements in a temporary

> > > storage

> > > >>

> > > >> > >> (e.g. local file) until a threshold is reached; after that, it

> > > would

> > > >>

> > > >> > >> invoke the original SinkWriter. And if a checkpoint barrier comes

> > > in

> > > >>

> > > >> > >> earlier, it would send written data to some aggregator. It will

> > > >>

> > > >> > >> increase checkpoint delay (async phase) compared to the current

> > > Flink;

> > > >>

> > > >> > >> but not compared to the write-read-merge solution, IIUC.

> > > >>

> > > >> > >> Then such "BufferingSinkWriters" could aggregate input elements

> > > from

> > > >>

> > > >> > >> each other, potentially recursively (I mean something like

> > > >>

> > > >> > >>

> > > https://cwiki.apache.org/confluence/download/attachments/173082889/DSTL-DFS-DAG.png

> > > >>

> > > >> > >> )

> > > >>

> > > >> > >>

> > > >>

> > > >> > >> 5. Reducing the number of files by reducing aggregator parallelism

> > > as

> > > >>

> > > >> > >> opposed to merging on reaching size threshold will likely be less

> > > >>

> > > >> > >> optimal and more difficult to configure. OTH, thresholds might be

> > > more

> > > >>

> > > >> > >> difficult to implement and (with recursive merging) would incur

> > > higher

> > > >>

> > > >> > >> latency. Maybe that's also something to decide explicitly or at

> > > least

> > > >>

> > > >> > >> mention in the FLIP.

> > > >>

> > > >> > >>

> > > >>

> > > >> > >>

> > > >>

> > > >> > >>

> > > >>

> > > >> > >> Regards,

> > > >>

> > > >> > >> Roman

> > > >>

> > > >> > >>

> > > >>

> > > >> > >>

> > > >>

> > > >> > >> On Tue, Nov 9, 2021 at 5:23 AM Reo Lei wrote:

> > > >>

> > > >> > >> >

> > > >>

> > > >> > >> > Hi Fabian,

> > > >>

> > > >> > >> >

> > > >>

> > > >> > >> > Thanks for drafting the FLIP and trying to support small file

> > > compaction. I

> > > >>

> > > >> > >> > think this feature is very urgent and valuable for users(at

> > > least for me).

> > > >>

> > > >> > >> >

> > > >>

> > > >> > >> > Currently I am trying to support streaming rewrite(compact) for

> > > Iceberg on

> > > >>

> > > >> > >> > PR#3323 . As Steven mentioned,

> > > >>

> > > >> > >> > Iceberg sink and compact data through the following steps:

> > > >>

> > > >> > >> > Step-1: Some parallel data writer(sinker) to write streaming

> > > data as files.

> > > >>

> > > >> > >> > Step-2: A single parallelism data files committer to commit the

> > > completed

> > > >>

> > > >> > >> > files as soon as possible to make them available.

> > > >>

> > > >> > >> > Step-3: Some parallel file rewriter(compactor) to collect

> > > committed files

> > > >>

> > > >> > >> > from multiple checkpoints, and rewriter(compact) them together

> > > once the

> > > >>

> > > >> > >> > total file size or number of files reach the threshold.

> > > >>

> > > >> > >> > Step-4: A single parallelism rewrite(compact) result committer

> > > to commit

> > > >>

> > > >> > >> > the rewritten(compacted) files to replace the old files and make

> > > them

> > > >>

> > > >> > >> > available.

> > > >>

> > > >> > >> >

> > > >>

> > > >> > >> >

> > > >>

> > > >> > >> > If Flink want to support small file compaction, some key point I

> > > think is

> > > >>

> > > >> > >> > necessary:

> > > >>

> > > >> > >> >

> > > >>

> > > >> > >> > 1, Compact files from multiple checkpoints.

> > > >>

> > > >> > >> > I totally agree with Jingsong, because completed file size

> > > usually could

> > > >>

> > > >> > >> > not reach the threshold in a single checkpoint. Especially for

> > > partitioned

> > > >>

> > > >> > >> > table, we need to compact the files of each partition, but

> > > usually the file

> > > >>

> > > >> > >> > size of each partition will be different and may not reach the

> > > merge

> > > >>

> > > >> > >> > threshold. If we compact these files, in a single checkpoint,

> > > regardless of

> > > >>

> > > >> > >> > whether the total file size reaches the threshold, then the

> > > value of

> > > >>

> > > >> > >> > compacting will be diminished and we will still get small files

> > > because

> > > >>

> > > >> > >> > these compacted files are not reach to target size. So we need

> > > the

> > > >>

> > > >> > >> > compactor to collect committed files from multiple checkpoints

> > > and compact

> > > >>

> > > >> > >> > them until they reach the threshold.

> > > >>

> > > >> > >> >

> > > >>

> > > >> > >> > 2, Separate write phase and compact phase.

> > > >>

> > > >> > >> > Users usually hope the data becomes available as soon as

> > > possible, and the

> > > >>

> > > >> > >> > end-to-end latency is very important. I think we need to

> > > separate the

> > > >>

> > > >> > >> > write and compact phase. For the write phase, there include the

> > > Step-1

> > > >>

> > > >> > >> > and Step-2, we sink data as file and commit it pre checkpoint

> > > and regardless

> > > >>

> > > >> > >> > of whether the file size it is. That could ensure the data will

> > > be

> > > >>

> > > >> > >> > available ASAP. For the compact phase, there include the Step-3

> > > >>

> > > >> > >> > and Step-4, the compactor should collect committed files from

> > > multiple

> > > >>

> > > >> > >> > checkpoints and compact them asynchronously once they reach the

> > > threshold,

> > > >>

> > > >> > >> > and the compact committer will commit the compaction result in

> > > the next

> > > >>

> > > >> > >> > checkpoint. We compact the committed files asynchronously

> > > because we don't

> > > >>

> > > >> > >> > want the compaction to affect the data sink or the whole

> > > pipeline.

> > > >>

> > > >> > >> >

> > > >>

> > > >> > >> > 3, Exactly once guarantee between write and compact phase.

> > > >>

> > > >> > >> > Once we separate write phase and compact phase, we need to

> > > consider

> > > >>

> > > >> > >> > how to guarantee

> > > >>

> > > >> > >> > the exact once semantic between two phases. We should not lose

> > > any data or

> > > >>

> > > >> > >> > files on the compactor(Step-3) in any case and cause the

> > > compaction result

> > > >>

> > > >> > >> > to be inconsistent with before. I think flink should provide an

> > > easy-to-use

> > > >>

> > > >> > >> > interface to make that easier.

> > > >>

> > > >> > >> >

> > > >>

> > > >> > >> > 4, Metadata operation and compaction result validation.

> > > >>

> > > >> > >> > In the compact phase, there may be not only compact files, but

> > > also a lot

> > > >>

> > > >> > >> > of metadata operations, such as the iceberg needing to

> > > read/write manifest

> > > >>

> > > >> > >> > and do MOR. And we need some interface to support users to do

> > > some

> > > >>

> > > >> > >> > validation of the compaction result. I think these points should

> > > be

> > > >>

> > > >> > >> > considered when we design the compaction API.

> > > >>

> > > >> > >> >

> > > >>

> > > >> > >> >

> > > >>

> > > >> > >> > Back to FLIP-191, option 1 looks very complicated while option 2

> > > is

> > > >>

> > > >> > >> > relatively simple, but neither of these two solutions separates

> > > the write

> > > >>

> > > >> > >> > phase from the compact phase. So I think we should consider the

> > > points I

> > > >>

> > > >> > >> > mentioned above. And if you have any other questions you can

> > > always feel

> > > >>

> > > >> > >> > free to reach out to me!

> > > >>

> > > >> > >> >

> > > >>

> > > >> > >> > BR,

> > > >>

> > > >> > >> > Reo

> > > >>

> > > >> > >> >

> > > >>

> > > >> > >> > Fabian Paul 于2021年11月8日周一 下午7:59写道:

> > > >>

> > > >> > >> >

> > > >>

> > > >> > >> > > Hi all,

> > > >>

> > > >> > >> > >

> > > >>

> > > >> > >> > > Thanks for the lively discussions. I am really excited to see

> > > so many

> > > >>

> > > >> > >> > > people

> > > >>

> > > >> > >> > > participating in this thread. It also underlines the need that

> > > many people

> > > >>

> > > >> > >> > > would

> > > >>

> > > >> > >> > > like to see a solution soon.

> > > >>

> > > >> > >> > >

> > > >>

> > > >> > >> > > I have updated the FLIP and removed the parallelism

> > > configuration because

> > > >>

> > > >> > >> > > it is

> > > >>

> > > >> > >> > > unnecessary since users can configure a constant exchange key

> > > to send all

> > > >>

> > > >> > >> > > committables to only one committable aggregator.

> > > >>

> > > >> > >> > >

> > > >>

> > > >> > >> > >

> > > >>

> > > >> > >> > > 1. Burden for developers w.r.t batch stream unification.

> > > >>

> > > >> > >> > >

> > > >>

> > > >> > >> > > @yun @guowei, from a theoretical point you are right about

> > > exposing the

> > > >>

> > > >> > >> > > DataStream

> > > >>

> > > >> > >> > > API in the sink users have the full power to write correct

> > > batch and

> > > >>

> > > >> > >> > > streaming

> > > >>

> > > >> > >> > > sinks. I think in reality a lot of users still struggle to

> > > build pipelines

> > > >>

> > > >> > >> > > with

> > > >>

> > > >> > >> > > i.e. the operator pipeline which works correct in streaming

> > > and batch mode.

> > > >>

> > > >> > >> > > Another problem I see is by exposing more deeper concepts is

> > > that we

> > > >>

> > > >> > >> > > cannot do

> > > >>

> > > >> > >> > > any optimization because we cannot reason about how sinks are

> > > built in the

> > > >>

> > > >> > >> > > future.

> > > >>

> > > >> > >> > >

> > > >>

> > > >> > >> > > We should also try to steer users towards using only

> > > `Functions` to give

> > > >>

> > > >> > >> > > us more

> > > >>

> > > >> > >> > > flexibility to swap the internal operator representation. I

> > > agree with

> > > >>

> > > >> > >> > > @yun we

> > > >>

> > > >> > >> > > should try to make the `ProcessFunction` more versatile to

> > > work on that

> > > >>

> > > >> > >> > > goal but

> > > >>

> > > >> > >> > > I see this as unrelated to the FLIP.

> > > >>

> > > >> > >> > >

> > > >>

> > > >> > >> > >

> > > >>

> > > >> > >> > > 2. Regarding Commit / Global commit

> > > >>

> > > >> > >> > >

> > > >>

> > > >> > >> > > I envision the global committer to be specific depending on

> > > the data lake

> > > >>

> > > >> > >> > > solution you want to write to. However, it is entirely

> > > orthogonal to the

> > > >>

> > > >> > >> > > compaction.

> > > >>

> > > >> > >> > > Currently, I do not expect any changes w.r.t the Global commit

> > > introduces

> > > >>

> > > >> > >> > > by

> > > >>

> > > >> > >> > > this FLIP.

> > > >>

> > > >> > >> > >

> > > >>

> > > >> > >> > >

> > > >>

> > > >> > >> > > 3. Regarding the case of trans-checkpoints merging

> > > >>

> > > >> > >> > >

> > > >>

> > > >> > >> > > @yun, as user, I would expect that if the committer receives

> > > in a

> > > >>

> > > >> > >> > > checkpoint files

> > > >>

> > > >> > >> > > to merge/commit that these are also finished when the

> > > checkpoint finishes.

> > > >>

> > > >> > >> > > I think all sinks rely on this principle currently i.e.,

> > > KafkaSink needs to

> > > >>

> > > >> > >> > > commit all open transactions until the next checkpoint can

> > > happen.

> > > >>

> > > >> > >> > >

> > > >>

> > > >> > >> > > Maybe in the future, we can somehow move the Committer#commit

> > > call to an

> > > >>

> > > >> > >> > > asynchronous execution, but we should discuss it as a separate

> > > thread.

> > > >>

> > > >> > >> > >

> > > >>

> > > >> > >> > > > We probably should first describe the different causes of

> > > small files and

> > > >>

> > > >> > >> > > > what problems was this proposal trying to solve. I wrote a

> > > data shuffling

> > > >>

> > > >> > >> > > > proposal [1] for Flink Iceberg sink (shared with Iceberg

> > > community [2]).

> > > >>

> > > >> > >> > > It

> > > >>

> > > >> > >> > > > can address small files problems due to skewed data

> > > distribution across

> > > >>

> > > >> > >> > > > Iceberg table partitions. Streaming shuffling before writers

> > > (to files)

> > > >>

> > > >> > >> > > is

> > > >>

> > > >> > >> > > > typically more efficient than post-write file compaction

> > > (which involves

> > > >>

> > > >> > >> > > > read-merge-write). It is usually cheaper to prevent a

> > > problem (small

> > > >>

> > > >> > >> > > files)

> > > >>

> > > >> > >> > > > than fixing it.

> > > >>

> > > >> > >> > >

> > > >>

> > > >> > >> > >

> > > >>

> > > >> > >> > > @steven you are raising a good point, although I think only

> > > using a

> > > >>

> > > >> > >> > > customizable

> > > >>

> > > >> > >> > > shuffle won't address the generation of small files. One

> > > assumption is that

> > > >>

> > > >> > >> > > at least the sink generates one file per subtask, which can

> > > already be too

> > > >>

> > > >> > >> > > many.

> > > >>

> > > >> > >> > > Another problem is that with low checkpointing intervals, the

> > > files do not

> > > >>

> > > >> > >> > > meet

> > > >>

> > > >> > >> > > the required size. The latter point is probably addressable by

> > > changing the

> > > >>

> > > >> > >> > > checkpoint interval, which might be inconvenient for some

> > > users.

> > > >>

> > > >> > >> > >

> > > >>

> > > >> > >> > > > The sink coordinator checkpoint problem (mentioned in option

> > > 1) would be

> > > >>

> > > >> > >> > > > great if Flink can address it. In the spirit of source

> > > >>

> > > >> > >> > > (enumerator-reader)

> > > >>

> > > >> > >> > > > and sink (writer-coordinator) duality, sink coordinator

> > > checkpoint should

> > > >>

> > > >> > >> > > > happen after the writer operator. This would be a natural

> > > fit to support

> > > >>

> > > >> > >> > > > global committer in FLIP-143. It is probably an orthogonal

> > > matter to this

> > > >>

> > > >> > >> > > > proposal.

> > > >>

> > > >> > >> > >

> > > >>

> > > >> > >> > >

> > > >>

> > > >> > >> > > To me the question here is what are the benefits of having a

> > > coordinator in

> > > >>

> > > >> > >> > > comparison to a global committer/aggregator operator.

> > > >>

> > > >> > >> > >

> > > >>

> > > >> > >> > > > Personally, I am usually in favor of keeping streaming

> > > ingestion (to data

> > > >>

> > > >> > >> > > > lake) relatively simple and stable. Also sometimes

> > > compaction and sorting

> > > >>

> > > >> > >> > > > are performed together in data rewrite maintenance jobs to

> > > improve read

> > > >>

> > > >> > >> > > > performance. In that case, the value of compacting (in Flink

> > > streaming

> > > >>

> > > >> > >> > > > ingestion) diminishes.

> > > >>

> > > >> > >> > >

> > > >>

> > > >> > >> > >

> > > >>

> > > >> > >> > > I agree it is always possible to have scheduled maintenance

> > > jobs keeping

> > > >>

> > > >> > >> > > care of

> > > >>

> > > >> > >> > > your data i.e., doing compaction. Unfortunately, the downside

> > > is that you

> > > >>

> > > >> > >> > > have to your data after it is already available for other

> > > downstream

> > > >>

> > > >> > >> > > consumers.

> > > >>

> > > >> > >> > > I guess this can lead to all kinds of visibility problems. I

> > > am also

> > > >>

> > > >> > >> > > surprised that

> > > >>

> > > >> > >> > > you personally are a fan of this approach and, on the other

> > > hand, are

> > > >>

> > > >> > >> > > developing

> > > >>

> > > >> > >> > > the Iceberg sink, which goes somewhat against your mentioned

> > > principle of

> > > >>

> > > >> > >> > > keeping

> > > >>

> > > >> > >> > > the sink simple.

> > > >>

> > > >> > >> > >

> > > >>

> > > >> > >> > > > Currently, it is unclear from the doc and this thread where

> > > the

> > > >>

> > > >> > >> > > compaction

> > > >>

> > > >> > >> > > > is actually happening. Jingsong's reply described one model

> > > >>

> > > >> > >> > > > writer (parallel) -> aggregator (single-parallelism

> > > compaction planner)

> > > >>

> > > >> > >> > > ->

> > > >>

> > > >> > >> > > > compactor (parallel) -> global committer (single-parallelism)

> > > >>

> > > >> > >> > >

> > > >>

> > > >> > >> > >

> > > >>

> > > >> > >> > > My idea of the topology is very similar to the one outlined by

> > > Jinsong. The

> > > >>

> > > >> > >> > > compaction will happen in the committer operator.

> > > >>

> > > >> > >> > >

> > > >>

> > > >> > >> > > >

> > > >>

> > > >> > >> > > > In the Iceberg community, the following model has been

> > > discussed. It is

> > > >>

> > > >> > >> > > > better for Iceberg because it won't delay the data

> > > availability.

> > > >>

> > > >> > >> > > > writer (parallel) -> global committer for append (single

> > > parallelism) ->

> > > >>

> > > >> > >> > > > compactor (parallel) -> global committer for rewrite commit

> > > (single

> > > >>

> > > >> > >> > > > parallelism)

> > > >>

> > > >> > >> > >

> > > >>

> > > >> > >> > >

> > > >>

> > > >> > >> > > From a quick glimpse, it seems that the exact same topology is

> > > possible to

> > > >>

> > > >> > >> > > express with the committable aggregator, but this definitely

> > > depends on

> > > >>

> > > >> > >> > > the exact

> > > >>

> > > >> > >> > > setup.

> > > >>

> > > >> > >> > >

> > > >>

> > > >> > >> > > Best,

> > > >>

> > > >> > >> > > Fabian

> > > >>

> > > >> > >>

> > >

>

>

Re: Re: [DISCUSS] FLIP-191: Extend unified Sink interface to support small file compaction

Posted by Fabian Paul <fp...@apache.org>.
Hi Yun,

Thanks for summarizing the two issues.

1. Losing intermediate shuffle data in batch mode

I fully agree with your analysis. We will start to mitigate the
problem by introducing the blocking exchanges and documenting that it
will not prevent duplication in case a complete taskmanager is lost.
In the future, we may introduce some persistent HA storage to write
these intermediate results to but I think this should not be part of
this FLIP. One other option is to use a remote shuffle service that
can persist the results. In the end, probably the final solution
requires some kind of persistent storage.

2. Cascading checkpoints in the post-commit topology for the final checkpoint

One thing I'd like to mention is that this problem already exists with
the current implementation of the global committer so we do not worsen
the situation. I do not think we need multiple cascading commits but
only a single additional checkpoint for the post-commit topology after
the final checkpoint. Our idea is to trigger such a checkpoint and
merge it with the previously taken result so it only appears to be one
checkpoint.
Overall, I want to fix this problem as soon as possible but I see the
other API changes are more critical and we can treat this problem as a
bug fix because it also affects the Sink V1 global committer.
For only taking a savepoint I do not see a problem that requires
additional logic because when the job is recovered the previous
committables are emitted again on the next checkpoint.

Hopefully, this answers your question. I will start the vote if there
are no other concerns since we are moving towards the feature freeze
deadline and the FLIP is targeted for Flink 1.15.

Best,
Fabian

On Tue, Dec 21, 2021 at 11:29 AM Yun Gao <yu...@aliyun.com.invalid> wrote:
>
> Hi,
>
> As Arvid appointed out, with some more checks, if a job vertex have multiple downstream vertices and
> they are all connected via blocking edges, they should not affect each other, as long as we could
> ensure the intermediate shuffle data do not get lost, the finished precedent job vertex not restarted.
> Thus as a whole, for 2pc under batch mode, we should be able to ensure exactly-once with
> JM Failover + HA result partition + all-to-all blocking edge before the committer. For the first step,
> we could first have all-to-all blocking edge to reduce the probability of repeat data.
>
> And for the post-committer topology, sorry it reminds me another issue we ever discussed
> before: for the streaming mode, we should not be able to support cascade commit since now
> emit records in notifyCheckpointComplete is not supported after FLIP-147. Thus I think the
> committables we emit from the committer to the post-committer topology should be before
> they actually committed, right? If it is the case, perhaps we either document the situation clearly
> and if one job requires taking actions after the committables get committed, he could further check
> if the committables are committed manully(like scanning the filesystem to see if the files are renamed),
> or we might postpone supporting the post-committer topology until we have further determined the
> semantics and solutions ?
>
> Best,
> Yun
>
>
>
> ------------------------------------------------------------------
> From:Arvid Heise <ar...@apache.org>
> Send Time:2021 Dec. 16 (Thu.) 21:48
> To:Yun Gao <yu...@aliyun.com>
> Cc:dev <de...@flink.apache.org>
> Subject:Re: Re: [DISCUSS] FLIP-191: Extend unified Sink interface to support small file compaction
>
> I just noticed offline with Yun that I have some misconception on how how blocking data exchange work in Flink. Apparently, a subtask of the stage 2 is already started when all connected subtasks of stage 1 are finished. I was assuming that it works similarly to Hadoop where all tasks of stage 1 have to be finished before any task of stage 2 is started.
> So to amend the proposal, we would introduce a full shuffle before committer in batch. That would simulate the behavior of Hadoop.
>
> We also need to verify what happens in case of multiple sinks. In that case, we may have started committer of sink 1 while sink 2 is still writing. That would be fine as long as sink 1 writers are not restarted. If not then it seems as if we may improve scheduler to actually not restart "finished" tasks. It seems like an orthogonal issue though.
>
> Note that all discussed issues apply to Sink V1 as well, so the FLIP is not making anything worse.
>
>
> On Thu, Dec 16, 2021 at 1:53 PM Arvid Heise <ar...@apache.org> wrote:
> Hi Yun,
>
> the basic idea is to use different regions for the different stages.
>
> So you have your stages:
> Stage 1: <job> -> pre-writer topology -> writer -> committables (materialized)
> Stage 2: committables (materialized) -> pre-commit topology -> committer -> succesful committables (materialized)
> Stage 3: succesful committables (materialized) -> post-commit topology
>
> If any instance of a given stages fails, the whole stage is restarted.
> So, in case of the main pipeline (stage 1) fails, no data will be committed at all. On a rerun, we start fresh (or from the previous stage).
> Only, when all data has been written, we start with committing the data. An issue during committing, will retrigger the commit stage (stage 2) and only that stage. Thus, all committables are stable and remain stable.
> When we are done committing all committables, we run the post-commit topology similarly "atomically".
>
> So now the cavaets:
> - If committer is rerun, all committables are restarted. So the committer needs to be idempotent. That is the same with STREAMING mode now and afaik there is no way around it.
> - If we lose a TM during commit phase, we will run into the original issues of inconstent data as we need to rerun the whole job. That would be solved with HA storage and we haven't found any solution that doesn't require some kind of external storage. However, the commit phase should be rather fast and errors are improbable (low volume).
>
> I'd still like to have an HA storage but that issue is also in Sink V1 and kind of orthogonal. It's also nothing that we can solve without involving more folks (I'm trying to kick start that in the background).
>
> On Thu, Dec 16, 2021 at 1:31 PM Yun Gao <yu...@aliyun.com> wrote:
> Hi,
>
> Very thanks Fabian for the explanation and it solves most of the issues.
> There is one left issue I want to have a double confirmation is that for
> the edges between writer and committer and in the post-committer topology,
> perhaps the result partition with HA storage is not enough solve all the issues
> directly ? It is due to after the committer and post-committer topology is finished
> and the data is committed, it might still be restarted due to JM failover and the
> deterministic problem (namely the example of  (A -> [B1, B2], A, B1 have finished and
> B2 failed, if -> is rebalance / random / rescale,  all of A, B1, B2 would restarted). Then
> the records would be produced and created for the second times.
>
> We might let the writers to skip producing the new records, but if we have multiple sinks like
> OP1 -> (writer 1 -> committer 1)
>      |--> (writer 2 -> committer 2)
>
> and the failover happens after writer 1 & committer 1 get finished but writer 2 is running,
> if op1 produced different records across the two runs, then the two sinks would produces
> different data, which might be not suitable in some cases. Perhaps we need some support
> from the scheduler side?
>
> But I also agree this could be a separate issue and we could solve it separately in some future
> as long as we know how to solve it~
>
> Best,
> Yun
>
>
> ------------------------------------------------------------------
> From:Arvid Heise <ar...@apache.org>
> Send Time:2021 Dec. 16 (Thu.) 19:54
> To:dev <de...@flink.apache.org>
> Cc:Yun Gao <yu...@aliyun.com>
> Subject:Re: Re: [DISCUSS] FLIP-191: Extend unified Sink interface to support small file compaction
>
> Just a quick amend: There will be no blocking exchange in the pre-writer exchange for performance reasons.
> After the writer, we have tiny data volume and are free to add as many as we see necessary.
>
> On Thu, Dec 16, 2021 at 11:18 AM Fabian Paul <fp...@apache.org> wrote:
> Hi Till,
>
>  Good point, in the scenario with the blocking keyed exchange between
>  the writer and committer my idea is to make the committer effectively
>  the global committer. With Sink V2 there is no real difference anymore
>  between the committer and global committer.
>  You are right that everything after the committer would be part of the
>  same failover region but we plan to insert a blocking exchange by
>  default before all of the custom topologies.
>
>  Best,
>  Fabian
>
>  On Thu, Dec 16, 2021 at 11:08 AM Till Rohrmann <tr...@apache.org> wrote:
>  >
>  > Hi Fabian,
>  >
>  > quick question on your comment 3. If there is a pipelined data exchange
>  > with a keyBy between the writers/committers and the component that does the
>  > global commit, then there will only be a single failover region. So is it
>  > correct that you assumed blocking data exchanges for the scenario you
>  > described?
>  >
>  > Cheers,
>  > Till
>  >
>  > On Thu, Dec 16, 2021 at 9:23 AM Fabian Paul <fp...@apache.org> wrote:
>  >
>  > > Hi Yun,
>  > >
>  > > Thanks for your fast feedback. Let me clarify your points.
>  > >
>  > > 1. We solve it by using StreamExchangeMode.BATCH before any exchange.
>  > > That obviously doesn’t help with lost TM but we would need to employ
>  > > HA storage for that. Same issue as now and orthogonal.
>  > >
>  > > 2. Extending V1 with V2 or vice versa would require renames of methods
>  > > (since return types are non-optional) and is making API changes. Even
>  > > though Experimental, we want to give connector developers the
>  > > opportunity to provide 1 implementation for all of Flink 1.X. We will
>  > > offer an internal adapter from V1 to V2, 2 sinkTo , and internally
>  > > just have one code-path.
>  > >
>  > > 3. DataStreamSink would act as a unified view on all the operators and
>  > > update them all at once when using setParallelism and so on (setName
>  > > and setUid will receive suffixes per operator).
>  > > Iceberg actually has a different requirement: They want to have a
>  > > committer with parallelism 1 but as a coordinator such that
>  > > embarrassingly parallel pipelines have different fail-over regions. I
>  > > was thinking that in this case, they need to implement a no-op
>  > > committer (that just forwards the committables) and use a post-commit
>  > > topology that achieves that.
>  > > Another option is that they use the preCommit topology and insert a
>  > > constant key-by that forwards all committables to a single committer.
>  > > We are planning to provide building blocks for such pipelines as we
>  > > go.
>  > >
>  > > Best,
>  > > Fabian
>  > >
>  > > On Thu, Dec 16, 2021 at 5:50 AM Yun Gao <yu...@aliyun.com> wrote:
>  > > >
>  > > > Hi Fabian,
>  > > >
>  > > > Very thanks for the update! I think the latest version in general looks
>  > > good from my side
>  > > > and I think using separate feature interface would be much more easy to
>  > > understand
>  > > > and extend in the future. I have some pending issues on the details
>  > > though:
>  > > >
>  > > > 1. The first one is if we could support end-to-end exactly-once with
>  > > post-committing
>  > > > topology in the batch mode ? Since for the batch mode, currently we
>  > > could only commit
>  > > >  all the transactions after the whole job is finished, otherwise if
>  > > there are JM failover or the
>  > > > writer / committer get restarted due to indeterminstic (A -> [B1, B2],
>  > > A, B1 have finished and
>  > > >  B2 failed, if -> is rebalance / random / rescale,  all of A, B1, B2
>  > > would restarted), there might
>  > > > be repeat records. Previously one possible thought is to move committer
>  > > and global committer
>  > > >  to the operator coordinator, but if it is a topology, we might need
>  > > some other kind of solutions?
>  > > >
>  > > > 2. I also want to have a dobule confirmation with the compatibility:
>  > > since the old sink is also named
>  > > > with Sink, do we want to put the Sink v2 in a new package ? Besides,
>  > > since we might want to keep
>  > > > only have one `sinkTo(Sink<?> sink)` , perhaps we also need to make the
>  > > Sink v1 to be a subclass of
>  > > > Sink v2 and extends the stateful and two-phase-commit sinks, right?
>  > > >
>  > > > 3. I'd like also have a confirmation on ours thoughts with the
>  > > `DataStreamSink` returned by the sinkTo method:
>  > > > The main issue is how do we implement the method like `setParallelism`
>  > > or `setMaxParallelism` since now the sink
>  > > > would be translated to multiple transformations? perhaps we could make
>  > > it the default values for all the transformations
>  > > > for the sink? A related issue would be for iceberg sink, I think it
>  > > would need to have only one committer to avoid the
>  > > > competition of the optimistic locks (which would cause performance
>  > > degradation), then it might need to have N writers
>  > > > with 1 committers, to build such topology, perhaps we might need to add
>  > > new methods to specify the parallelism of
>  > > > the writers and committers separately?
>  > > >
>  > > > Best,
>  > > > Yun
>  > > >
>  > > >
>  > > > ------------------Original Mail ------------------
>  > > > Sender:Fabian Paul <fp...@apache.org>
>  > > > Send Date:Mon Dec 13 23:59:43 2021
>  > > > Recipients:dev <de...@flink.apache.org>
>  > > > Subject:Re: [DISCUSS] FLIP-191: Extend unified Sink interface to support
>  > > small file compaction
>  > > >>
>  > > >> Hi all,
>  > > >>
>  > > >>
>  > > >>
>  > > >> After a lot of discussions with different, we received very fruitful
>  > > >>
>  > > >> feedback and reworked the ideas behind this FLIP. Initially, we had
>  > > >>
>  > > >> the impression that the compaction problem is solvable by a single
>  > > >>
>  > > >> topology that we can reuse across different sinks. We now have a
>  > > >>
>  > > >> better understanding that different external systems require different
>  > > >>
>  > > >> compaction mechanism i.e. Hive requires compaction before finally
>  > > >>
>  > > >> registering the file in the metastore or Iceberg compacts the files
>  > > >>
>  > > >> after they have been registered and just lazily compacts them.
>  > > >>
>  > > >>
>  > > >>
>  > > >> Considering all these different views we came up with a design that
>  > > >>
>  > > >> builds upon what @guowei.mgw@gmail.com and @yungao.gy@aliyun.com have
>  > > >>
>  > > >> proposed at the beginning. We allow inserting custom topologies before
>  > > >>
>  > > >> and after the SinkWriters and Committers. Furthermore, we do not see
>  > > >>
>  > > >> it as a downside. The Sink interfaces that will expose the DataStream
>  > > >>
>  > > >> to the user reside in flink-streaming-java in contrast to the basic
>  > > >>
>  > > >> Sink interfaces that reside fin flink-core deem it to be only used by
>  > > >>
>  > > >> expert users.
>  > > >>
>  > > >>
>  > > >>
>  > > >> Moreover, we also wanted to remove the global committer from the
>  > > >>
>  > > >> unified Sink interfaces and replace it with a custom post-commit
>  > > >>
>  > > >> topology. Unfortunately, we cannot do it without breaking the Sink
>  > > >>
>  > > >> interface since the GlobalCommittables are part of the parameterized
>  > > >>
>  > > >> Sink interface. Thus, we propose building a new Sink V2 interface
>  > > >>
>  > > >> consisting of composable interfaces that do not offer the
>  > > >>
>  > > >> GlobalCommitter anymore. We will implement a utility to extend a Sink
>  > > >>
>  > > >> with post topology that mimics the behavior of the GlobalCommitter.
>  > > >>
>  > > >> The new Sink V2 provides the same sort of methods as the Sink V1
>  > > >>
>  > > >> interface, so a migration of sinks that do not use the GlobalCommitter
>  > > >>
>  > > >> should be very easy.
>  > > >>
>  > > >> We plan to keep the existing Sink V1 interfaces to not break
>  > > >>
>  > > >> externally built sinks. As part of this FLIP, we migrate all the
>  > > >>
>  > > >> connectors inside of the main repository to the new Sink V2 API.
>  > > >>
>  > > >>
>  > > >>
>  > > >> The FLIP document is also updated and includes the proposed changes.
>  > > >>
>  > > >>
>  > > >>
>  > > >> Looking forward to your feedback,
>  > > >>
>  > > >> Fabian
>  > > >>
>  > > >>
>  > > >>
>  > > >>
>  > > https://cwiki.apache.org/confluence/display/FLINK/FLIP-191%3A+Extend+unified+Sink+interface+to+support+small+file+compaction
>  > > >>
>  > > >>
>  > > >>
>  > > >>
>  > > >>
>  > > >> On Thu, Dec 2, 2021 at 10:15 AM Roman Khachatryan wrote:
>  > > >>
>  > > >> >
>  > > >>
>  > > >> > Thanks for clarifying (I was initially confused by merging state files
>  > > >>
>  > > >> > rather than output files).
>  > > >>
>  > > >> >
>  > > >>
>  > > >> > > At some point, Flink will definitely have some WAL adapter that can
>  > > turn any sink into an exactly-once sink (with some caveats). For now, we
>  > > keep that as an orthogonal solution as it has a rather high price (bursty
>  > > workload with high latency). Ideally, we can keep the compaction
>  > > asynchronously...
>  > > >>
>  > > >> >
>  > > >>
>  > > >> > Yes, that would be something like a WAL. I agree that it would have a
>  > > >>
>  > > >> > different set of trade-offs.
>  > > >>
>  > > >> >
>  > > >>
>  > > >> >
>  > > >>
>  > > >> > Regards,
>  > > >>
>  > > >> > Roman
>  > > >>
>  > > >> >
>  > > >>
>  > > >> > On Mon, Nov 29, 2021 at 3:33 PM Arvid Heise wrote:
>  > > >>
>  > > >> > >>
>  > > >>
>  > > >> > >> > One way to avoid write-read-merge is by wrapping SinkWriter with
>  > > >>
>  > > >> > >> > another one, which would buffer input elements in a temporary
>  > > storage
>  > > >>
>  > > >> > >> > (e.g. local file) until a threshold is reached; after that, it
>  > > would
>  > > >>
>  > > >> > >> > invoke the original SinkWriter. And if a checkpoint barrier
>  > > comes in
>  > > >>
>  > > >> > >> > earlier, it would send written data to some aggregator.
>  > > >>
>  > > >> > >>
>  > > >>
>  > > >> > >> I think perhaps this seems to be a kind of WAL method? Namely we
>  > > first
>  > > >>
>  > > >> > >> write the elements to some WAL logs and persist them on checkpoint
>  > > >>
>  > > >> > >> (in snapshot or remote FS), or we directly write WAL logs to the
>  > > remote
>  > > >>
>  > > >> > >> FS eagerly.
>  > > >>
>  > > >> > >>
>  > > >>
>  > > >> > > At some point, Flink will definitely have some WAL adapter that can
>  > > turn any sink into an exactly-once sink (with some caveats). For now, we
>  > > keep that as an orthogonal solution as it has a rather high price (bursty
>  > > workload with high latency). Ideally, we can keep the compaction
>  > > asynchronously...
>  > > >>
>  > > >> > >
>  > > >>
>  > > >> > > On Mon, Nov 29, 2021 at 8:52 AM Yun Gao wrote:
>  > > >>
>  > > >> > >>
>  > > >>
>  > > >> > >> Hi,
>  > > >>
>  > > >> > >>
>  > > >>
>  > > >> > >> @Roman very sorry for the late response for a long time,
>  > > >>
>  > > >> > >>
>  > > >>
>  > > >> > >> > Merging artifacts from multiple checkpoints would apparently
>  > > >>
>  > > >> > >> require multiple concurrent checkpoints
>  > > >>
>  > > >> > >>
>  > > >>
>  > > >> > >> I think it might not need concurrent checkpoints: suppose some
>  > > >>
>  > > >> > >> operators (like the committer aggregator in the option 2) maintains
>  > > >>
>  > > >> > >> the list of files to merge, it could stores the lists of files to
>  > > merge
>  > > >>
>  > > >> > >> in the states, then after several checkpoints are done and we have
>  > > >>
>  > > >> > >> enough files, we could merge all the files in the list.
>  > > >>
>  > > >> > >>
>  > > >>
>  > > >> > >> > Asynchronous merging in an aggregator would require some
>  > > resolution
>  > > >>
>  > > >> > >> > logic on recovery, so that a merged artifact can be used if the
>  > > >>
>  > > >> > >> > original one was deleted. Otherwise, wouldn't recovery fail
>  > > because
>  > > >>
>  > > >> > >> > some artifacts are missing?
>  > > >>
>  > > >> > >> > We could also defer deletion until the "compacted" checkpoint is
>  > > >>
>  > > >> > >> > subsumed - but isn't it too late, as it will be deleted anyways
>  > > once
>  > > >>
>  > > >> > >> > subsumed?
>  > > >>
>  > > >> > >>
>  > > >>
>  > > >> > >> I think logically we could delete the original files once the
>  > > "compacted" checkpoint
>  > > >>
>  > > >> > >> (which finish merging the compacted files and record it in the
>  > > checkpoint) is completed
>  > > >>
>  > > >> > >> in all the options. If there are failover before we it, we could
>  > > restart the merging and if
>  > > >>
>  > > >> > >> there are failover after it, we could have already recorded the
>  > > files in the checkpoint.
>  > > >>
>  > > >> > >>
>  > > >>
>  > > >> > >> > One way to avoid write-read-merge is by wrapping SinkWriter with
>  > > >>
>  > > >> > >> > another one, which would buffer input elements in a temporary
>  > > storage
>  > > >>
>  > > >> > >> > (e.g. local file) until a threshold is reached; after that, it
>  > > would
>  > > >>
>  > > >> > >> > invoke the original SinkWriter. And if a checkpoint barrier
>  > > comes in
>  > > >>
>  > > >> > >> > earlier, it would send written data to some aggregator.
>  > > >>
>  > > >> > >>
>  > > >>
>  > > >> > >> I think perhaps this seems to be a kind of WAL method? Namely we
>  > > first
>  > > >>
>  > > >> > >> write the elements to some WAL logs and persist them on checkpoint
>  > > >>
>  > > >> > >> (in snapshot or remote FS), or we directly write WAL logs to the
>  > > remote
>  > > >>
>  > > >> > >> FS eagerly.
>  > > >>
>  > > >> > >>
>  > > >>
>  > > >> > >> Sorry if I do not understand correctly somewhere.
>  > > >>
>  > > >> > >>
>  > > >>
>  > > >> > >> Best,
>  > > >>
>  > > >> > >> Yun
>  > > >>
>  > > >> > >>
>  > > >>
>  > > >> > >>
>  > > >>
>  > > >> > >> ------------------------------------------------------------------
>  > > >>
>  > > >> > >> From:Roman Khachatryan
>  > > >>
>  > > >> > >> Send Time:2021 Nov. 9 (Tue.) 22:03
>  > > >>
>  > > >> > >> To:dev
>  > > >>
>  > > >> > >> Subject:Re: [DISCUSS] FLIP-191: Extend unified Sink interface to
>  > > support small file compaction
>  > > >>
>  > > >> > >>
>  > > >>
>  > > >> > >> Hi everyone,
>  > > >>
>  > > >> > >>
>  > > >>
>  > > >> > >> Thanks for the proposal and the discussion, I have some remarks:
>  > > >>
>  > > >> > >> (I'm not very familiar with the new Sink API but I thought about
>  > > the
>  > > >>
>  > > >> > >> same problem in context of the changelog state backend)
>  > > >>
>  > > >> > >>
>  > > >>
>  > > >> > >> 1. Merging artifacts from multiple checkpoints would apparently
>  > > >>
>  > > >> > >> require multiple concurrent checkpoints (otherwise, a new
>  > > checkpoint
>  > > >>
>  > > >> > >> won't be started before completing the previous one; and the
>  > > previous
>  > > >>
>  > > >> > >> one can't be completed before durably storing the artifacts).
>  > > However,
>  > > >>
>  > > >> > >> concurrent checkpoints are currently not supported with Unaligned
>  > > >>
>  > > >> > >> checkpoints (this is besides increasing e2e-latency).
>  > > >>
>  > > >> > >>
>  > > >>
>  > > >> > >> 2. Asynchronous merging in an aggregator would require some
>  > > resolution
>  > > >>
>  > > >> > >> logic on recovery, so that a merged artifact can be used if the
>  > > >>
>  > > >> > >> original one was deleted. Otherwise, wouldn't recovery fail because
>  > > >>
>  > > >> > >> some artifacts are missing?
>  > > >>
>  > > >> > >> We could also defer deletion until the "compacted" checkpoint is
>  > > >>
>  > > >> > >> subsumed - but isn't it too late, as it will be deleted anyways
>  > > once
>  > > >>
>  > > >> > >> subsumed?
>  > > >>
>  > > >> > >>
>  > > >>
>  > > >> > >> 3. Writing small files, then reading and merging them for *every*
>  > > >>
>  > > >> > >> checkpoint seems worse than only reading them on recovery. I guess
>  > > I'm
>  > > >>
>  > > >> > >> missing some cases of reading, so to me it would make sense to
>  > > mention
>  > > >>
>  > > >> > >> these cases explicitly in the FLIP motivation section.
>  > > >>
>  > > >> > >>
>  > > >>
>  > > >> > >> 4. One way to avoid write-read-merge is by wrapping SinkWriter with
>  > > >>
>  > > >> > >> another one, which would buffer input elements in a temporary
>  > > storage
>  > > >>
>  > > >> > >> (e.g. local file) until a threshold is reached; after that, it
>  > > would
>  > > >>
>  > > >> > >> invoke the original SinkWriter. And if a checkpoint barrier comes
>  > > in
>  > > >>
>  > > >> > >> earlier, it would send written data to some aggregator. It will
>  > > >>
>  > > >> > >> increase checkpoint delay (async phase) compared to the current
>  > > Flink;
>  > > >>
>  > > >> > >> but not compared to the write-read-merge solution, IIUC.
>  > > >>
>  > > >> > >> Then such "BufferingSinkWriters" could aggregate input elements
>  > > from
>  > > >>
>  > > >> > >> each other, potentially recursively (I mean something like
>  > > >>
>  > > >> > >>
>  > > https://cwiki.apache.org/confluence/download/attachments/173082889/DSTL-DFS-DAG.png
>  > > >>
>  > > >> > >> )
>  > > >>
>  > > >> > >>
>  > > >>
>  > > >> > >> 5. Reducing the number of files by reducing aggregator parallelism
>  > > as
>  > > >>
>  > > >> > >> opposed to merging on reaching size threshold will likely be less
>  > > >>
>  > > >> > >> optimal and more difficult to configure. OTH, thresholds might be
>  > > more
>  > > >>
>  > > >> > >> difficult to implement and (with recursive merging) would incur
>  > > higher
>  > > >>
>  > > >> > >> latency. Maybe that's also something to decide explicitly or at
>  > > least
>  > > >>
>  > > >> > >> mention in the FLIP.
>  > > >>
>  > > >> > >>
>  > > >>
>  > > >> > >>
>  > > >>
>  > > >> > >>
>  > > >>
>  > > >> > >> Regards,
>  > > >>
>  > > >> > >> Roman
>  > > >>
>  > > >> > >>
>  > > >>
>  > > >> > >>
>  > > >>
>  > > >> > >> On Tue, Nov 9, 2021 at 5:23 AM Reo Lei wrote:
>  > > >>
>  > > >> > >> >
>  > > >>
>  > > >> > >> > Hi Fabian,
>  > > >>
>  > > >> > >> >
>  > > >>
>  > > >> > >> > Thanks for drafting the FLIP and trying to support small file
>  > > compaction. I
>  > > >>
>  > > >> > >> > think this feature is very urgent and valuable for users(at
>  > > least for me).
>  > > >>
>  > > >> > >> >
>  > > >>
>  > > >> > >> > Currently I am trying to support streaming rewrite(compact) for
>  > > Iceberg on
>  > > >>
>  > > >> > >> > PR#3323 . As Steven mentioned,
>  > > >>
>  > > >> > >> > Iceberg sink and compact data through the following steps:
>  > > >>
>  > > >> > >> > Step-1: Some parallel data writer(sinker) to write streaming
>  > > data as files.
>  > > >>
>  > > >> > >> > Step-2: A single parallelism data files committer to commit the
>  > > completed
>  > > >>
>  > > >> > >> > files as soon as possible to make them available.
>  > > >>
>  > > >> > >> > Step-3: Some parallel file rewriter(compactor) to collect
>  > > committed files
>  > > >>
>  > > >> > >> > from multiple checkpoints, and rewriter(compact) them together
>  > > once the
>  > > >>
>  > > >> > >> > total file size or number of files reach the threshold.
>  > > >>
>  > > >> > >> > Step-4: A single parallelism rewrite(compact) result committer
>  > > to commit
>  > > >>
>  > > >> > >> > the rewritten(compacted) files to replace the old files and make
>  > > them
>  > > >>
>  > > >> > >> > available.
>  > > >>
>  > > >> > >> >
>  > > >>
>  > > >> > >> >
>  > > >>
>  > > >> > >> > If Flink want to support small file compaction, some key point I
>  > > think is
>  > > >>
>  > > >> > >> > necessary:
>  > > >>
>  > > >> > >> >
>  > > >>
>  > > >> > >> > 1, Compact files from multiple checkpoints.
>  > > >>
>  > > >> > >> > I totally agree with Jingsong, because completed file size
>  > > usually could
>  > > >>
>  > > >> > >> > not reach the threshold in a single checkpoint. Especially for
>  > > partitioned
>  > > >>
>  > > >> > >> > table, we need to compact the files of each partition, but
>  > > usually the file
>  > > >>
>  > > >> > >> > size of each partition will be different and may not reach the
>  > > merge
>  > > >>
>  > > >> > >> > threshold. If we compact these files, in a single checkpoint,
>  > > regardless of
>  > > >>
>  > > >> > >> > whether the total file size reaches the threshold, then the
>  > > value of
>  > > >>
>  > > >> > >> > compacting will be diminished and we will still get small files
>  > > because
>  > > >>
>  > > >> > >> > these compacted files are not reach to target size. So we need
>  > > the
>  > > >>
>  > > >> > >> > compactor to collect committed files from multiple checkpoints
>  > > and compact
>  > > >>
>  > > >> > >> > them until they reach the threshold.
>  > > >>
>  > > >> > >> >
>  > > >>
>  > > >> > >> > 2, Separate write phase and compact phase.
>  > > >>
>  > > >> > >> > Users usually hope the data becomes available as soon as
>  > > possible, and the
>  > > >>
>  > > >> > >> > end-to-end latency is very important. I think we need to
>  > > separate the
>  > > >>
>  > > >> > >> > write and compact phase. For the write phase, there include the
>  > > Step-1
>  > > >>
>  > > >> > >> > and Step-2, we sink data as file and commit it pre checkpoint
>  > > and regardless
>  > > >>
>  > > >> > >> > of whether the file size it is. That could ensure the data will
>  > > be
>  > > >>
>  > > >> > >> > available ASAP. For the compact phase, there include the Step-3
>  > > >>
>  > > >> > >> > and Step-4, the compactor should collect committed files from
>  > > multiple
>  > > >>
>  > > >> > >> > checkpoints and compact them asynchronously once they reach the
>  > > threshold,
>  > > >>
>  > > >> > >> > and the compact committer will commit the compaction result in
>  > > the next
>  > > >>
>  > > >> > >> > checkpoint. We compact the committed files asynchronously
>  > > because we don't
>  > > >>
>  > > >> > >> > want the compaction to affect the data sink or the whole
>  > > pipeline.
>  > > >>
>  > > >> > >> >
>  > > >>
>  > > >> > >> > 3, Exactly once guarantee between write and compact phase.
>  > > >>
>  > > >> > >> > Once we separate write phase and compact phase, we need to
>  > > consider
>  > > >>
>  > > >> > >> > how to guarantee
>  > > >>
>  > > >> > >> > the exact once semantic between two phases. We should not lose
>  > > any data or
>  > > >>
>  > > >> > >> > files on the compactor(Step-3) in any case and cause the
>  > > compaction result
>  > > >>
>  > > >> > >> > to be inconsistent with before. I think flink should provide an
>  > > easy-to-use
>  > > >>
>  > > >> > >> > interface to make that easier.
>  > > >>
>  > > >> > >> >
>  > > >>
>  > > >> > >> > 4, Metadata operation and compaction result validation.
>  > > >>
>  > > >> > >> > In the compact phase, there may be not only compact files, but
>  > > also a lot
>  > > >>
>  > > >> > >> > of metadata operations, such as the iceberg needing to
>  > > read/write manifest
>  > > >>
>  > > >> > >> > and do MOR. And we need some interface to support users to do
>  > > some
>  > > >>
>  > > >> > >> > validation of the compaction result. I think these points should
>  > > be
>  > > >>
>  > > >> > >> > considered when we design the compaction API.
>  > > >>
>  > > >> > >> >
>  > > >>
>  > > >> > >> >
>  > > >>
>  > > >> > >> > Back to FLIP-191, option 1 looks very complicated while option 2
>  > > is
>  > > >>
>  > > >> > >> > relatively simple, but neither of these two solutions separates
>  > > the write
>  > > >>
>  > > >> > >> > phase from the compact phase. So I think we should consider the
>  > > points I
>  > > >>
>  > > >> > >> > mentioned above. And if you have any other questions you can
>  > > always feel
>  > > >>
>  > > >> > >> > free to reach out to me!
>  > > >>
>  > > >> > >> >
>  > > >>
>  > > >> > >> > BR,
>  > > >>
>  > > >> > >> > Reo
>  > > >>
>  > > >> > >> >
>  > > >>
>  > > >> > >> > Fabian Paul 于2021年11月8日周一 下午7:59写道:
>  > > >>
>  > > >> > >> >
>  > > >>
>  > > >> > >> > > Hi all,
>  > > >>
>  > > >> > >> > >
>  > > >>
>  > > >> > >> > > Thanks for the lively discussions. I am really excited to see
>  > > so many
>  > > >>
>  > > >> > >> > > people
>  > > >>
>  > > >> > >> > > participating in this thread. It also underlines the need that
>  > > many people
>  > > >>
>  > > >> > >> > > would
>  > > >>
>  > > >> > >> > > like to see a solution soon.
>  > > >>
>  > > >> > >> > >
>  > > >>
>  > > >> > >> > > I have updated the FLIP and removed the parallelism
>  > > configuration because
>  > > >>
>  > > >> > >> > > it is
>  > > >>
>  > > >> > >> > > unnecessary since users can configure a constant exchange key
>  > > to send all
>  > > >>
>  > > >> > >> > > committables to only one committable aggregator.
>  > > >>
>  > > >> > >> > >
>  > > >>
>  > > >> > >> > >
>  > > >>
>  > > >> > >> > > 1. Burden for developers w.r.t batch stream unification.
>  > > >>
>  > > >> > >> > >
>  > > >>
>  > > >> > >> > > @yun @guowei, from a theoretical point you are right about
>  > > exposing the
>  > > >>
>  > > >> > >> > > DataStream
>  > > >>
>  > > >> > >> > > API in the sink users have the full power to write correct
>  > > batch and
>  > > >>
>  > > >> > >> > > streaming
>  > > >>
>  > > >> > >> > > sinks. I think in reality a lot of users still struggle to
>  > > build pipelines
>  > > >>
>  > > >> > >> > > with
>  > > >>
>  > > >> > >> > > i.e. the operator pipeline which works correct in streaming
>  > > and batch mode.
>  > > >>
>  > > >> > >> > > Another problem I see is by exposing more deeper concepts is
>  > > that we
>  > > >>
>  > > >> > >> > > cannot do
>  > > >>
>  > > >> > >> > > any optimization because we cannot reason about how sinks are
>  > > built in the
>  > > >>
>  > > >> > >> > > future.
>  > > >>
>  > > >> > >> > >
>  > > >>
>  > > >> > >> > > We should also try to steer users towards using only
>  > > `Functions` to give
>  > > >>
>  > > >> > >> > > us more
>  > > >>
>  > > >> > >> > > flexibility to swap the internal operator representation. I
>  > > agree with
>  > > >>
>  > > >> > >> > > @yun we
>  > > >>
>  > > >> > >> > > should try to make the `ProcessFunction` more versatile to
>  > > work on that
>  > > >>
>  > > >> > >> > > goal but
>  > > >>
>  > > >> > >> > > I see this as unrelated to the FLIP.
>  > > >>
>  > > >> > >> > >
>  > > >>
>  > > >> > >> > >
>  > > >>
>  > > >> > >> > > 2. Regarding Commit / Global commit
>  > > >>
>  > > >> > >> > >
>  > > >>
>  > > >> > >> > > I envision the global committer to be specific depending on
>  > > the data lake
>  > > >>
>  > > >> > >> > > solution you want to write to. However, it is entirely
>  > > orthogonal to the
>  > > >>
>  > > >> > >> > > compaction.
>  > > >>
>  > > >> > >> > > Currently, I do not expect any changes w.r.t the Global commit
>  > > introduces
>  > > >>
>  > > >> > >> > > by
>  > > >>
>  > > >> > >> > > this FLIP.
>  > > >>
>  > > >> > >> > >
>  > > >>
>  > > >> > >> > >
>  > > >>
>  > > >> > >> > > 3. Regarding the case of trans-checkpoints merging
>  > > >>
>  > > >> > >> > >
>  > > >>
>  > > >> > >> > > @yun, as user, I would expect that if the committer receives
>  > > in a
>  > > >>
>  > > >> > >> > > checkpoint files
>  > > >>
>  > > >> > >> > > to merge/commit that these are also finished when the
>  > > checkpoint finishes.
>  > > >>
>  > > >> > >> > > I think all sinks rely on this principle currently i.e.,
>  > > KafkaSink needs to
>  > > >>
>  > > >> > >> > > commit all open transactions until the next checkpoint can
>  > > happen.
>  > > >>
>  > > >> > >> > >
>  > > >>
>  > > >> > >> > > Maybe in the future, we can somehow move the Committer#commit
>  > > call to an
>  > > >>
>  > > >> > >> > > asynchronous execution, but we should discuss it as a separate
>  > > thread.
>  > > >>
>  > > >> > >> > >
>  > > >>
>  > > >> > >> > > > We probably should first describe the different causes of
>  > > small files and
>  > > >>
>  > > >> > >> > > > what problems was this proposal trying to solve. I wrote a
>  > > data shuffling
>  > > >>
>  > > >> > >> > > > proposal [1] for Flink Iceberg sink (shared with Iceberg
>  > > community [2]).
>  > > >>
>  > > >> > >> > > It
>  > > >>
>  > > >> > >> > > > can address small files problems due to skewed data
>  > > distribution across
>  > > >>
>  > > >> > >> > > > Iceberg table partitions. Streaming shuffling before writers
>  > > (to files)
>  > > >>
>  > > >> > >> > > is
>  > > >>
>  > > >> > >> > > > typically more efficient than post-write file compaction
>  > > (which involves
>  > > >>
>  > > >> > >> > > > read-merge-write). It is usually cheaper to prevent a
>  > > problem (small
>  > > >>
>  > > >> > >> > > files)
>  > > >>
>  > > >> > >> > > > than fixing it.
>  > > >>
>  > > >> > >> > >
>  > > >>
>  > > >> > >> > >
>  > > >>
>  > > >> > >> > > @steven you are raising a good point, although I think only
>  > > using a
>  > > >>
>  > > >> > >> > > customizable
>  > > >>
>  > > >> > >> > > shuffle won't address the generation of small files. One
>  > > assumption is that
>  > > >>
>  > > >> > >> > > at least the sink generates one file per subtask, which can
>  > > already be too
>  > > >>
>  > > >> > >> > > many.
>  > > >>
>  > > >> > >> > > Another problem is that with low checkpointing intervals, the
>  > > files do not
>  > > >>
>  > > >> > >> > > meet
>  > > >>
>  > > >> > >> > > the required size. The latter point is probably addressable by
>  > > changing the
>  > > >>
>  > > >> > >> > > checkpoint interval, which might be inconvenient for some
>  > > users.
>  > > >>
>  > > >> > >> > >
>  > > >>
>  > > >> > >> > > > The sink coordinator checkpoint problem (mentioned in option
>  > > 1) would be
>  > > >>
>  > > >> > >> > > > great if Flink can address it. In the spirit of source
>  > > >>
>  > > >> > >> > > (enumerator-reader)
>  > > >>
>  > > >> > >> > > > and sink (writer-coordinator) duality, sink coordinator
>  > > checkpoint should
>  > > >>
>  > > >> > >> > > > happen after the writer operator. This would be a natural
>  > > fit to support
>  > > >>
>  > > >> > >> > > > global committer in FLIP-143. It is probably an orthogonal
>  > > matter to this
>  > > >>
>  > > >> > >> > > > proposal.
>  > > >>
>  > > >> > >> > >
>  > > >>
>  > > >> > >> > >
>  > > >>
>  > > >> > >> > > To me the question here is what are the benefits of having a
>  > > coordinator in
>  > > >>
>  > > >> > >> > > comparison to a global committer/aggregator operator.
>  > > >>
>  > > >> > >> > >
>  > > >>
>  > > >> > >> > > > Personally, I am usually in favor of keeping streaming
>  > > ingestion (to data
>  > > >>
>  > > >> > >> > > > lake) relatively simple and stable. Also sometimes
>  > > compaction and sorting
>  > > >>
>  > > >> > >> > > > are performed together in data rewrite maintenance jobs to
>  > > improve read
>  > > >>
>  > > >> > >> > > > performance. In that case, the value of compacting (in Flink
>  > > streaming
>  > > >>
>  > > >> > >> > > > ingestion) diminishes.
>  > > >>
>  > > >> > >> > >
>  > > >>
>  > > >> > >> > >
>  > > >>
>  > > >> > >> > > I agree it is always possible to have scheduled maintenance
>  > > jobs keeping
>  > > >>
>  > > >> > >> > > care of
>  > > >>
>  > > >> > >> > > your data i.e., doing compaction. Unfortunately, the downside
>  > > is that you
>  > > >>
>  > > >> > >> > > have to your data after it is already available for other
>  > > downstream
>  > > >>
>  > > >> > >> > > consumers.
>  > > >>
>  > > >> > >> > > I guess this can lead to all kinds of visibility problems. I
>  > > am also
>  > > >>
>  > > >> > >> > > surprised that
>  > > >>
>  > > >> > >> > > you personally are a fan of this approach and, on the other
>  > > hand, are
>  > > >>
>  > > >> > >> > > developing
>  > > >>
>  > > >> > >> > > the Iceberg sink, which goes somewhat against your mentioned
>  > > principle of
>  > > >>
>  > > >> > >> > > keeping
>  > > >>
>  > > >> > >> > > the sink simple.
>  > > >>
>  > > >> > >> > >
>  > > >>
>  > > >> > >> > > > Currently, it is unclear from the doc and this thread where
>  > > the
>  > > >>
>  > > >> > >> > > compaction
>  > > >>
>  > > >> > >> > > > is actually happening. Jingsong's reply described one model
>  > > >>
>  > > >> > >> > > > writer (parallel) -> aggregator (single-parallelism
>  > > compaction planner)
>  > > >>
>  > > >> > >> > > ->
>  > > >>
>  > > >> > >> > > > compactor (parallel) -> global committer (single-parallelism)
>  > > >>
>  > > >> > >> > >
>  > > >>
>  > > >> > >> > >
>  > > >>
>  > > >> > >> > > My idea of the topology is very similar to the one outlined by
>  > > Jinsong. The
>  > > >>
>  > > >> > >> > > compaction will happen in the committer operator.
>  > > >>
>  > > >> > >> > >
>  > > >>
>  > > >> > >> > > >
>  > > >>
>  > > >> > >> > > > In the Iceberg community, the following model has been
>  > > discussed. It is
>  > > >>
>  > > >> > >> > > > better for Iceberg because it won't delay the data
>  > > availability.
>  > > >>
>  > > >> > >> > > > writer (parallel) -> global committer for append (single
>  > > parallelism) ->
>  > > >>
>  > > >> > >> > > > compactor (parallel) -> global committer for rewrite commit
>  > > (single
>  > > >>
>  > > >> > >> > > > parallelism)
>  > > >>
>  > > >> > >> > >
>  > > >>
>  > > >> > >> > >
>  > > >>
>  > > >> > >> > > From a quick glimpse, it seems that the exact same topology is
>  > > possible to
>  > > >>
>  > > >> > >> > > express with the committable aggregator, but this definitely
>  > > depends on
>  > > >>
>  > > >> > >> > > the exact
>  > > >>
>  > > >> > >> > > setup.
>  > > >>
>  > > >> > >> > >
>  > > >>
>  > > >> > >> > > Best,
>  > > >>
>  > > >> > >> > > Fabian
>  > > >>
>  > > >> > >>
>  > >
>
>

Re: Re: [DISCUSS] FLIP-191: Extend unified Sink interface to support small file compaction

Posted by Yun Gao <yu...@aliyun.com.INVALID>.
Hi, 

As Arvid appointed out, with some more checks, if a job vertex have multiple downstream vertices and 
they are all connected via blocking edges, they should not affect each other, as long as we could
ensure the intermediate shuffle data do not get lost, the finished precedent job vertex not restarted. 
Thus as a whole, for 2pc under batch mode, we should be able to ensure exactly-once with 
JM Failover + HA result partition + all-to-all blocking edge before the committer. For the first step, 
we could first have all-to-all blocking edge to reduce the probability of repeat data. 

And for the post-committer topology, sorry it reminds me another issue we ever discussed
before: for the streaming mode, we should not be able to support cascade commit since now
emit records in notifyCheckpointComplete is not supported after FLIP-147. Thus I think the
committables we emit from the committer to the post-committer topology should be before
they actually committed, right? If it is the case, perhaps we either document the situation clearly
and if one job requires taking actions after the committables get committed, he could further check
if the committables are committed manully(like scanning the filesystem to see if the files are renamed), 
or we might postpone supporting the post-committer topology until we have further determined the 
semantics and solutions ? 

Best,
Yun



------------------------------------------------------------------
From:Arvid Heise <ar...@apache.org>
Send Time:2021 Dec. 16 (Thu.) 21:48
To:Yun Gao <yu...@aliyun.com>
Cc:dev <de...@flink.apache.org>
Subject:Re: Re: [DISCUSS] FLIP-191: Extend unified Sink interface to support small file compaction

I just noticed offline with Yun that I have some misconception on how how blocking data exchange work in Flink. Apparently, a subtask of the stage 2 is already started when all connected subtasks of stage 1 are finished. I was assuming that it works similarly to Hadoop where all tasks of stage 1 have to be finished before any task of stage 2 is started.
So to amend the proposal, we would introduce a full shuffle before committer in batch. That would simulate the behavior of Hadoop.

We also need to verify what happens in case of multiple sinks. In that case, we may have started committer of sink 1 while sink 2 is still writing. That would be fine as long as sink 1 writers are not restarted. If not then it seems as if we may improve scheduler to actually not restart "finished" tasks. It seems like an orthogonal issue though.

Note that all discussed issues apply to Sink V1 as well, so the FLIP is not making anything worse.


On Thu, Dec 16, 2021 at 1:53 PM Arvid Heise <ar...@apache.org> wrote:
Hi Yun,

the basic idea is to use different regions for the different stages.

So you have your stages: 
Stage 1: <job> -> pre-writer topology -> writer -> committables (materialized)
Stage 2: committables (materialized) -> pre-commit topology -> committer -> succesful committables (materialized)
Stage 3: succesful committables (materialized) -> post-commit topology

If any instance of a given stages fails, the whole stage is restarted. 
So, in case of the main pipeline (stage 1) fails, no data will be committed at all. On a rerun, we start fresh (or from the previous stage).
Only, when all data has been written, we start with committing the data. An issue during committing, will retrigger the commit stage (stage 2) and only that stage. Thus, all committables are stable and remain stable.
When we are done committing all committables, we run the post-commit topology similarly "atomically".

So now the cavaets: 
- If committer is rerun, all committables are restarted. So the committer needs to be idempotent. That is the same with STREAMING mode now and afaik there is no way around it.
- If we lose a TM during commit phase, we will run into the original issues of inconstent data as we need to rerun the whole job. That would be solved with HA storage and we haven't found any solution that doesn't require some kind of external storage. However, the commit phase should be rather fast and errors are improbable (low volume).

I'd still like to have an HA storage but that issue is also in Sink V1 and kind of orthogonal. It's also nothing that we can solve without involving more folks (I'm trying to kick start that in the background).

On Thu, Dec 16, 2021 at 1:31 PM Yun Gao <yu...@aliyun.com> wrote:
Hi,

Very thanks Fabian for the explanation and it solves most of the issues. 
There is one left issue I want to have a double confirmation is that for 
the edges between writer and committer and in the post-committer topology,  
perhaps the result partition with HA storage is not enough solve all the issues 
directly ? It is due to after the committer and post-committer topology is finished
and the data is committed, it might still be restarted due to JM failover and the
deterministic problem (namely the example of  (A -> [B1, B2], A, B1 have finished and
B2 failed, if -> is rebalance / random / rescale,  all of A, B1, B2 would restarted). Then
the records would be produced and created for the second times. 

We might let the writers to skip producing the new records, but if we have multiple sinks like
OP1 -> (writer 1 -> committer 1) 
     |--> (writer 2 -> committer 2)

and the failover happens after writer 1 & committer 1 get finished but writer 2 is running,
if op1 produced different records across the two runs, then the two sinks would produces 
different data, which might be not suitable in some cases. Perhaps we need some support
from the scheduler side? 

But I also agree this could be a separate issue and we could solve it separately in some future
as long as we know how to solve it~

Best,
Yun


------------------------------------------------------------------
From:Arvid Heise <ar...@apache.org>
Send Time:2021 Dec. 16 (Thu.) 19:54
To:dev <de...@flink.apache.org>
Cc:Yun Gao <yu...@aliyun.com>
Subject:Re: Re: [DISCUSS] FLIP-191: Extend unified Sink interface to support small file compaction

Just a quick amend: There will be no blocking exchange in the pre-writer exchange for performance reasons. 
After the writer, we have tiny data volume and are free to add as many as we see necessary.

On Thu, Dec 16, 2021 at 11:18 AM Fabian Paul <fp...@apache.org> wrote:
Hi Till,

 Good point, in the scenario with the blocking keyed exchange between
 the writer and committer my idea is to make the committer effectively
 the global committer. With Sink V2 there is no real difference anymore
 between the committer and global committer.
 You are right that everything after the committer would be part of the
 same failover region but we plan to insert a blocking exchange by
 default before all of the custom topologies.

 Best,
 Fabian

 On Thu, Dec 16, 2021 at 11:08 AM Till Rohrmann <tr...@apache.org> wrote:
 >
 > Hi Fabian,
 >
 > quick question on your comment 3. If there is a pipelined data exchange
 > with a keyBy between the writers/committers and the component that does the
 > global commit, then there will only be a single failover region. So is it
 > correct that you assumed blocking data exchanges for the scenario you
 > described?
 >
 > Cheers,
 > Till
 >
 > On Thu, Dec 16, 2021 at 9:23 AM Fabian Paul <fp...@apache.org> wrote:
 >
 > > Hi Yun,
 > >
 > > Thanks for your fast feedback. Let me clarify your points.
 > >
 > > 1. We solve it by using StreamExchangeMode.BATCH before any exchange.
 > > That obviously doesn’t help with lost TM but we would need to employ
 > > HA storage for that. Same issue as now and orthogonal.
 > >
 > > 2. Extending V1 with V2 or vice versa would require renames of methods
 > > (since return types are non-optional) and is making API changes. Even
 > > though Experimental, we want to give connector developers the
 > > opportunity to provide 1 implementation for all of Flink 1.X. We will
 > > offer an internal adapter from V1 to V2, 2 sinkTo , and internally
 > > just have one code-path.
 > >
 > > 3. DataStreamSink would act as a unified view on all the operators and
 > > update them all at once when using setParallelism and so on (setName
 > > and setUid will receive suffixes per operator).
 > > Iceberg actually has a different requirement: They want to have a
 > > committer with parallelism 1 but as a coordinator such that
 > > embarrassingly parallel pipelines have different fail-over regions. I
 > > was thinking that in this case, they need to implement a no-op
 > > committer (that just forwards the committables) and use a post-commit
 > > topology that achieves that.
 > > Another option is that they use the preCommit topology and insert a
 > > constant key-by that forwards all committables to a single committer.
 > > We are planning to provide building blocks for such pipelines as we
 > > go.
 > >
 > > Best,
 > > Fabian
 > >
 > > On Thu, Dec 16, 2021 at 5:50 AM Yun Gao <yu...@aliyun.com> wrote:
 > > >
 > > > Hi Fabian,
 > > >
 > > > Very thanks for the update! I think the latest version in general looks
 > > good from my side
 > > > and I think using separate feature interface would be much more easy to
 > > understand
 > > > and extend in the future. I have some pending issues on the details
 > > though:
 > > >
 > > > 1. The first one is if we could support end-to-end exactly-once with
 > > post-committing
 > > > topology in the batch mode ? Since for the batch mode, currently we
 > > could only commit
 > > >  all the transactions after the whole job is finished, otherwise if
 > > there are JM failover or the
 > > > writer / committer get restarted due to indeterminstic (A -> [B1, B2],
 > > A, B1 have finished and
 > > >  B2 failed, if -> is rebalance / random / rescale,  all of A, B1, B2
 > > would restarted), there might
 > > > be repeat records. Previously one possible thought is to move committer
 > > and global committer
 > > >  to the operator coordinator, but if it is a topology, we might need
 > > some other kind of solutions?
 > > >
 > > > 2. I also want to have a dobule confirmation with the compatibility:
 > > since the old sink is also named
 > > > with Sink, do we want to put the Sink v2 in a new package ? Besides,
 > > since we might want to keep
 > > > only have one `sinkTo(Sink<?> sink)` , perhaps we also need to make the
 > > Sink v1 to be a subclass of
 > > > Sink v2 and extends the stateful and two-phase-commit sinks, right?
 > > >
 > > > 3. I'd like also have a confirmation on ours thoughts with the
 > > `DataStreamSink` returned by the sinkTo method:
 > > > The main issue is how do we implement the method like `setParallelism`
 > > or `setMaxParallelism` since now the sink
 > > > would be translated to multiple transformations? perhaps we could make
 > > it the default values for all the transformations
 > > > for the sink? A related issue would be for iceberg sink, I think it
 > > would need to have only one committer to avoid the
 > > > competition of the optimistic locks (which would cause performance
 > > degradation), then it might need to have N writers
 > > > with 1 committers, to build such topology, perhaps we might need to add
 > > new methods to specify the parallelism of
 > > > the writers and committers separately?
 > > >
 > > > Best,
 > > > Yun
 > > >
 > > >
 > > > ------------------Original Mail ------------------
 > > > Sender:Fabian Paul <fp...@apache.org>
 > > > Send Date:Mon Dec 13 23:59:43 2021
 > > > Recipients:dev <de...@flink.apache.org>
 > > > Subject:Re: [DISCUSS] FLIP-191: Extend unified Sink interface to support
 > > small file compaction
 > > >>
 > > >> Hi all,
 > > >>
 > > >>
 > > >>
 > > >> After a lot of discussions with different, we received very fruitful
 > > >>
 > > >> feedback and reworked the ideas behind this FLIP. Initially, we had
 > > >>
 > > >> the impression that the compaction problem is solvable by a single
 > > >>
 > > >> topology that we can reuse across different sinks. We now have a
 > > >>
 > > >> better understanding that different external systems require different
 > > >>
 > > >> compaction mechanism i.e. Hive requires compaction before finally
 > > >>
 > > >> registering the file in the metastore or Iceberg compacts the files
 > > >>
 > > >> after they have been registered and just lazily compacts them.
 > > >>
 > > >>
 > > >>
 > > >> Considering all these different views we came up with a design that
 > > >>
 > > >> builds upon what @guowei.mgw@gmail.com and @yungao.gy@aliyun.com have
 > > >>
 > > >> proposed at the beginning. We allow inserting custom topologies before
 > > >>
 > > >> and after the SinkWriters and Committers. Furthermore, we do not see
 > > >>
 > > >> it as a downside. The Sink interfaces that will expose the DataStream
 > > >>
 > > >> to the user reside in flink-streaming-java in contrast to the basic
 > > >>
 > > >> Sink interfaces that reside fin flink-core deem it to be only used by
 > > >>
 > > >> expert users.
 > > >>
 > > >>
 > > >>
 > > >> Moreover, we also wanted to remove the global committer from the
 > > >>
 > > >> unified Sink interfaces and replace it with a custom post-commit
 > > >>
 > > >> topology. Unfortunately, we cannot do it without breaking the Sink
 > > >>
 > > >> interface since the GlobalCommittables are part of the parameterized
 > > >>
 > > >> Sink interface. Thus, we propose building a new Sink V2 interface
 > > >>
 > > >> consisting of composable interfaces that do not offer the
 > > >>
 > > >> GlobalCommitter anymore. We will implement a utility to extend a Sink
 > > >>
 > > >> with post topology that mimics the behavior of the GlobalCommitter.
 > > >>
 > > >> The new Sink V2 provides the same sort of methods as the Sink V1
 > > >>
 > > >> interface, so a migration of sinks that do not use the GlobalCommitter
 > > >>
 > > >> should be very easy.
 > > >>
 > > >> We plan to keep the existing Sink V1 interfaces to not break
 > > >>
 > > >> externally built sinks. As part of this FLIP, we migrate all the
 > > >>
 > > >> connectors inside of the main repository to the new Sink V2 API.
 > > >>
 > > >>
 > > >>
 > > >> The FLIP document is also updated and includes the proposed changes.
 > > >>
 > > >>
 > > >>
 > > >> Looking forward to your feedback,
 > > >>
 > > >> Fabian
 > > >>
 > > >>
 > > >>
 > > >>
 > > https://cwiki.apache.org/confluence/display/FLINK/FLIP-191%3A+Extend+unified+Sink+interface+to+support+small+file+compaction
 > > >>
 > > >>
 > > >>
 > > >>
 > > >>
 > > >> On Thu, Dec 2, 2021 at 10:15 AM Roman Khachatryan wrote:
 > > >>
 > > >> >
 > > >>
 > > >> > Thanks for clarifying (I was initially confused by merging state files
 > > >>
 > > >> > rather than output files).
 > > >>
 > > >> >
 > > >>
 > > >> > > At some point, Flink will definitely have some WAL adapter that can
 > > turn any sink into an exactly-once sink (with some caveats). For now, we
 > > keep that as an orthogonal solution as it has a rather high price (bursty
 > > workload with high latency). Ideally, we can keep the compaction
 > > asynchronously...
 > > >>
 > > >> >
 > > >>
 > > >> > Yes, that would be something like a WAL. I agree that it would have a
 > > >>
 > > >> > different set of trade-offs.
 > > >>
 > > >> >
 > > >>
 > > >> >
 > > >>
 > > >> > Regards,
 > > >>
 > > >> > Roman
 > > >>
 > > >> >
 > > >>
 > > >> > On Mon, Nov 29, 2021 at 3:33 PM Arvid Heise wrote:
 > > >>
 > > >> > >>
 > > >>
 > > >> > >> > One way to avoid write-read-merge is by wrapping SinkWriter with
 > > >>
 > > >> > >> > another one, which would buffer input elements in a temporary
 > > storage
 > > >>
 > > >> > >> > (e.g. local file) until a threshold is reached; after that, it
 > > would
 > > >>
 > > >> > >> > invoke the original SinkWriter. And if a checkpoint barrier
 > > comes in
 > > >>
 > > >> > >> > earlier, it would send written data to some aggregator.
 > > >>
 > > >> > >>
 > > >>
 > > >> > >> I think perhaps this seems to be a kind of WAL method? Namely we
 > > first
 > > >>
 > > >> > >> write the elements to some WAL logs and persist them on checkpoint
 > > >>
 > > >> > >> (in snapshot or remote FS), or we directly write WAL logs to the
 > > remote
 > > >>
 > > >> > >> FS eagerly.
 > > >>
 > > >> > >>
 > > >>
 > > >> > > At some point, Flink will definitely have some WAL adapter that can
 > > turn any sink into an exactly-once sink (with some caveats). For now, we
 > > keep that as an orthogonal solution as it has a rather high price (bursty
 > > workload with high latency). Ideally, we can keep the compaction
 > > asynchronously...
 > > >>
 > > >> > >
 > > >>
 > > >> > > On Mon, Nov 29, 2021 at 8:52 AM Yun Gao wrote:
 > > >>
 > > >> > >>
 > > >>
 > > >> > >> Hi,
 > > >>
 > > >> > >>
 > > >>
 > > >> > >> @Roman very sorry for the late response for a long time,
 > > >>
 > > >> > >>
 > > >>
 > > >> > >> > Merging artifacts from multiple checkpoints would apparently
 > > >>
 > > >> > >> require multiple concurrent checkpoints
 > > >>
 > > >> > >>
 > > >>
 > > >> > >> I think it might not need concurrent checkpoints: suppose some
 > > >>
 > > >> > >> operators (like the committer aggregator in the option 2) maintains
 > > >>
 > > >> > >> the list of files to merge, it could stores the lists of files to
 > > merge
 > > >>
 > > >> > >> in the states, then after several checkpoints are done and we have
 > > >>
 > > >> > >> enough files, we could merge all the files in the list.
 > > >>
 > > >> > >>
 > > >>
 > > >> > >> > Asynchronous merging in an aggregator would require some
 > > resolution
 > > >>
 > > >> > >> > logic on recovery, so that a merged artifact can be used if the
 > > >>
 > > >> > >> > original one was deleted. Otherwise, wouldn't recovery fail
 > > because
 > > >>
 > > >> > >> > some artifacts are missing?
 > > >>
 > > >> > >> > We could also defer deletion until the "compacted" checkpoint is
 > > >>
 > > >> > >> > subsumed - but isn't it too late, as it will be deleted anyways
 > > once
 > > >>
 > > >> > >> > subsumed?
 > > >>
 > > >> > >>
 > > >>
 > > >> > >> I think logically we could delete the original files once the
 > > "compacted" checkpoint
 > > >>
 > > >> > >> (which finish merging the compacted files and record it in the
 > > checkpoint) is completed
 > > >>
 > > >> > >> in all the options. If there are failover before we it, we could
 > > restart the merging and if
 > > >>
 > > >> > >> there are failover after it, we could have already recorded the
 > > files in the checkpoint.
 > > >>
 > > >> > >>
 > > >>
 > > >> > >> > One way to avoid write-read-merge is by wrapping SinkWriter with
 > > >>
 > > >> > >> > another one, which would buffer input elements in a temporary
 > > storage
 > > >>
 > > >> > >> > (e.g. local file) until a threshold is reached; after that, it
 > > would
 > > >>
 > > >> > >> > invoke the original SinkWriter. And if a checkpoint barrier
 > > comes in
 > > >>
 > > >> > >> > earlier, it would send written data to some aggregator.
 > > >>
 > > >> > >>
 > > >>
 > > >> > >> I think perhaps this seems to be a kind of WAL method? Namely we
 > > first
 > > >>
 > > >> > >> write the elements to some WAL logs and persist them on checkpoint
 > > >>
 > > >> > >> (in snapshot or remote FS), or we directly write WAL logs to the
 > > remote
 > > >>
 > > >> > >> FS eagerly.
 > > >>
 > > >> > >>
 > > >>
 > > >> > >> Sorry if I do not understand correctly somewhere.
 > > >>
 > > >> > >>
 > > >>
 > > >> > >> Best,
 > > >>
 > > >> > >> Yun
 > > >>
 > > >> > >>
 > > >>
 > > >> > >>
 > > >>
 > > >> > >> ------------------------------------------------------------------
 > > >>
 > > >> > >> From:Roman Khachatryan
 > > >>
 > > >> > >> Send Time:2021 Nov. 9 (Tue.) 22:03
 > > >>
 > > >> > >> To:dev
 > > >>
 > > >> > >> Subject:Re: [DISCUSS] FLIP-191: Extend unified Sink interface to
 > > support small file compaction
 > > >>
 > > >> > >>
 > > >>
 > > >> > >> Hi everyone,
 > > >>
 > > >> > >>
 > > >>
 > > >> > >> Thanks for the proposal and the discussion, I have some remarks:
 > > >>
 > > >> > >> (I'm not very familiar with the new Sink API but I thought about
 > > the
 > > >>
 > > >> > >> same problem in context of the changelog state backend)
 > > >>
 > > >> > >>
 > > >>
 > > >> > >> 1. Merging artifacts from multiple checkpoints would apparently
 > > >>
 > > >> > >> require multiple concurrent checkpoints (otherwise, a new
 > > checkpoint
 > > >>
 > > >> > >> won't be started before completing the previous one; and the
 > > previous
 > > >>
 > > >> > >> one can't be completed before durably storing the artifacts).
 > > However,
 > > >>
 > > >> > >> concurrent checkpoints are currently not supported with Unaligned
 > > >>
 > > >> > >> checkpoints (this is besides increasing e2e-latency).
 > > >>
 > > >> > >>
 > > >>
 > > >> > >> 2. Asynchronous merging in an aggregator would require some
 > > resolution
 > > >>
 > > >> > >> logic on recovery, so that a merged artifact can be used if the
 > > >>
 > > >> > >> original one was deleted. Otherwise, wouldn't recovery fail because
 > > >>
 > > >> > >> some artifacts are missing?
 > > >>
 > > >> > >> We could also defer deletion until the "compacted" checkpoint is
 > > >>
 > > >> > >> subsumed - but isn't it too late, as it will be deleted anyways
 > > once
 > > >>
 > > >> > >> subsumed?
 > > >>
 > > >> > >>
 > > >>
 > > >> > >> 3. Writing small files, then reading and merging them for *every*
 > > >>
 > > >> > >> checkpoint seems worse than only reading them on recovery. I guess
 > > I'm
 > > >>
 > > >> > >> missing some cases of reading, so to me it would make sense to
 > > mention
 > > >>
 > > >> > >> these cases explicitly in the FLIP motivation section.
 > > >>
 > > >> > >>
 > > >>
 > > >> > >> 4. One way to avoid write-read-merge is by wrapping SinkWriter with
 > > >>
 > > >> > >> another one, which would buffer input elements in a temporary
 > > storage
 > > >>
 > > >> > >> (e.g. local file) until a threshold is reached; after that, it
 > > would
 > > >>
 > > >> > >> invoke the original SinkWriter. And if a checkpoint barrier comes
 > > in
 > > >>
 > > >> > >> earlier, it would send written data to some aggregator. It will
 > > >>
 > > >> > >> increase checkpoint delay (async phase) compared to the current
 > > Flink;
 > > >>
 > > >> > >> but not compared to the write-read-merge solution, IIUC.
 > > >>
 > > >> > >> Then such "BufferingSinkWriters" could aggregate input elements
 > > from
 > > >>
 > > >> > >> each other, potentially recursively (I mean something like
 > > >>
 > > >> > >>
 > > https://cwiki.apache.org/confluence/download/attachments/173082889/DSTL-DFS-DAG.png
 > > >>
 > > >> > >> )
 > > >>
 > > >> > >>
 > > >>
 > > >> > >> 5. Reducing the number of files by reducing aggregator parallelism
 > > as
 > > >>
 > > >> > >> opposed to merging on reaching size threshold will likely be less
 > > >>
 > > >> > >> optimal and more difficult to configure. OTH, thresholds might be
 > > more
 > > >>
 > > >> > >> difficult to implement and (with recursive merging) would incur
 > > higher
 > > >>
 > > >> > >> latency. Maybe that's also something to decide explicitly or at
 > > least
 > > >>
 > > >> > >> mention in the FLIP.
 > > >>
 > > >> > >>
 > > >>
 > > >> > >>
 > > >>
 > > >> > >>
 > > >>
 > > >> > >> Regards,
 > > >>
 > > >> > >> Roman
 > > >>
 > > >> > >>
 > > >>
 > > >> > >>
 > > >>
 > > >> > >> On Tue, Nov 9, 2021 at 5:23 AM Reo Lei wrote:
 > > >>
 > > >> > >> >
 > > >>
 > > >> > >> > Hi Fabian,
 > > >>
 > > >> > >> >
 > > >>
 > > >> > >> > Thanks for drafting the FLIP and trying to support small file
 > > compaction. I
 > > >>
 > > >> > >> > think this feature is very urgent and valuable for users(at
 > > least for me).
 > > >>
 > > >> > >> >
 > > >>
 > > >> > >> > Currently I am trying to support streaming rewrite(compact) for
 > > Iceberg on
 > > >>
 > > >> > >> > PR#3323 . As Steven mentioned,
 > > >>
 > > >> > >> > Iceberg sink and compact data through the following steps:
 > > >>
 > > >> > >> > Step-1: Some parallel data writer(sinker) to write streaming
 > > data as files.
 > > >>
 > > >> > >> > Step-2: A single parallelism data files committer to commit the
 > > completed
 > > >>
 > > >> > >> > files as soon as possible to make them available.
 > > >>
 > > >> > >> > Step-3: Some parallel file rewriter(compactor) to collect
 > > committed files
 > > >>
 > > >> > >> > from multiple checkpoints, and rewriter(compact) them together
 > > once the
 > > >>
 > > >> > >> > total file size or number of files reach the threshold.
 > > >>
 > > >> > >> > Step-4: A single parallelism rewrite(compact) result committer
 > > to commit
 > > >>
 > > >> > >> > the rewritten(compacted) files to replace the old files and make
 > > them
 > > >>
 > > >> > >> > available.
 > > >>
 > > >> > >> >
 > > >>
 > > >> > >> >
 > > >>
 > > >> > >> > If Flink want to support small file compaction, some key point I
 > > think is
 > > >>
 > > >> > >> > necessary:
 > > >>
 > > >> > >> >
 > > >>
 > > >> > >> > 1, Compact files from multiple checkpoints.
 > > >>
 > > >> > >> > I totally agree with Jingsong, because completed file size
 > > usually could
 > > >>
 > > >> > >> > not reach the threshold in a single checkpoint. Especially for
 > > partitioned
 > > >>
 > > >> > >> > table, we need to compact the files of each partition, but
 > > usually the file
 > > >>
 > > >> > >> > size of each partition will be different and may not reach the
 > > merge
 > > >>
 > > >> > >> > threshold. If we compact these files, in a single checkpoint,
 > > regardless of
 > > >>
 > > >> > >> > whether the total file size reaches the threshold, then the
 > > value of
 > > >>
 > > >> > >> > compacting will be diminished and we will still get small files
 > > because
 > > >>
 > > >> > >> > these compacted files are not reach to target size. So we need
 > > the
 > > >>
 > > >> > >> > compactor to collect committed files from multiple checkpoints
 > > and compact
 > > >>
 > > >> > >> > them until they reach the threshold.
 > > >>
 > > >> > >> >
 > > >>
 > > >> > >> > 2, Separate write phase and compact phase.
 > > >>
 > > >> > >> > Users usually hope the data becomes available as soon as
 > > possible, and the
 > > >>
 > > >> > >> > end-to-end latency is very important. I think we need to
 > > separate the
 > > >>
 > > >> > >> > write and compact phase. For the write phase, there include the
 > > Step-1
 > > >>
 > > >> > >> > and Step-2, we sink data as file and commit it pre checkpoint
 > > and regardless
 > > >>
 > > >> > >> > of whether the file size it is. That could ensure the data will
 > > be
 > > >>
 > > >> > >> > available ASAP. For the compact phase, there include the Step-3
 > > >>
 > > >> > >> > and Step-4, the compactor should collect committed files from
 > > multiple
 > > >>
 > > >> > >> > checkpoints and compact them asynchronously once they reach the
 > > threshold,
 > > >>
 > > >> > >> > and the compact committer will commit the compaction result in
 > > the next
 > > >>
 > > >> > >> > checkpoint. We compact the committed files asynchronously
 > > because we don't
 > > >>
 > > >> > >> > want the compaction to affect the data sink or the whole
 > > pipeline.
 > > >>
 > > >> > >> >
 > > >>
 > > >> > >> > 3, Exactly once guarantee between write and compact phase.
 > > >>
 > > >> > >> > Once we separate write phase and compact phase, we need to
 > > consider
 > > >>
 > > >> > >> > how to guarantee
 > > >>
 > > >> > >> > the exact once semantic between two phases. We should not lose
 > > any data or
 > > >>
 > > >> > >> > files on the compactor(Step-3) in any case and cause the
 > > compaction result
 > > >>
 > > >> > >> > to be inconsistent with before. I think flink should provide an
 > > easy-to-use
 > > >>
 > > >> > >> > interface to make that easier.
 > > >>
 > > >> > >> >
 > > >>
 > > >> > >> > 4, Metadata operation and compaction result validation.
 > > >>
 > > >> > >> > In the compact phase, there may be not only compact files, but
 > > also a lot
 > > >>
 > > >> > >> > of metadata operations, such as the iceberg needing to
 > > read/write manifest
 > > >>
 > > >> > >> > and do MOR. And we need some interface to support users to do
 > > some
 > > >>
 > > >> > >> > validation of the compaction result. I think these points should
 > > be
 > > >>
 > > >> > >> > considered when we design the compaction API.
 > > >>
 > > >> > >> >
 > > >>
 > > >> > >> >
 > > >>
 > > >> > >> > Back to FLIP-191, option 1 looks very complicated while option 2
 > > is
 > > >>
 > > >> > >> > relatively simple, but neither of these two solutions separates
 > > the write
 > > >>
 > > >> > >> > phase from the compact phase. So I think we should consider the
 > > points I
 > > >>
 > > >> > >> > mentioned above. And if you have any other questions you can
 > > always feel
 > > >>
 > > >> > >> > free to reach out to me!
 > > >>
 > > >> > >> >
 > > >>
 > > >> > >> > BR,
 > > >>
 > > >> > >> > Reo
 > > >>
 > > >> > >> >
 > > >>
 > > >> > >> > Fabian Paul 于2021年11月8日周一 下午7:59写道:
 > > >>
 > > >> > >> >
 > > >>
 > > >> > >> > > Hi all,
 > > >>
 > > >> > >> > >
 > > >>
 > > >> > >> > > Thanks for the lively discussions. I am really excited to see
 > > so many
 > > >>
 > > >> > >> > > people
 > > >>
 > > >> > >> > > participating in this thread. It also underlines the need that
 > > many people
 > > >>
 > > >> > >> > > would
 > > >>
 > > >> > >> > > like to see a solution soon.
 > > >>
 > > >> > >> > >
 > > >>
 > > >> > >> > > I have updated the FLIP and removed the parallelism
 > > configuration because
 > > >>
 > > >> > >> > > it is
 > > >>
 > > >> > >> > > unnecessary since users can configure a constant exchange key
 > > to send all
 > > >>
 > > >> > >> > > committables to only one committable aggregator.
 > > >>
 > > >> > >> > >
 > > >>
 > > >> > >> > >
 > > >>
 > > >> > >> > > 1. Burden for developers w.r.t batch stream unification.
 > > >>
 > > >> > >> > >
 > > >>
 > > >> > >> > > @yun @guowei, from a theoretical point you are right about
 > > exposing the
 > > >>
 > > >> > >> > > DataStream
 > > >>
 > > >> > >> > > API in the sink users have the full power to write correct
 > > batch and
 > > >>
 > > >> > >> > > streaming
 > > >>
 > > >> > >> > > sinks. I think in reality a lot of users still struggle to
 > > build pipelines
 > > >>
 > > >> > >> > > with
 > > >>
 > > >> > >> > > i.e. the operator pipeline which works correct in streaming
 > > and batch mode.
 > > >>
 > > >> > >> > > Another problem I see is by exposing more deeper concepts is
 > > that we
 > > >>
 > > >> > >> > > cannot do
 > > >>
 > > >> > >> > > any optimization because we cannot reason about how sinks are
 > > built in the
 > > >>
 > > >> > >> > > future.
 > > >>
 > > >> > >> > >
 > > >>
 > > >> > >> > > We should also try to steer users towards using only
 > > `Functions` to give
 > > >>
 > > >> > >> > > us more
 > > >>
 > > >> > >> > > flexibility to swap the internal operator representation. I
 > > agree with
 > > >>
 > > >> > >> > > @yun we
 > > >>
 > > >> > >> > > should try to make the `ProcessFunction` more versatile to
 > > work on that
 > > >>
 > > >> > >> > > goal but
 > > >>
 > > >> > >> > > I see this as unrelated to the FLIP.
 > > >>
 > > >> > >> > >
 > > >>
 > > >> > >> > >
 > > >>
 > > >> > >> > > 2. Regarding Commit / Global commit
 > > >>
 > > >> > >> > >
 > > >>
 > > >> > >> > > I envision the global committer to be specific depending on
 > > the data lake
 > > >>
 > > >> > >> > > solution you want to write to. However, it is entirely
 > > orthogonal to the
 > > >>
 > > >> > >> > > compaction.
 > > >>
 > > >> > >> > > Currently, I do not expect any changes w.r.t the Global commit
 > > introduces
 > > >>
 > > >> > >> > > by
 > > >>
 > > >> > >> > > this FLIP.
 > > >>
 > > >> > >> > >
 > > >>
 > > >> > >> > >
 > > >>
 > > >> > >> > > 3. Regarding the case of trans-checkpoints merging
 > > >>
 > > >> > >> > >
 > > >>
 > > >> > >> > > @yun, as user, I would expect that if the committer receives
 > > in a
 > > >>
 > > >> > >> > > checkpoint files
 > > >>
 > > >> > >> > > to merge/commit that these are also finished when the
 > > checkpoint finishes.
 > > >>
 > > >> > >> > > I think all sinks rely on this principle currently i.e.,
 > > KafkaSink needs to
 > > >>
 > > >> > >> > > commit all open transactions until the next checkpoint can
 > > happen.
 > > >>
 > > >> > >> > >
 > > >>
 > > >> > >> > > Maybe in the future, we can somehow move the Committer#commit
 > > call to an
 > > >>
 > > >> > >> > > asynchronous execution, but we should discuss it as a separate
 > > thread.
 > > >>
 > > >> > >> > >
 > > >>
 > > >> > >> > > > We probably should first describe the different causes of
 > > small files and
 > > >>
 > > >> > >> > > > what problems was this proposal trying to solve. I wrote a
 > > data shuffling
 > > >>
 > > >> > >> > > > proposal [1] for Flink Iceberg sink (shared with Iceberg
 > > community [2]).
 > > >>
 > > >> > >> > > It
 > > >>
 > > >> > >> > > > can address small files problems due to skewed data
 > > distribution across
 > > >>
 > > >> > >> > > > Iceberg table partitions. Streaming shuffling before writers
 > > (to files)
 > > >>
 > > >> > >> > > is
 > > >>
 > > >> > >> > > > typically more efficient than post-write file compaction
 > > (which involves
 > > >>
 > > >> > >> > > > read-merge-write). It is usually cheaper to prevent a
 > > problem (small
 > > >>
 > > >> > >> > > files)
 > > >>
 > > >> > >> > > > than fixing it.
 > > >>
 > > >> > >> > >
 > > >>
 > > >> > >> > >
 > > >>
 > > >> > >> > > @steven you are raising a good point, although I think only
 > > using a
 > > >>
 > > >> > >> > > customizable
 > > >>
 > > >> > >> > > shuffle won't address the generation of small files. One
 > > assumption is that
 > > >>
 > > >> > >> > > at least the sink generates one file per subtask, which can
 > > already be too
 > > >>
 > > >> > >> > > many.
 > > >>
 > > >> > >> > > Another problem is that with low checkpointing intervals, the
 > > files do not
 > > >>
 > > >> > >> > > meet
 > > >>
 > > >> > >> > > the required size. The latter point is probably addressable by
 > > changing the
 > > >>
 > > >> > >> > > checkpoint interval, which might be inconvenient for some
 > > users.
 > > >>
 > > >> > >> > >
 > > >>
 > > >> > >> > > > The sink coordinator checkpoint problem (mentioned in option
 > > 1) would be
 > > >>
 > > >> > >> > > > great if Flink can address it. In the spirit of source
 > > >>
 > > >> > >> > > (enumerator-reader)
 > > >>
 > > >> > >> > > > and sink (writer-coordinator) duality, sink coordinator
 > > checkpoint should
 > > >>
 > > >> > >> > > > happen after the writer operator. This would be a natural
 > > fit to support
 > > >>
 > > >> > >> > > > global committer in FLIP-143. It is probably an orthogonal
 > > matter to this
 > > >>
 > > >> > >> > > > proposal.
 > > >>
 > > >> > >> > >
 > > >>
 > > >> > >> > >
 > > >>
 > > >> > >> > > To me the question here is what are the benefits of having a
 > > coordinator in
 > > >>
 > > >> > >> > > comparison to a global committer/aggregator operator.
 > > >>
 > > >> > >> > >
 > > >>
 > > >> > >> > > > Personally, I am usually in favor of keeping streaming
 > > ingestion (to data
 > > >>
 > > >> > >> > > > lake) relatively simple and stable. Also sometimes
 > > compaction and sorting
 > > >>
 > > >> > >> > > > are performed together in data rewrite maintenance jobs to
 > > improve read
 > > >>
 > > >> > >> > > > performance. In that case, the value of compacting (in Flink
 > > streaming
 > > >>
 > > >> > >> > > > ingestion) diminishes.
 > > >>
 > > >> > >> > >
 > > >>
 > > >> > >> > >
 > > >>
 > > >> > >> > > I agree it is always possible to have scheduled maintenance
 > > jobs keeping
 > > >>
 > > >> > >> > > care of
 > > >>
 > > >> > >> > > your data i.e., doing compaction. Unfortunately, the downside
 > > is that you
 > > >>
 > > >> > >> > > have to your data after it is already available for other
 > > downstream
 > > >>
 > > >> > >> > > consumers.
 > > >>
 > > >> > >> > > I guess this can lead to all kinds of visibility problems. I
 > > am also
 > > >>
 > > >> > >> > > surprised that
 > > >>
 > > >> > >> > > you personally are a fan of this approach and, on the other
 > > hand, are
 > > >>
 > > >> > >> > > developing
 > > >>
 > > >> > >> > > the Iceberg sink, which goes somewhat against your mentioned
 > > principle of
 > > >>
 > > >> > >> > > keeping
 > > >>
 > > >> > >> > > the sink simple.
 > > >>
 > > >> > >> > >
 > > >>
 > > >> > >> > > > Currently, it is unclear from the doc and this thread where
 > > the
 > > >>
 > > >> > >> > > compaction
 > > >>
 > > >> > >> > > > is actually happening. Jingsong's reply described one model
 > > >>
 > > >> > >> > > > writer (parallel) -> aggregator (single-parallelism
 > > compaction planner)
 > > >>
 > > >> > >> > > ->
 > > >>
 > > >> > >> > > > compactor (parallel) -> global committer (single-parallelism)
 > > >>
 > > >> > >> > >
 > > >>
 > > >> > >> > >
 > > >>
 > > >> > >> > > My idea of the topology is very similar to the one outlined by
 > > Jinsong. The
 > > >>
 > > >> > >> > > compaction will happen in the committer operator.
 > > >>
 > > >> > >> > >
 > > >>
 > > >> > >> > > >
 > > >>
 > > >> > >> > > > In the Iceberg community, the following model has been
 > > discussed. It is
 > > >>
 > > >> > >> > > > better for Iceberg because it won't delay the data
 > > availability.
 > > >>
 > > >> > >> > > > writer (parallel) -> global committer for append (single
 > > parallelism) ->
 > > >>
 > > >> > >> > > > compactor (parallel) -> global committer for rewrite commit
 > > (single
 > > >>
 > > >> > >> > > > parallelism)
 > > >>
 > > >> > >> > >
 > > >>
 > > >> > >> > >
 > > >>
 > > >> > >> > > From a quick glimpse, it seems that the exact same topology is
 > > possible to
 > > >>
 > > >> > >> > > express with the committable aggregator, but this definitely
 > > depends on
 > > >>
 > > >> > >> > > the exact
 > > >>
 > > >> > >> > > setup.
 > > >>
 > > >> > >> > >
 > > >>
 > > >> > >> > > Best,
 > > >>
 > > >> > >> > > Fabian
 > > >>
 > > >> > >>
 > >



Re: Re: [DISCUSS] FLIP-191: Extend unified Sink interface to support small file compaction

Posted by Arvid Heise <ar...@apache.org>.
Hi Yun,

the basic idea is to use different regions for the different stages.

So you have your stages:
Stage 1: <job> -> pre-writer topology -> writer -> committables
(materialized)
Stage 2: committables (materialized) -> pre-commit topology -> committer ->
succesful committables (materialized)
Stage 3: succesful committables (materialized) -> post-commit topology

If any instance of a given stages fails, the whole stage is restarted.
So, in case of the main pipeline (stage 1) fails, no data will be committed
at all. On a rerun, we start fresh (or from the previous stage).
Only, when all data has been written, we start with committing the data. An
issue during committing, will retrigger the commit stage (stage 2) and only
that stage. Thus, all committables are stable and remain stable.
When we are done committing all committables, we run the post-commit
topology similarly "atomically".

So now the cavaets:
- If committer is rerun, all committables are restarted. So the committer
needs to be idempotent. That is the same with STREAMING mode now and afaik
there is no way around it.
- If we lose a TM during commit phase, we will run into the original issues
of inconstent data as we need to rerun the whole job. That would be solved
with HA storage and we haven't found any solution that doesn't require some
kind of external storage. However, the commit phase should be rather fast
and errors are improbable (low volume).

I'd still like to have an HA storage but that issue is also in Sink V1 and
kind of orthogonal. It's also nothing that we can solve without involving
more folks (I'm trying to kick start that in the background).

On Thu, Dec 16, 2021 at 1:31 PM Yun Gao <yu...@aliyun.com> wrote:

> Hi,
>
> Very thanks Fabian for the explanation and it solves most of the issues.
> There is one left issue I want to have a double confirmation is that for
> the edges between writer and committer and in the post-committer
> topology,
> perhaps the result partition with HA storage is not enough solve all the
> issues
> directly ? It is due to after the committer and post-committer topology is
> finished
> and the data is committed, it might still be restarted due to JM failover
> and the
> deterministic problem (namely the example of  (A -> [B1, B2], A, B1 have
> finished and
> B2 failed, if -> is rebalance / random / rescale,  all of A, B1, B2 would
> restarted). Then
> the records would be produced and created for the second times.
>
> We might let the writers to skip producing the new records, but if we have
> multiple sinks like
> OP1 -> (writer 1 -> committer 1)
>      |--> (writer 2 -> committer 2)
>
> and the failover happens after writer 1 & committer 1 get finished but
> writer 2 is running,
> if op1 produced different records across the two runs, then the two sinks
> would produces
> different data, which might be not suitable in some cases. Perhaps we need
> some support
> from the scheduler side?
>
> But I also agree this could be a separate issue and we could solve it separately
> in some future
> as long as we know how to solve it~
>
> Best,
> Yun
>
>
> ------------------------------------------------------------------
> From:Arvid Heise <ar...@apache.org>
> Send Time:2021 Dec. 16 (Thu.) 19:54
> To:dev <de...@flink.apache.org>
> Cc:Yun Gao <yu...@aliyun.com>
> Subject:Re: Re: [DISCUSS] FLIP-191: Extend unified Sink interface to
> support small file compaction
>
> Just a quick amend: There will be no blocking exchange in the pre-writer
> exchange for performance reasons.
> After the writer, we have tiny data volume and are free to add as many as
> we see necessary.
>
> On Thu, Dec 16, 2021 at 11:18 AM Fabian Paul <fp...@apache.org> wrote:
> Hi Till,
>
> Good point, in the scenario with the blocking keyed exchange between
> the writer and committer my idea is to make the committer effectively
> the global committer. With Sink V2 there is no real difference anymore
> between the committer and global committer.
> You are right that everything after the committer would be part of the
> same failover region but we plan to insert a blocking exchange by
> default before all of the custom topologies.
>
> Best,
> Fabian
>
> On Thu, Dec 16, 2021 at 11:08 AM Till Rohrmann <tr...@apache.org>
> wrote:
> >
> > Hi Fabian,
> >
> > quick question on your comment 3. If there is a pipelined data exchange
> > with a keyBy between the writers/committers and the component that does
> the
> > global commit, then there will only be a single failover region. So is it
> > correct that you assumed blocking data exchanges for the scenario you
> > described?
> >
> > Cheers,
> > Till
> >
> > On Thu, Dec 16, 2021 at 9:23 AM Fabian Paul <fp...@apache.org> wrote:
> >
> > > Hi Yun,
> > >
> > > Thanks for your fast feedback. Let me clarify your points.
> > >
> > > 1. We solve it by using StreamExchangeMode.BATCH before any exchange.
> > > That obviously doesn’t help with lost TM but we would need to employ
> > > HA storage for that. Same issue as now and orthogonal.
> > >
> > > 2. Extending V1 with V2 or vice versa would require renames of methods
> > > (since return types are non-optional) and is making API changes. Even
> > > though Experimental, we want to give connector developers the
> > > opportunity to provide 1 implementation for all of Flink 1.X. We will
> > > offer an internal adapter from V1 to V2, 2 sinkTo , and internally
> > > just have one code-path.
> > >
> > > 3. DataStreamSink would act as a unified view on all the operators and
> > > update them all at once when using setParallelism and so on (setName
> > > and setUid will receive suffixes per operator).
> > > Iceberg actually has a different requirement: They want to have a
> > > committer with parallelism 1 but as a coordinator such that
> > > embarrassingly parallel pipelines have different fail-over regions. I
> > > was thinking that in this case, they need to implement a no-op
> > > committer (that just forwards the committables) and use a post-commit
> > > topology that achieves that.
> > > Another option is that they use the preCommit topology and insert a
> > > constant key-by that forwards all committables to a single committer.
> > > We are planning to provide building blocks for such pipelines as we
> > > go.
> > >
> > > Best,
> > > Fabian
> > >
> > > On Thu, Dec 16, 2021 at 5:50 AM Yun Gao <yu...@aliyun.com> wrote:
> > > >
> > > > Hi Fabian,
> > > >
> > > > Very thanks for the update! I think the latest version in general
> looks
> > > good from my side
> > > > and I think using separate feature interface would be much more easy
> to
> > > understand
> > > > and extend in the future. I have some pending issues on the details
> > > though:
> > > >
> > > > 1. The first one is if we could support end-to-end exactly-once with
> > > post-committing
> > > > topology in the batch mode ? Since for the batch mode, currently we
> > > could only commit
> > > >  all the transactions after the whole job is finished, otherwise if
> > > there are JM failover or the
> > > > writer / committer get restarted due to indeterminstic (A -> [B1,
> B2],
> > > A, B1 have finished and
> > > >  B2 failed, if -> is rebalance / random / rescale,  all of A, B1, B2
> > > would restarted), there might
> > > > be repeat records. Previously one possible thought is to move
> committer
> > > and global committer
> > > >  to the operator coordinator, but if it is a topology, we might need
> > > some other kind of solutions?
> > > >
> > > > 2. I also want to have a dobule confirmation with the compatibility:
> > > since the old sink is also named
> > > > with Sink, do we want to put the Sink v2 in a new package ? Besides,
> > > since we might want to keep
> > > > only have one `sinkTo(Sink<?> sink)` , perhaps we also need to make
> the
> > > Sink v1 to be a subclass of
> > > > Sink v2 and extends the stateful and two-phase-commit sinks, right?
> > > >
> > > > 3. I'd like also have a confirmation on ours thoughts with the
> > > `DataStreamSink` returned by the sinkTo method:
> > > > The main issue is how do we implement the method like
> `setParallelism`
> > > or `setMaxParallelism` since now the sink
> > > > would be translated to multiple transformations? perhaps we could
> make
> > > it the default values for all the transformations
> > > > for the sink? A related issue would be for iceberg sink, I think it
> > > would need to have only one committer to avoid the
> > > > competition of the optimistic locks (which would cause performance
> > > degradation), then it might need to have N writers
> > > > with 1 committers, to build such topology, perhaps we might need to
> add
> > > new methods to specify the parallelism of
> > > > the writers and committers separately?
> > > >
> > > > Best,
> > > > Yun
> > > >
> > > >
> > > > ------------------Original Mail ------------------
> > > > Sender:Fabian Paul <fp...@apache.org>
> > > > Send Date:Mon Dec 13 23:59:43 2021
> > > > Recipients:dev <de...@flink.apache.org>
> > > > Subject:Re: [DISCUSS] FLIP-191: Extend unified Sink interface to
> support
> > > small file compaction
> > > >>
> > > >> Hi all,
> > > >>
> > > >>
> > > >>
> > > >> After a lot of discussions with different, we received very fruitful
> > > >>
> > > >> feedback and reworked the ideas behind this FLIP. Initially, we had
> > > >>
> > > >> the impression that the compaction problem is solvable by a single
> > > >>
> > > >> topology that we can reuse across different sinks. We now have a
> > > >>
> > > >> better understanding that different external systems require
> different
> > > >>
> > > >> compaction mechanism i.e. Hive requires compaction before finally
> > > >>
> > > >> registering the file in the metastore or Iceberg compacts the files
> > > >>
> > > >> after they have been registered and just lazily compacts them.
> > > >>
> > > >>
> > > >>
> > > >> Considering all these different views we came up with a design that
> > > >>
> > > >> builds upon what @guowei.mgw@gmail.com and @yungao.gy@aliyun.com
> have
> > > >>
> > > >> proposed at the beginning. We allow inserting custom topologies
> before
> > > >>
> > > >> and after the SinkWriters and Committers. Furthermore, we do not see
> > > >>
> > > >> it as a downside. The Sink interfaces that will expose the
> DataStream
> > > >>
> > > >> to the user reside in flink-streaming-java in contrast to the basic
> > > >>
> > > >> Sink interfaces that reside fin flink-core deem it to be only used
> by
> > > >>
> > > >> expert users.
> > > >>
> > > >>
> > > >>
> > > >> Moreover, we also wanted to remove the global committer from the
> > > >>
> > > >> unified Sink interfaces and replace it with a custom post-commit
> > > >>
> > > >> topology. Unfortunately, we cannot do it without breaking the Sink
> > > >>
> > > >> interface since the GlobalCommittables are part of the parameterized
> > > >>
> > > >> Sink interface. Thus, we propose building a new Sink V2 interface
> > > >>
> > > >> consisting of composable interfaces that do not offer the
> > > >>
> > > >> GlobalCommitter anymore. We will implement a utility to extend a
> Sink
> > > >>
> > > >> with post topology that mimics the behavior of the GlobalCommitter.
> > > >>
> > > >> The new Sink V2 provides the same sort of methods as the Sink V1
> > > >>
> > > >> interface, so a migration of sinks that do not use the
> GlobalCommitter
> > > >>
> > > >> should be very easy.
> > > >>
> > > >> We plan to keep the existing Sink V1 interfaces to not break
> > > >>
> > > >> externally built sinks. As part of this FLIP, we migrate all the
> > > >>
> > > >> connectors inside of the main repository to the new Sink V2 API.
> > > >>
> > > >>
> > > >>
> > > >> The FLIP document is also updated and includes the proposed changes.
> > > >>
> > > >>
> > > >>
> > > >> Looking forward to your feedback,
> > > >>
> > > >> Fabian
> > > >>
> > > >>
> > > >>
> > > >>
> > >
> https://cwiki.apache.org/confluence/display/FLINK/FLIP-191%3A+Extend+unified+Sink+interface+to+support+small+file+compaction
> > > >>
> > > >>
> > > >>
> > > >>
> > > >>
> > > >> On Thu, Dec 2, 2021 at 10:15 AM Roman Khachatryan wrote:
> > > >>
> > > >> >
> > > >>
> > > >> > Thanks for clarifying (I was initially confused by merging state
> files
> > > >>
> > > >> > rather than output files).
> > > >>
> > > >> >
> > > >>
> > > >> > > At some point, Flink will definitely have some WAL adapter that
> can
> > > turn any sink into an exactly-once sink (with some caveats). For now,
> we
> > > keep that as an orthogonal solution as it has a rather high price
> (bursty
> > > workload with high latency). Ideally, we can keep the compaction
> > > asynchronously...
> > > >>
> > > >> >
> > > >>
> > > >> > Yes, that would be something like a WAL. I agree that it would
> have a
> > > >>
> > > >> > different set of trade-offs.
> > > >>
> > > >> >
> > > >>
> > > >> >
> > > >>
> > > >> > Regards,
> > > >>
> > > >> > Roman
> > > >>
> > > >> >
> > > >>
> > > >> > On Mon, Nov 29, 2021 at 3:33 PM Arvid Heise wrote:
> > > >>
> > > >> > >>
> > > >>
> > > >> > >> > One way to avoid write-read-merge is by wrapping SinkWriter
> with
> > > >>
> > > >> > >> > another one, which would buffer input elements in a temporary
> > > storage
> > > >>
> > > >> > >> > (e.g. local file) until a threshold is reached; after that,
> it
> > > would
> > > >>
> > > >> > >> > invoke the original SinkWriter. And if a checkpoint barrier
> > > comes in
> > > >>
> > > >> > >> > earlier, it would send written data to some aggregator.
> > > >>
> > > >> > >>
> > > >>
> > > >> > >> I think perhaps this seems to be a kind of WAL method? Namely
> we
> > > first
> > > >>
> > > >> > >> write the elements to some WAL logs and persist them on
> checkpoint
> > > >>
> > > >> > >> (in snapshot or remote FS), or we directly write WAL logs to
> the
> > > remote
> > > >>
> > > >> > >> FS eagerly.
> > > >>
> > > >> > >>
> > > >>
> > > >> > > At some point, Flink will definitely have some WAL adapter that
> can
> > > turn any sink into an exactly-once sink (with some caveats). For now,
> we
> > > keep that as an orthogonal solution as it has a rather high price
> (bursty
> > > workload with high latency). Ideally, we can keep the compaction
> > > asynchronously...
> > > >>
> > > >> > >
> > > >>
> > > >> > > On Mon, Nov 29, 2021 at 8:52 AM Yun Gao wrote:
> > > >>
> > > >> > >>
> > > >>
> > > >> > >> Hi,
> > > >>
> > > >> > >>
> > > >>
> > > >> > >> @Roman very sorry for the late response for a long time,
> > > >>
> > > >> > >>
> > > >>
> > > >> > >> > Merging artifacts from multiple checkpoints would apparently
> > > >>
> > > >> > >> require multiple concurrent checkpoints
> > > >>
> > > >> > >>
> > > >>
> > > >> > >> I think it might not need concurrent checkpoints: suppose some
> > > >>
> > > >> > >> operators (like the committer aggregator in the option 2)
> maintains
> > > >>
> > > >> > >> the list of files to merge, it could stores the lists of files
> to
> > > merge
> > > >>
> > > >> > >> in the states, then after several checkpoints are done and we
> have
> > > >>
> > > >> > >> enough files, we could merge all the files in the list.
> > > >>
> > > >> > >>
> > > >>
> > > >> > >> > Asynchronous merging in an aggregator would require some
> > > resolution
> > > >>
> > > >> > >> > logic on recovery, so that a merged artifact can be used if
> the
> > > >>
> > > >> > >> > original one was deleted. Otherwise, wouldn't recovery fail
> > > because
> > > >>
> > > >> > >> > some artifacts are missing?
> > > >>
> > > >> > >> > We could also defer deletion until the "compacted"
> checkpoint is
> > > >>
> > > >> > >> > subsumed - but isn't it too late, as it will be deleted
> anyways
> > > once
> > > >>
> > > >> > >> > subsumed?
> > > >>
> > > >> > >>
> > > >>
> > > >> > >> I think logically we could delete the original files once the
> > > "compacted" checkpoint
> > > >>
> > > >> > >> (which finish merging the compacted files and record it in the
> > > checkpoint) is completed
> > > >>
> > > >> > >> in all the options. If there are failover before we it, we
> could
> > > restart the merging and if
> > > >>
> > > >> > >> there are failover after it, we could have already recorded the
> > > files in the checkpoint.
> > > >>
> > > >> > >>
> > > >>
> > > >> > >> > One way to avoid write-read-merge is by wrapping SinkWriter
> with
> > > >>
> > > >> > >> > another one, which would buffer input elements in a temporary
> > > storage
> > > >>
> > > >> > >> > (e.g. local file) until a threshold is reached; after that,
> it
> > > would
> > > >>
> > > >> > >> > invoke the original SinkWriter. And if a checkpoint barrier
> > > comes in
> > > >>
> > > >> > >> > earlier, it would send written data to some aggregator.
> > > >>
> > > >> > >>
> > > >>
> > > >> > >> I think perhaps this seems to be a kind of WAL method? Namely
> we
> > > first
> > > >>
> > > >> > >> write the elements to some WAL logs and persist them on
> checkpoint
> > > >>
> > > >> > >> (in snapshot or remote FS), or we directly write WAL logs to
> the
> > > remote
> > > >>
> > > >> > >> FS eagerly.
> > > >>
> > > >> > >>
> > > >>
> > > >> > >> Sorry if I do not understand correctly somewhere.
> > > >>
> > > >> > >>
> > > >>
> > > >> > >> Best,
> > > >>
> > > >> > >> Yun
> > > >>
> > > >> > >>
> > > >>
> > > >> > >>
> > > >>
> > > >> > >>
> ------------------------------------------------------------------
> > > >>
> > > >> > >> From:Roman Khachatryan
> > > >>
> > > >> > >> Send Time:2021 Nov. 9 (Tue.) 22:03
> > > >>
> > > >> > >> To:dev
> > > >>
> > > >> > >> Subject:Re: [DISCUSS] FLIP-191: Extend unified Sink interface
> to
> > > support small file compaction
> > > >>
> > > >> > >>
> > > >>
> > > >> > >> Hi everyone,
> > > >>
> > > >> > >>
> > > >>
> > > >> > >> Thanks for the proposal and the discussion, I have some
> remarks:
> > > >>
> > > >> > >> (I'm not very familiar with the new Sink API but I thought
> about
> > > the
> > > >>
> > > >> > >> same problem in context of the changelog state backend)
> > > >>
> > > >> > >>
> > > >>
> > > >> > >> 1. Merging artifacts from multiple checkpoints would apparently
> > > >>
> > > >> > >> require multiple concurrent checkpoints (otherwise, a new
> > > checkpoint
> > > >>
> > > >> > >> won't be started before completing the previous one; and the
> > > previous
> > > >>
> > > >> > >> one can't be completed before durably storing the artifacts).
> > > However,
> > > >>
> > > >> > >> concurrent checkpoints are currently not supported with
> Unaligned
> > > >>
> > > >> > >> checkpoints (this is besides increasing e2e-latency).
> > > >>
> > > >> > >>
> > > >>
> > > >> > >> 2. Asynchronous merging in an aggregator would require some
> > > resolution
> > > >>
> > > >> > >> logic on recovery, so that a merged artifact can be used if the
> > > >>
> > > >> > >> original one was deleted. Otherwise, wouldn't recovery fail
> because
> > > >>
> > > >> > >> some artifacts are missing?
> > > >>
> > > >> > >> We could also defer deletion until the "compacted" checkpoint
> is
> > > >>
> > > >> > >> subsumed - but isn't it too late, as it will be deleted anyways
> > > once
> > > >>
> > > >> > >> subsumed?
> > > >>
> > > >> > >>
> > > >>
> > > >> > >> 3. Writing small files, then reading and merging them for
> *every*
> > > >>
> > > >> > >> checkpoint seems worse than only reading them on recovery. I
> guess
> > > I'm
> > > >>
> > > >> > >> missing some cases of reading, so to me it would make sense to
> > > mention
> > > >>
> > > >> > >> these cases explicitly in the FLIP motivation section.
> > > >>
> > > >> > >>
> > > >>
> > > >> > >> 4. One way to avoid write-read-merge is by wrapping SinkWriter
> with
> > > >>
> > > >> > >> another one, which would buffer input elements in a temporary
> > > storage
> > > >>
> > > >> > >> (e.g. local file) until a threshold is reached; after that, it
> > > would
> > > >>
> > > >> > >> invoke the original SinkWriter. And if a checkpoint barrier
> comes
> > > in
> > > >>
> > > >> > >> earlier, it would send written data to some aggregator. It will
> > > >>
> > > >> > >> increase checkpoint delay (async phase) compared to the current
> > > Flink;
> > > >>
> > > >> > >> but not compared to the write-read-merge solution, IIUC.
> > > >>
> > > >> > >> Then such "BufferingSinkWriters" could aggregate input elements
> > > from
> > > >>
> > > >> > >> each other, potentially recursively (I mean something like
> > > >>
> > > >> > >>
> > >
> https://cwiki.apache.org/confluence/download/attachments/173082889/DSTL-DFS-DAG.png
> > > >>
> > > >> > >> )
> > > >>
> > > >> > >>
> > > >>
> > > >> > >> 5. Reducing the number of files by reducing aggregator
> parallelism
> > > as
> > > >>
> > > >> > >> opposed to merging on reaching size threshold will likely be
> less
> > > >>
> > > >> > >> optimal and more difficult to configure. OTH, thresholds might
> be
> > > more
> > > >>
> > > >> > >> difficult to implement and (with recursive merging) would incur
> > > higher
> > > >>
> > > >> > >> latency. Maybe that's also something to decide explicitly or at
> > > least
> > > >>
> > > >> > >> mention in the FLIP.
> > > >>
> > > >> > >>
> > > >>
> > > >> > >>
> > > >>
> > > >> > >>
> > > >>
> > > >> > >> Regards,
> > > >>
> > > >> > >> Roman
> > > >>
> > > >> > >>
> > > >>
> > > >> > >>
> > > >>
> > > >> > >> On Tue, Nov 9, 2021 at 5:23 AM Reo Lei wrote:
> > > >>
> > > >> > >> >
> > > >>
> > > >> > >> > Hi Fabian,
> > > >>
> > > >> > >> >
> > > >>
> > > >> > >> > Thanks for drafting the FLIP and trying to support small file
> > > compaction. I
> > > >>
> > > >> > >> > think this feature is very urgent and valuable for users(at
> > > least for me).
> > > >>
> > > >> > >> >
> > > >>
> > > >> > >> > Currently I am trying to support streaming rewrite(compact)
> for
> > > Iceberg on
> > > >>
> > > >> > >> > PR#3323 . As Steven mentioned,
> > > >>
> > > >> > >> > Iceberg sink and compact data through the following steps:
> > > >>
> > > >> > >> > Step-1: Some parallel data writer(sinker) to write streaming
> > > data as files.
> > > >>
> > > >> > >> > Step-2: A single parallelism data files committer to commit
> the
> > > completed
> > > >>
> > > >> > >> > files as soon as possible to make them available.
> > > >>
> > > >> > >> > Step-3: Some parallel file rewriter(compactor) to collect
> > > committed files
> > > >>
> > > >> > >> > from multiple checkpoints, and rewriter(compact) them
> together
> > > once the
> > > >>
> > > >> > >> > total file size or number of files reach the threshold.
> > > >>
> > > >> > >> > Step-4: A single parallelism rewrite(compact) result
> committer
> > > to commit
> > > >>
> > > >> > >> > the rewritten(compacted) files to replace the old files and
> make
> > > them
> > > >>
> > > >> > >> > available.
> > > >>
> > > >> > >> >
> > > >>
> > > >> > >> >
> > > >>
> > > >> > >> > If Flink want to support small file compaction, some key
> point I
> > > think is
> > > >>
> > > >> > >> > necessary:
> > > >>
> > > >> > >> >
> > > >>
> > > >> > >> > 1, Compact files from multiple checkpoints.
> > > >>
> > > >> > >> > I totally agree with Jingsong, because completed file size
> > > usually could
> > > >>
> > > >> > >> > not reach the threshold in a single checkpoint. Especially
> for
> > > partitioned
> > > >>
> > > >> > >> > table, we need to compact the files of each partition, but
> > > usually the file
> > > >>
> > > >> > >> > size of each partition will be different and may not reach
> the
> > > merge
> > > >>
> > > >> > >> > threshold. If we compact these files, in a single checkpoint,
> > > regardless of
> > > >>
> > > >> > >> > whether the total file size reaches the threshold, then the
> > > value of
> > > >>
> > > >> > >> > compacting will be diminished and we will still get small
> files
> > > because
> > > >>
> > > >> > >> > these compacted files are not reach to target size. So we
> need
> > > the
> > > >>
> > > >> > >> > compactor to collect committed files from multiple
> checkpoints
> > > and compact
> > > >>
> > > >> > >> > them until they reach the threshold.
> > > >>
> > > >> > >> >
> > > >>
> > > >> > >> > 2, Separate write phase and compact phase.
> > > >>
> > > >> > >> > Users usually hope the data becomes available as soon as
> > > possible, and the
> > > >>
> > > >> > >> > end-to-end latency is very important. I think we need to
> > > separate the
> > > >>
> > > >> > >> > write and compact phase. For the write phase, there include
> the
> > > Step-1
> > > >>
> > > >> > >> > and Step-2, we sink data as file and commit it pre checkpoint
> > > and regardless
> > > >>
> > > >> > >> > of whether the file size it is. That could ensure the data
> will
> > > be
> > > >>
> > > >> > >> > available ASAP. For the compact phase, there include the
> Step-3
> > > >>
> > > >> > >> > and Step-4, the compactor should collect committed files from
> > > multiple
> > > >>
> > > >> > >> > checkpoints and compact them asynchronously once they reach
> the
> > > threshold,
> > > >>
> > > >> > >> > and the compact committer will commit the compaction result
> in
> > > the next
> > > >>
> > > >> > >> > checkpoint. We compact the committed files asynchronously
> > > because we don't
> > > >>
> > > >> > >> > want the compaction to affect the data sink or the whole
> > > pipeline.
> > > >>
> > > >> > >> >
> > > >>
> > > >> > >> > 3, Exactly once guarantee between write and compact phase.
> > > >>
> > > >> > >> > Once we separate write phase and compact phase, we need to
> > > consider
> > > >>
> > > >> > >> > how to guarantee
> > > >>
> > > >> > >> > the exact once semantic between two phases. We should not
> lose
> > > any data or
> > > >>
> > > >> > >> > files on the compactor(Step-3) in any case and cause the
> > > compaction result
> > > >>
> > > >> > >> > to be inconsistent with before. I think flink should provide
> an
> > > easy-to-use
> > > >>
> > > >> > >> > interface to make that easier.
> > > >>
> > > >> > >> >
> > > >>
> > > >> > >> > 4, Metadata operation and compaction result validation.
> > > >>
> > > >> > >> > In the compact phase, there may be not only compact files,
> but
> > > also a lot
> > > >>
> > > >> > >> > of metadata operations, such as the iceberg needing to
> > > read/write manifest
> > > >>
> > > >> > >> > and do MOR. And we need some interface to support users to do
> > > some
> > > >>
> > > >> > >> > validation of the compaction result. I think these points
> should
> > > be
> > > >>
> > > >> > >> > considered when we design the compaction API.
> > > >>
> > > >> > >> >
> > > >>
> > > >> > >> >
> > > >>
> > > >> > >> > Back to FLIP-191, option 1 looks very complicated while
> option 2
> > > is
> > > >>
> > > >> > >> > relatively simple, but neither of these two solutions
> separates
> > > the write
> > > >>
> > > >> > >> > phase from the compact phase. So I think we should consider
> the
> > > points I
> > > >>
> > > >> > >> > mentioned above. And if you have any other questions you can
> > > always feel
> > > >>
> > > >> > >> > free to reach out to me!
> > > >>
> > > >> > >> >
> > > >>
> > > >> > >> > BR,
> > > >>
> > > >> > >> > Reo
> > > >>
> > > >> > >> >
> > > >>
> > > >> > >> > Fabian Paul 于2021年11月8日周一 下午7:59写道:
> > > >>
> > > >> > >> >
> > > >>
> > > >> > >> > > Hi all,
> > > >>
> > > >> > >> > >
> > > >>
> > > >> > >> > > Thanks for the lively discussions. I am really excited to
> see
> > > so many
> > > >>
> > > >> > >> > > people
> > > >>
> > > >> > >> > > participating in this thread. It also underlines the need
> that
> > > many people
> > > >>
> > > >> > >> > > would
> > > >>
> > > >> > >> > > like to see a solution soon.
> > > >>
> > > >> > >> > >
> > > >>
> > > >> > >> > > I have updated the FLIP and removed the parallelism
> > > configuration because
> > > >>
> > > >> > >> > > it is
> > > >>
> > > >> > >> > > unnecessary since users can configure a constant exchange
> key
> > > to send all
> > > >>
> > > >> > >> > > committables to only one committable aggregator.
> > > >>
> > > >> > >> > >
> > > >>
> > > >> > >> > >
> > > >>
> > > >> > >> > > 1. Burden for developers w.r.t batch stream unification.
> > > >>
> > > >> > >> > >
> > > >>
> > > >> > >> > > @yun @guowei, from a theoretical point you are right about
> > > exposing the
> > > >>
> > > >> > >> > > DataStream
> > > >>
> > > >> > >> > > API in the sink users have the full power to write correct
> > > batch and
> > > >>
> > > >> > >> > > streaming
> > > >>
> > > >> > >> > > sinks. I think in reality a lot of users still struggle to
> > > build pipelines
> > > >>
> > > >> > >> > > with
> > > >>
> > > >> > >> > > i.e. the operator pipeline which works correct in streaming
> > > and batch mode.
> > > >>
> > > >> > >> > > Another problem I see is by exposing more deeper concepts
> is
> > > that we
> > > >>
> > > >> > >> > > cannot do
> > > >>
> > > >> > >> > > any optimization because we cannot reason about how sinks
> are
> > > built in the
> > > >>
> > > >> > >> > > future.
> > > >>
> > > >> > >> > >
> > > >>
> > > >> > >> > > We should also try to steer users towards using only
> > > `Functions` to give
> > > >>
> > > >> > >> > > us more
> > > >>
> > > >> > >> > > flexibility to swap the internal operator representation. I
> > > agree with
> > > >>
> > > >> > >> > > @yun we
> > > >>
> > > >> > >> > > should try to make the `ProcessFunction` more versatile to
> > > work on that
> > > >>
> > > >> > >> > > goal but
> > > >>
> > > >> > >> > > I see this as unrelated to the FLIP.
> > > >>
> > > >> > >> > >
> > > >>
> > > >> > >> > >
> > > >>
> > > >> > >> > > 2. Regarding Commit / Global commit
> > > >>
> > > >> > >> > >
> > > >>
> > > >> > >> > > I envision the global committer to be specific depending on
> > > the data lake
> > > >>
> > > >> > >> > > solution you want to write to. However, it is entirely
> > > orthogonal to the
> > > >>
> > > >> > >> > > compaction.
> > > >>
> > > >> > >> > > Currently, I do not expect any changes w.r.t the Global
> commit
> > > introduces
> > > >>
> > > >> > >> > > by
> > > >>
> > > >> > >> > > this FLIP.
> > > >>
> > > >> > >> > >
> > > >>
> > > >> > >> > >
> > > >>
> > > >> > >> > > 3. Regarding the case of trans-checkpoints merging
> > > >>
> > > >> > >> > >
> > > >>
> > > >> > >> > > @yun, as user, I would expect that if the committer
> receives
> > > in a
> > > >>
> > > >> > >> > > checkpoint files
> > > >>
> > > >> > >> > > to merge/commit that these are also finished when the
> > > checkpoint finishes.
> > > >>
> > > >> > >> > > I think all sinks rely on this principle currently i.e.,
> > > KafkaSink needs to
> > > >>
> > > >> > >> > > commit all open transactions until the next checkpoint can
> > > happen.
> > > >>
> > > >> > >> > >
> > > >>
> > > >> > >> > > Maybe in the future, we can somehow move the
> Committer#commit
> > > call to an
> > > >>
> > > >> > >> > > asynchronous execution, but we should discuss it as a
> separate
> > > thread.
> > > >>
> > > >> > >> > >
> > > >>
> > > >> > >> > > > We probably should first describe the different causes of
> > > small files and
> > > >>
> > > >> > >> > > > what problems was this proposal trying to solve. I wrote
> a
> > > data shuffling
> > > >>
> > > >> > >> > > > proposal [1] for Flink Iceberg sink (shared with Iceberg
> > > community [2]).
> > > >>
> > > >> > >> > > It
> > > >>
> > > >> > >> > > > can address small files problems due to skewed data
> > > distribution across
> > > >>
> > > >> > >> > > > Iceberg table partitions. Streaming shuffling before
> writers
> > > (to files)
> > > >>
> > > >> > >> > > is
> > > >>
> > > >> > >> > > > typically more efficient than post-write file compaction
> > > (which involves
> > > >>
> > > >> > >> > > > read-merge-write). It is usually cheaper to prevent a
> > > problem (small
> > > >>
> > > >> > >> > > files)
> > > >>
> > > >> > >> > > > than fixing it.
> > > >>
> > > >> > >> > >
> > > >>
> > > >> > >> > >
> > > >>
> > > >> > >> > > @steven you are raising a good point, although I think only
> > > using a
> > > >>
> > > >> > >> > > customizable
> > > >>
> > > >> > >> > > shuffle won't address the generation of small files. One
> > > assumption is that
> > > >>
> > > >> > >> > > at least the sink generates one file per subtask, which can
> > > already be too
> > > >>
> > > >> > >> > > many.
> > > >>
> > > >> > >> > > Another problem is that with low checkpointing intervals,
> the
> > > files do not
> > > >>
> > > >> > >> > > meet
> > > >>
> > > >> > >> > > the required size. The latter point is probably
> addressable by
> > > changing the
> > > >>
> > > >> > >> > > checkpoint interval, which might be inconvenient for some
> > > users.
> > > >>
> > > >> > >> > >
> > > >>
> > > >> > >> > > > The sink coordinator checkpoint problem (mentioned in
> option
> > > 1) would be
> > > >>
> > > >> > >> > > > great if Flink can address it. In the spirit of source
> > > >>
> > > >> > >> > > (enumerator-reader)
> > > >>
> > > >> > >> > > > and sink (writer-coordinator) duality, sink coordinator
> > > checkpoint should
> > > >>
> > > >> > >> > > > happen after the writer operator. This would be a natural
> > > fit to support
> > > >>
> > > >> > >> > > > global committer in FLIP-143. It is probably an
> orthogonal
> > > matter to this
> > > >>
> > > >> > >> > > > proposal.
> > > >>
> > > >> > >> > >
> > > >>
> > > >> > >> > >
> > > >>
> > > >> > >> > > To me the question here is what are the benefits of having
> a
> > > coordinator in
> > > >>
> > > >> > >> > > comparison to a global committer/aggregator operator.
> > > >>
> > > >> > >> > >
> > > >>
> > > >> > >> > > > Personally, I am usually in favor of keeping streaming
> > > ingestion (to data
> > > >>
> > > >> > >> > > > lake) relatively simple and stable. Also sometimes
> > > compaction and sorting
> > > >>
> > > >> > >> > > > are performed together in data rewrite maintenance jobs
> to
> > > improve read
> > > >>
> > > >> > >> > > > performance. In that case, the value of compacting (in
> Flink
> > > streaming
> > > >>
> > > >> > >> > > > ingestion) diminishes.
> > > >>
> > > >> > >> > >
> > > >>
> > > >> > >> > >
> > > >>
> > > >> > >> > > I agree it is always possible to have scheduled maintenance
> > > jobs keeping
> > > >>
> > > >> > >> > > care of
> > > >>
> > > >> > >> > > your data i.e., doing compaction. Unfortunately, the
> downside
> > > is that you
> > > >>
> > > >> > >> > > have to your data after it is already available for other
> > > downstream
> > > >>
> > > >> > >> > > consumers.
> > > >>
> > > >> > >> > > I guess this can lead to all kinds of visibility problems.
> I
> > > am also
> > > >>
> > > >> > >> > > surprised that
> > > >>
> > > >> > >> > > you personally are a fan of this approach and, on the other
> > > hand, are
> > > >>
> > > >> > >> > > developing
> > > >>
> > > >> > >> > > the Iceberg sink, which goes somewhat against your
> mentioned
> > > principle of
> > > >>
> > > >> > >> > > keeping
> > > >>
> > > >> > >> > > the sink simple.
> > > >>
> > > >> > >> > >
> > > >>
> > > >> > >> > > > Currently, it is unclear from the doc and this thread
> where
> > > the
> > > >>
> > > >> > >> > > compaction
> > > >>
> > > >> > >> > > > is actually happening. Jingsong's reply described one
> model
> > > >>
> > > >> > >> > > > writer (parallel) -> aggregator (single-parallelism
> > > compaction planner)
> > > >>
> > > >> > >> > > ->
> > > >>
> > > >> > >> > > > compactor (parallel) -> global committer
> (single-parallelism)
> > > >>
> > > >> > >> > >
> > > >>
> > > >> > >> > >
> > > >>
> > > >> > >> > > My idea of the topology is very similar to the one
> outlined by
> > > Jinsong. The
> > > >>
> > > >> > >> > > compaction will happen in the committer operator.
> > > >>
> > > >> > >> > >
> > > >>
> > > >> > >> > > >
> > > >>
> > > >> > >> > > > In the Iceberg community, the following model has been
> > > discussed. It is
> > > >>
> > > >> > >> > > > better for Iceberg because it won't delay the data
> > > availability.
> > > >>
> > > >> > >> > > > writer (parallel) -> global committer for append (single
> > > parallelism) ->
> > > >>
> > > >> > >> > > > compactor (parallel) -> global committer for rewrite
> commit
> > > (single
> > > >>
> > > >> > >> > > > parallelism)
> > > >>
> > > >> > >> > >
> > > >>
> > > >> > >> > >
> > > >>
> > > >> > >> > > From a quick glimpse, it seems that the exact same
> topology is
> > > possible to
> > > >>
> > > >> > >> > > express with the committable aggregator, but this
> definitely
> > > depends on
> > > >>
> > > >> > >> > > the exact
> > > >>
> > > >> > >> > > setup.
> > > >>
> > > >> > >> > >
> > > >>
> > > >> > >> > > Best,
> > > >>
> > > >> > >> > > Fabian
> > > >>
> > > >> > >>
> > >
>
>
>

Re: Re: [DISCUSS] FLIP-191: Extend unified Sink interface to support small file compaction

Posted by Yun Gao <yu...@aliyun.com.INVALID>.
Hi,

Very thanks Fabian for the explanation and it solves most of the issues. 
There is one left issue I want to have a double confirmation is that for 
the edges between writer and committer and in the post-committer topology,  
perhaps the result partition with HA storage is not enough solve all the issues 
directly ? It is due to after the committer and post-committer topology is finished
and the data is committed, it might still be restarted due to JM failover and the
deterministic problem (namely the example of  (A -> [B1, B2], A, B1 have finished and
B2 failed, if -> is rebalance / random / rescale,  all of A, B1, B2 would restarted). Then
the records would be produced and created for the second times. 

We might let the writers to skip producing the new records, but if we have multiple sinks like
OP1 -> (writer 1 -> committer 1) 
     |--> (writer 2 -> committer 2)

and the failover happens after writer 1 & committer 1 get finished but writer 2 is running,
if op1 produced different records across the two runs, then the two sinks would produces 
different data, which might be not suitable in some cases. Perhaps we need some support
from the scheduler side? 

But I also agree this could be a separate issue and we could solve it separately in some future
as long as we know how to solve it~

Best,
Yun



------------------------------------------------------------------
From:Arvid Heise <ar...@apache.org>
Send Time:2021 Dec. 16 (Thu.) 19:54
To:dev <de...@flink.apache.org>
Cc:Yun Gao <yu...@aliyun.com>
Subject:Re: Re: [DISCUSS] FLIP-191: Extend unified Sink interface to support small file compaction

Just a quick amend: There will be no blocking exchange in the pre-writer exchange for performance reasons. 
After the writer, we have tiny data volume and are free to add as many as we see necessary.

On Thu, Dec 16, 2021 at 11:18 AM Fabian Paul <fp...@apache.org> wrote:
Hi Till,

 Good point, in the scenario with the blocking keyed exchange between
 the writer and committer my idea is to make the committer effectively
 the global committer. With Sink V2 there is no real difference anymore
 between the committer and global committer.
 You are right that everything after the committer would be part of the
 same failover region but we plan to insert a blocking exchange by
 default before all of the custom topologies.

 Best,
 Fabian

 On Thu, Dec 16, 2021 at 11:08 AM Till Rohrmann <tr...@apache.org> wrote:
 >
 > Hi Fabian,
 >
 > quick question on your comment 3. If there is a pipelined data exchange
 > with a keyBy between the writers/committers and the component that does the
 > global commit, then there will only be a single failover region. So is it
 > correct that you assumed blocking data exchanges for the scenario you
 > described?
 >
 > Cheers,
 > Till
 >
 > On Thu, Dec 16, 2021 at 9:23 AM Fabian Paul <fp...@apache.org> wrote:
 >
 > > Hi Yun,
 > >
 > > Thanks for your fast feedback. Let me clarify your points.
 > >
 > > 1. We solve it by using StreamExchangeMode.BATCH before any exchange.
 > > That obviously doesn’t help with lost TM but we would need to employ
 > > HA storage for that. Same issue as now and orthogonal.
 > >
 > > 2. Extending V1 with V2 or vice versa would require renames of methods
 > > (since return types are non-optional) and is making API changes. Even
 > > though Experimental, we want to give connector developers the
 > > opportunity to provide 1 implementation for all of Flink 1.X. We will
 > > offer an internal adapter from V1 to V2, 2 sinkTo , and internally
 > > just have one code-path.
 > >
 > > 3. DataStreamSink would act as a unified view on all the operators and
 > > update them all at once when using setParallelism and so on (setName
 > > and setUid will receive suffixes per operator).
 > > Iceberg actually has a different requirement: They want to have a
 > > committer with parallelism 1 but as a coordinator such that
 > > embarrassingly parallel pipelines have different fail-over regions. I
 > > was thinking that in this case, they need to implement a no-op
 > > committer (that just forwards the committables) and use a post-commit
 > > topology that achieves that.
 > > Another option is that they use the preCommit topology and insert a
 > > constant key-by that forwards all committables to a single committer.
 > > We are planning to provide building blocks for such pipelines as we
 > > go.
 > >
 > > Best,
 > > Fabian
 > >
 > > On Thu, Dec 16, 2021 at 5:50 AM Yun Gao <yu...@aliyun.com> wrote:
 > > >
 > > > Hi Fabian,
 > > >
 > > > Very thanks for the update! I think the latest version in general looks
 > > good from my side
 > > > and I think using separate feature interface would be much more easy to
 > > understand
 > > > and extend in the future. I have some pending issues on the details
 > > though:
 > > >
 > > > 1. The first one is if we could support end-to-end exactly-once with
 > > post-committing
 > > > topology in the batch mode ? Since for the batch mode, currently we
 > > could only commit
 > > >  all the transactions after the whole job is finished, otherwise if
 > > there are JM failover or the
 > > > writer / committer get restarted due to indeterminstic (A -> [B1, B2],
 > > A, B1 have finished and
 > > >  B2 failed, if -> is rebalance / random / rescale,  all of A, B1, B2
 > > would restarted), there might
 > > > be repeat records. Previously one possible thought is to move committer
 > > and global committer
 > > >  to the operator coordinator, but if it is a topology, we might need
 > > some other kind of solutions?
 > > >
 > > > 2. I also want to have a dobule confirmation with the compatibility:
 > > since the old sink is also named
 > > > with Sink, do we want to put the Sink v2 in a new package ? Besides,
 > > since we might want to keep
 > > > only have one `sinkTo(Sink<?> sink)` , perhaps we also need to make the
 > > Sink v1 to be a subclass of
 > > > Sink v2 and extends the stateful and two-phase-commit sinks, right?
 > > >
 > > > 3. I'd like also have a confirmation on ours thoughts with the
 > > `DataStreamSink` returned by the sinkTo method:
 > > > The main issue is how do we implement the method like `setParallelism`
 > > or `setMaxParallelism` since now the sink
 > > > would be translated to multiple transformations? perhaps we could make
 > > it the default values for all the transformations
 > > > for the sink? A related issue would be for iceberg sink, I think it
 > > would need to have only one committer to avoid the
 > > > competition of the optimistic locks (which would cause performance
 > > degradation), then it might need to have N writers
 > > > with 1 committers, to build such topology, perhaps we might need to add
 > > new methods to specify the parallelism of
 > > > the writers and committers separately?
 > > >
 > > > Best,
 > > > Yun
 > > >
 > > >
 > > > ------------------Original Mail ------------------
 > > > Sender:Fabian Paul <fp...@apache.org>
 > > > Send Date:Mon Dec 13 23:59:43 2021
 > > > Recipients:dev <de...@flink.apache.org>
 > > > Subject:Re: [DISCUSS] FLIP-191: Extend unified Sink interface to support
 > > small file compaction
 > > >>
 > > >> Hi all,
 > > >>
 > > >>
 > > >>
 > > >> After a lot of discussions with different, we received very fruitful
 > > >>
 > > >> feedback and reworked the ideas behind this FLIP. Initially, we had
 > > >>
 > > >> the impression that the compaction problem is solvable by a single
 > > >>
 > > >> topology that we can reuse across different sinks. We now have a
 > > >>
 > > >> better understanding that different external systems require different
 > > >>
 > > >> compaction mechanism i.e. Hive requires compaction before finally
 > > >>
 > > >> registering the file in the metastore or Iceberg compacts the files
 > > >>
 > > >> after they have been registered and just lazily compacts them.
 > > >>
 > > >>
 > > >>
 > > >> Considering all these different views we came up with a design that
 > > >>
 > > >> builds upon what @guowei.mgw@gmail.com and @yungao.gy@aliyun.com have
 > > >>
 > > >> proposed at the beginning. We allow inserting custom topologies before
 > > >>
 > > >> and after the SinkWriters and Committers. Furthermore, we do not see
 > > >>
 > > >> it as a downside. The Sink interfaces that will expose the DataStream
 > > >>
 > > >> to the user reside in flink-streaming-java in contrast to the basic
 > > >>
 > > >> Sink interfaces that reside fin flink-core deem it to be only used by
 > > >>
 > > >> expert users.
 > > >>
 > > >>
 > > >>
 > > >> Moreover, we also wanted to remove the global committer from the
 > > >>
 > > >> unified Sink interfaces and replace it with a custom post-commit
 > > >>
 > > >> topology. Unfortunately, we cannot do it without breaking the Sink
 > > >>
 > > >> interface since the GlobalCommittables are part of the parameterized
 > > >>
 > > >> Sink interface. Thus, we propose building a new Sink V2 interface
 > > >>
 > > >> consisting of composable interfaces that do not offer the
 > > >>
 > > >> GlobalCommitter anymore. We will implement a utility to extend a Sink
 > > >>
 > > >> with post topology that mimics the behavior of the GlobalCommitter.
 > > >>
 > > >> The new Sink V2 provides the same sort of methods as the Sink V1
 > > >>
 > > >> interface, so a migration of sinks that do not use the GlobalCommitter
 > > >>
 > > >> should be very easy.
 > > >>
 > > >> We plan to keep the existing Sink V1 interfaces to not break
 > > >>
 > > >> externally built sinks. As part of this FLIP, we migrate all the
 > > >>
 > > >> connectors inside of the main repository to the new Sink V2 API.
 > > >>
 > > >>
 > > >>
 > > >> The FLIP document is also updated and includes the proposed changes.
 > > >>
 > > >>
 > > >>
 > > >> Looking forward to your feedback,
 > > >>
 > > >> Fabian
 > > >>
 > > >>
 > > >>
 > > >>
 > > https://cwiki.apache.org/confluence/display/FLINK/FLIP-191%3A+Extend+unified+Sink+interface+to+support+small+file+compaction
 > > >>
 > > >>
 > > >>
 > > >>
 > > >>
 > > >> On Thu, Dec 2, 2021 at 10:15 AM Roman Khachatryan wrote:
 > > >>
 > > >> >
 > > >>
 > > >> > Thanks for clarifying (I was initially confused by merging state files
 > > >>
 > > >> > rather than output files).
 > > >>
 > > >> >
 > > >>
 > > >> > > At some point, Flink will definitely have some WAL adapter that can
 > > turn any sink into an exactly-once sink (with some caveats). For now, we
 > > keep that as an orthogonal solution as it has a rather high price (bursty
 > > workload with high latency). Ideally, we can keep the compaction
 > > asynchronously...
 > > >>
 > > >> >
 > > >>
 > > >> > Yes, that would be something like a WAL. I agree that it would have a
 > > >>
 > > >> > different set of trade-offs.
 > > >>
 > > >> >
 > > >>
 > > >> >
 > > >>
 > > >> > Regards,
 > > >>
 > > >> > Roman
 > > >>
 > > >> >
 > > >>
 > > >> > On Mon, Nov 29, 2021 at 3:33 PM Arvid Heise wrote:
 > > >>
 > > >> > >>
 > > >>
 > > >> > >> > One way to avoid write-read-merge is by wrapping SinkWriter with
 > > >>
 > > >> > >> > another one, which would buffer input elements in a temporary
 > > storage
 > > >>
 > > >> > >> > (e.g. local file) until a threshold is reached; after that, it
 > > would
 > > >>
 > > >> > >> > invoke the original SinkWriter. And if a checkpoint barrier
 > > comes in
 > > >>
 > > >> > >> > earlier, it would send written data to some aggregator.
 > > >>
 > > >> > >>
 > > >>
 > > >> > >> I think perhaps this seems to be a kind of WAL method? Namely we
 > > first
 > > >>
 > > >> > >> write the elements to some WAL logs and persist them on checkpoint
 > > >>
 > > >> > >> (in snapshot or remote FS), or we directly write WAL logs to the
 > > remote
 > > >>
 > > >> > >> FS eagerly.
 > > >>
 > > >> > >>
 > > >>
 > > >> > > At some point, Flink will definitely have some WAL adapter that can
 > > turn any sink into an exactly-once sink (with some caveats). For now, we
 > > keep that as an orthogonal solution as it has a rather high price (bursty
 > > workload with high latency). Ideally, we can keep the compaction
 > > asynchronously...
 > > >>
 > > >> > >
 > > >>
 > > >> > > On Mon, Nov 29, 2021 at 8:52 AM Yun Gao wrote:
 > > >>
 > > >> > >>
 > > >>
 > > >> > >> Hi,
 > > >>
 > > >> > >>
 > > >>
 > > >> > >> @Roman very sorry for the late response for a long time,
 > > >>
 > > >> > >>
 > > >>
 > > >> > >> > Merging artifacts from multiple checkpoints would apparently
 > > >>
 > > >> > >> require multiple concurrent checkpoints
 > > >>
 > > >> > >>
 > > >>
 > > >> > >> I think it might not need concurrent checkpoints: suppose some
 > > >>
 > > >> > >> operators (like the committer aggregator in the option 2) maintains
 > > >>
 > > >> > >> the list of files to merge, it could stores the lists of files to
 > > merge
 > > >>
 > > >> > >> in the states, then after several checkpoints are done and we have
 > > >>
 > > >> > >> enough files, we could merge all the files in the list.
 > > >>
 > > >> > >>
 > > >>
 > > >> > >> > Asynchronous merging in an aggregator would require some
 > > resolution
 > > >>
 > > >> > >> > logic on recovery, so that a merged artifact can be used if the
 > > >>
 > > >> > >> > original one was deleted. Otherwise, wouldn't recovery fail
 > > because
 > > >>
 > > >> > >> > some artifacts are missing?
 > > >>
 > > >> > >> > We could also defer deletion until the "compacted" checkpoint is
 > > >>
 > > >> > >> > subsumed - but isn't it too late, as it will be deleted anyways
 > > once
 > > >>
 > > >> > >> > subsumed?
 > > >>
 > > >> > >>
 > > >>
 > > >> > >> I think logically we could delete the original files once the
 > > "compacted" checkpoint
 > > >>
 > > >> > >> (which finish merging the compacted files and record it in the
 > > checkpoint) is completed
 > > >>
 > > >> > >> in all the options. If there are failover before we it, we could
 > > restart the merging and if
 > > >>
 > > >> > >> there are failover after it, we could have already recorded the
 > > files in the checkpoint.
 > > >>
 > > >> > >>
 > > >>
 > > >> > >> > One way to avoid write-read-merge is by wrapping SinkWriter with
 > > >>
 > > >> > >> > another one, which would buffer input elements in a temporary
 > > storage
 > > >>
 > > >> > >> > (e.g. local file) until a threshold is reached; after that, it
 > > would
 > > >>
 > > >> > >> > invoke the original SinkWriter. And if a checkpoint barrier
 > > comes in
 > > >>
 > > >> > >> > earlier, it would send written data to some aggregator.
 > > >>
 > > >> > >>
 > > >>
 > > >> > >> I think perhaps this seems to be a kind of WAL method? Namely we
 > > first
 > > >>
 > > >> > >> write the elements to some WAL logs and persist them on checkpoint
 > > >>
 > > >> > >> (in snapshot or remote FS), or we directly write WAL logs to the
 > > remote
 > > >>
 > > >> > >> FS eagerly.
 > > >>
 > > >> > >>
 > > >>
 > > >> > >> Sorry if I do not understand correctly somewhere.
 > > >>
 > > >> > >>
 > > >>
 > > >> > >> Best,
 > > >>
 > > >> > >> Yun
 > > >>
 > > >> > >>
 > > >>
 > > >> > >>
 > > >>
 > > >> > >> ------------------------------------------------------------------
 > > >>
 > > >> > >> From:Roman Khachatryan
 > > >>
 > > >> > >> Send Time:2021 Nov. 9 (Tue.) 22:03
 > > >>
 > > >> > >> To:dev
 > > >>
 > > >> > >> Subject:Re: [DISCUSS] FLIP-191: Extend unified Sink interface to
 > > support small file compaction
 > > >>
 > > >> > >>
 > > >>
 > > >> > >> Hi everyone,
 > > >>
 > > >> > >>
 > > >>
 > > >> > >> Thanks for the proposal and the discussion, I have some remarks:
 > > >>
 > > >> > >> (I'm not very familiar with the new Sink API but I thought about
 > > the
 > > >>
 > > >> > >> same problem in context of the changelog state backend)
 > > >>
 > > >> > >>
 > > >>
 > > >> > >> 1. Merging artifacts from multiple checkpoints would apparently
 > > >>
 > > >> > >> require multiple concurrent checkpoints (otherwise, a new
 > > checkpoint
 > > >>
 > > >> > >> won't be started before completing the previous one; and the
 > > previous
 > > >>
 > > >> > >> one can't be completed before durably storing the artifacts).
 > > However,
 > > >>
 > > >> > >> concurrent checkpoints are currently not supported with Unaligned
 > > >>
 > > >> > >> checkpoints (this is besides increasing e2e-latency).
 > > >>
 > > >> > >>
 > > >>
 > > >> > >> 2. Asynchronous merging in an aggregator would require some
 > > resolution
 > > >>
 > > >> > >> logic on recovery, so that a merged artifact can be used if the
 > > >>
 > > >> > >> original one was deleted. Otherwise, wouldn't recovery fail because
 > > >>
 > > >> > >> some artifacts are missing?
 > > >>
 > > >> > >> We could also defer deletion until the "compacted" checkpoint is
 > > >>
 > > >> > >> subsumed - but isn't it too late, as it will be deleted anyways
 > > once
 > > >>
 > > >> > >> subsumed?
 > > >>
 > > >> > >>
 > > >>
 > > >> > >> 3. Writing small files, then reading and merging them for *every*
 > > >>
 > > >> > >> checkpoint seems worse than only reading them on recovery. I guess
 > > I'm
 > > >>
 > > >> > >> missing some cases of reading, so to me it would make sense to
 > > mention
 > > >>
 > > >> > >> these cases explicitly in the FLIP motivation section.
 > > >>
 > > >> > >>
 > > >>
 > > >> > >> 4. One way to avoid write-read-merge is by wrapping SinkWriter with
 > > >>
 > > >> > >> another one, which would buffer input elements in a temporary
 > > storage
 > > >>
 > > >> > >> (e.g. local file) until a threshold is reached; after that, it
 > > would
 > > >>
 > > >> > >> invoke the original SinkWriter. And if a checkpoint barrier comes
 > > in
 > > >>
 > > >> > >> earlier, it would send written data to some aggregator. It will
 > > >>
 > > >> > >> increase checkpoint delay (async phase) compared to the current
 > > Flink;
 > > >>
 > > >> > >> but not compared to the write-read-merge solution, IIUC.
 > > >>
 > > >> > >> Then such "BufferingSinkWriters" could aggregate input elements
 > > from
 > > >>
 > > >> > >> each other, potentially recursively (I mean something like
 > > >>
 > > >> > >>
 > > https://cwiki.apache.org/confluence/download/attachments/173082889/DSTL-DFS-DAG.png
 > > >>
 > > >> > >> )
 > > >>
 > > >> > >>
 > > >>
 > > >> > >> 5. Reducing the number of files by reducing aggregator parallelism
 > > as
 > > >>
 > > >> > >> opposed to merging on reaching size threshold will likely be less
 > > >>
 > > >> > >> optimal and more difficult to configure. OTH, thresholds might be
 > > more
 > > >>
 > > >> > >> difficult to implement and (with recursive merging) would incur
 > > higher
 > > >>
 > > >> > >> latency. Maybe that's also something to decide explicitly or at
 > > least
 > > >>
 > > >> > >> mention in the FLIP.
 > > >>
 > > >> > >>
 > > >>
 > > >> > >>
 > > >>
 > > >> > >>
 > > >>
 > > >> > >> Regards,
 > > >>
 > > >> > >> Roman
 > > >>
 > > >> > >>
 > > >>
 > > >> > >>
 > > >>
 > > >> > >> On Tue, Nov 9, 2021 at 5:23 AM Reo Lei wrote:
 > > >>
 > > >> > >> >
 > > >>
 > > >> > >> > Hi Fabian,
 > > >>
 > > >> > >> >
 > > >>
 > > >> > >> > Thanks for drafting the FLIP and trying to support small file
 > > compaction. I
 > > >>
 > > >> > >> > think this feature is very urgent and valuable for users(at
 > > least for me).
 > > >>
 > > >> > >> >
 > > >>
 > > >> > >> > Currently I am trying to support streaming rewrite(compact) for
 > > Iceberg on
 > > >>
 > > >> > >> > PR#3323 . As Steven mentioned,
 > > >>
 > > >> > >> > Iceberg sink and compact data through the following steps:
 > > >>
 > > >> > >> > Step-1: Some parallel data writer(sinker) to write streaming
 > > data as files.
 > > >>
 > > >> > >> > Step-2: A single parallelism data files committer to commit the
 > > completed
 > > >>
 > > >> > >> > files as soon as possible to make them available.
 > > >>
 > > >> > >> > Step-3: Some parallel file rewriter(compactor) to collect
 > > committed files
 > > >>
 > > >> > >> > from multiple checkpoints, and rewriter(compact) them together
 > > once the
 > > >>
 > > >> > >> > total file size or number of files reach the threshold.
 > > >>
 > > >> > >> > Step-4: A single parallelism rewrite(compact) result committer
 > > to commit
 > > >>
 > > >> > >> > the rewritten(compacted) files to replace the old files and make
 > > them
 > > >>
 > > >> > >> > available.
 > > >>
 > > >> > >> >
 > > >>
 > > >> > >> >
 > > >>
 > > >> > >> > If Flink want to support small file compaction, some key point I
 > > think is
 > > >>
 > > >> > >> > necessary:
 > > >>
 > > >> > >> >
 > > >>
 > > >> > >> > 1, Compact files from multiple checkpoints.
 > > >>
 > > >> > >> > I totally agree with Jingsong, because completed file size
 > > usually could
 > > >>
 > > >> > >> > not reach the threshold in a single checkpoint. Especially for
 > > partitioned
 > > >>
 > > >> > >> > table, we need to compact the files of each partition, but
 > > usually the file
 > > >>
 > > >> > >> > size of each partition will be different and may not reach the
 > > merge
 > > >>
 > > >> > >> > threshold. If we compact these files, in a single checkpoint,
 > > regardless of
 > > >>
 > > >> > >> > whether the total file size reaches the threshold, then the
 > > value of
 > > >>
 > > >> > >> > compacting will be diminished and we will still get small files
 > > because
 > > >>
 > > >> > >> > these compacted files are not reach to target size. So we need
 > > the
 > > >>
 > > >> > >> > compactor to collect committed files from multiple checkpoints
 > > and compact
 > > >>
 > > >> > >> > them until they reach the threshold.
 > > >>
 > > >> > >> >
 > > >>
 > > >> > >> > 2, Separate write phase and compact phase.
 > > >>
 > > >> > >> > Users usually hope the data becomes available as soon as
 > > possible, and the
 > > >>
 > > >> > >> > end-to-end latency is very important. I think we need to
 > > separate the
 > > >>
 > > >> > >> > write and compact phase. For the write phase, there include the
 > > Step-1
 > > >>
 > > >> > >> > and Step-2, we sink data as file and commit it pre checkpoint
 > > and regardless
 > > >>
 > > >> > >> > of whether the file size it is. That could ensure the data will
 > > be
 > > >>
 > > >> > >> > available ASAP. For the compact phase, there include the Step-3
 > > >>
 > > >> > >> > and Step-4, the compactor should collect committed files from
 > > multiple
 > > >>
 > > >> > >> > checkpoints and compact them asynchronously once they reach the
 > > threshold,
 > > >>
 > > >> > >> > and the compact committer will commit the compaction result in
 > > the next
 > > >>
 > > >> > >> > checkpoint. We compact the committed files asynchronously
 > > because we don't
 > > >>
 > > >> > >> > want the compaction to affect the data sink or the whole
 > > pipeline.
 > > >>
 > > >> > >> >
 > > >>
 > > >> > >> > 3, Exactly once guarantee between write and compact phase.
 > > >>
 > > >> > >> > Once we separate write phase and compact phase, we need to
 > > consider
 > > >>
 > > >> > >> > how to guarantee
 > > >>
 > > >> > >> > the exact once semantic between two phases. We should not lose
 > > any data or
 > > >>
 > > >> > >> > files on the compactor(Step-3) in any case and cause the
 > > compaction result
 > > >>
 > > >> > >> > to be inconsistent with before. I think flink should provide an
 > > easy-to-use
 > > >>
 > > >> > >> > interface to make that easier.
 > > >>
 > > >> > >> >
 > > >>
 > > >> > >> > 4, Metadata operation and compaction result validation.
 > > >>
 > > >> > >> > In the compact phase, there may be not only compact files, but
 > > also a lot
 > > >>
 > > >> > >> > of metadata operations, such as the iceberg needing to
 > > read/write manifest
 > > >>
 > > >> > >> > and do MOR. And we need some interface to support users to do
 > > some
 > > >>
 > > >> > >> > validation of the compaction result. I think these points should
 > > be
 > > >>
 > > >> > >> > considered when we design the compaction API.
 > > >>
 > > >> > >> >
 > > >>
 > > >> > >> >
 > > >>
 > > >> > >> > Back to FLIP-191, option 1 looks very complicated while option 2
 > > is
 > > >>
 > > >> > >> > relatively simple, but neither of these two solutions separates
 > > the write
 > > >>
 > > >> > >> > phase from the compact phase. So I think we should consider the
 > > points I
 > > >>
 > > >> > >> > mentioned above. And if you have any other questions you can
 > > always feel
 > > >>
 > > >> > >> > free to reach out to me!
 > > >>
 > > >> > >> >
 > > >>
 > > >> > >> > BR,
 > > >>
 > > >> > >> > Reo
 > > >>
 > > >> > >> >
 > > >>
 > > >> > >> > Fabian Paul 于2021年11月8日周一 下午7:59写道:
 > > >>
 > > >> > >> >
 > > >>
 > > >> > >> > > Hi all,
 > > >>
 > > >> > >> > >
 > > >>
 > > >> > >> > > Thanks for the lively discussions. I am really excited to see
 > > so many
 > > >>
 > > >> > >> > > people
 > > >>
 > > >> > >> > > participating in this thread. It also underlines the need that
 > > many people
 > > >>
 > > >> > >> > > would
 > > >>
 > > >> > >> > > like to see a solution soon.
 > > >>
 > > >> > >> > >
 > > >>
 > > >> > >> > > I have updated the FLIP and removed the parallelism
 > > configuration because
 > > >>
 > > >> > >> > > it is
 > > >>
 > > >> > >> > > unnecessary since users can configure a constant exchange key
 > > to send all
 > > >>
 > > >> > >> > > committables to only one committable aggregator.
 > > >>
 > > >> > >> > >
 > > >>
 > > >> > >> > >
 > > >>
 > > >> > >> > > 1. Burden for developers w.r.t batch stream unification.
 > > >>
 > > >> > >> > >
 > > >>
 > > >> > >> > > @yun @guowei, from a theoretical point you are right about
 > > exposing the
 > > >>
 > > >> > >> > > DataStream
 > > >>
 > > >> > >> > > API in the sink users have the full power to write correct
 > > batch and
 > > >>
 > > >> > >> > > streaming
 > > >>
 > > >> > >> > > sinks. I think in reality a lot of users still struggle to
 > > build pipelines
 > > >>
 > > >> > >> > > with
 > > >>
 > > >> > >> > > i.e. the operator pipeline which works correct in streaming
 > > and batch mode.
 > > >>
 > > >> > >> > > Another problem I see is by exposing more deeper concepts is
 > > that we
 > > >>
 > > >> > >> > > cannot do
 > > >>
 > > >> > >> > > any optimization because we cannot reason about how sinks are
 > > built in the
 > > >>
 > > >> > >> > > future.
 > > >>
 > > >> > >> > >
 > > >>
 > > >> > >> > > We should also try to steer users towards using only
 > > `Functions` to give
 > > >>
 > > >> > >> > > us more
 > > >>
 > > >> > >> > > flexibility to swap the internal operator representation. I
 > > agree with
 > > >>
 > > >> > >> > > @yun we
 > > >>
 > > >> > >> > > should try to make the `ProcessFunction` more versatile to
 > > work on that
 > > >>
 > > >> > >> > > goal but
 > > >>
 > > >> > >> > > I see this as unrelated to the FLIP.
 > > >>
 > > >> > >> > >
 > > >>
 > > >> > >> > >
 > > >>
 > > >> > >> > > 2. Regarding Commit / Global commit
 > > >>
 > > >> > >> > >
 > > >>
 > > >> > >> > > I envision the global committer to be specific depending on
 > > the data lake
 > > >>
 > > >> > >> > > solution you want to write to. However, it is entirely
 > > orthogonal to the
 > > >>
 > > >> > >> > > compaction.
 > > >>
 > > >> > >> > > Currently, I do not expect any changes w.r.t the Global commit
 > > introduces
 > > >>
 > > >> > >> > > by
 > > >>
 > > >> > >> > > this FLIP.
 > > >>
 > > >> > >> > >
 > > >>
 > > >> > >> > >
 > > >>
 > > >> > >> > > 3. Regarding the case of trans-checkpoints merging
 > > >>
 > > >> > >> > >
 > > >>
 > > >> > >> > > @yun, as user, I would expect that if the committer receives
 > > in a
 > > >>
 > > >> > >> > > checkpoint files
 > > >>
 > > >> > >> > > to merge/commit that these are also finished when the
 > > checkpoint finishes.
 > > >>
 > > >> > >> > > I think all sinks rely on this principle currently i.e.,
 > > KafkaSink needs to
 > > >>
 > > >> > >> > > commit all open transactions until the next checkpoint can
 > > happen.
 > > >>
 > > >> > >> > >
 > > >>
 > > >> > >> > > Maybe in the future, we can somehow move the Committer#commit
 > > call to an
 > > >>
 > > >> > >> > > asynchronous execution, but we should discuss it as a separate
 > > thread.
 > > >>
 > > >> > >> > >
 > > >>
 > > >> > >> > > > We probably should first describe the different causes of
 > > small files and
 > > >>
 > > >> > >> > > > what problems was this proposal trying to solve. I wrote a
 > > data shuffling
 > > >>
 > > >> > >> > > > proposal [1] for Flink Iceberg sink (shared with Iceberg
 > > community [2]).
 > > >>
 > > >> > >> > > It
 > > >>
 > > >> > >> > > > can address small files problems due to skewed data
 > > distribution across
 > > >>
 > > >> > >> > > > Iceberg table partitions. Streaming shuffling before writers
 > > (to files)
 > > >>
 > > >> > >> > > is
 > > >>
 > > >> > >> > > > typically more efficient than post-write file compaction
 > > (which involves
 > > >>
 > > >> > >> > > > read-merge-write). It is usually cheaper to prevent a
 > > problem (small
 > > >>
 > > >> > >> > > files)
 > > >>
 > > >> > >> > > > than fixing it.
 > > >>
 > > >> > >> > >
 > > >>
 > > >> > >> > >
 > > >>
 > > >> > >> > > @steven you are raising a good point, although I think only
 > > using a
 > > >>
 > > >> > >> > > customizable
 > > >>
 > > >> > >> > > shuffle won't address the generation of small files. One
 > > assumption is that
 > > >>
 > > >> > >> > > at least the sink generates one file per subtask, which can
 > > already be too
 > > >>
 > > >> > >> > > many.
 > > >>
 > > >> > >> > > Another problem is that with low checkpointing intervals, the
 > > files do not
 > > >>
 > > >> > >> > > meet
 > > >>
 > > >> > >> > > the required size. The latter point is probably addressable by
 > > changing the
 > > >>
 > > >> > >> > > checkpoint interval, which might be inconvenient for some
 > > users.
 > > >>
 > > >> > >> > >
 > > >>
 > > >> > >> > > > The sink coordinator checkpoint problem (mentioned in option
 > > 1) would be
 > > >>
 > > >> > >> > > > great if Flink can address it. In the spirit of source
 > > >>
 > > >> > >> > > (enumerator-reader)
 > > >>
 > > >> > >> > > > and sink (writer-coordinator) duality, sink coordinator
 > > checkpoint should
 > > >>
 > > >> > >> > > > happen after the writer operator. This would be a natural
 > > fit to support
 > > >>
 > > >> > >> > > > global committer in FLIP-143. It is probably an orthogonal
 > > matter to this
 > > >>
 > > >> > >> > > > proposal.
 > > >>
 > > >> > >> > >
 > > >>
 > > >> > >> > >
 > > >>
 > > >> > >> > > To me the question here is what are the benefits of having a
 > > coordinator in
 > > >>
 > > >> > >> > > comparison to a global committer/aggregator operator.
 > > >>
 > > >> > >> > >
 > > >>
 > > >> > >> > > > Personally, I am usually in favor of keeping streaming
 > > ingestion (to data
 > > >>
 > > >> > >> > > > lake) relatively simple and stable. Also sometimes
 > > compaction and sorting
 > > >>
 > > >> > >> > > > are performed together in data rewrite maintenance jobs to
 > > improve read
 > > >>
 > > >> > >> > > > performance. In that case, the value of compacting (in Flink
 > > streaming
 > > >>
 > > >> > >> > > > ingestion) diminishes.
 > > >>
 > > >> > >> > >
 > > >>
 > > >> > >> > >
 > > >>
 > > >> > >> > > I agree it is always possible to have scheduled maintenance
 > > jobs keeping
 > > >>
 > > >> > >> > > care of
 > > >>
 > > >> > >> > > your data i.e., doing compaction. Unfortunately, the downside
 > > is that you
 > > >>
 > > >> > >> > > have to your data after it is already available for other
 > > downstream
 > > >>
 > > >> > >> > > consumers.
 > > >>
 > > >> > >> > > I guess this can lead to all kinds of visibility problems. I
 > > am also
 > > >>
 > > >> > >> > > surprised that
 > > >>
 > > >> > >> > > you personally are a fan of this approach and, on the other
 > > hand, are
 > > >>
 > > >> > >> > > developing
 > > >>
 > > >> > >> > > the Iceberg sink, which goes somewhat against your mentioned
 > > principle of
 > > >>
 > > >> > >> > > keeping
 > > >>
 > > >> > >> > > the sink simple.
 > > >>
 > > >> > >> > >
 > > >>
 > > >> > >> > > > Currently, it is unclear from the doc and this thread where
 > > the
 > > >>
 > > >> > >> > > compaction
 > > >>
 > > >> > >> > > > is actually happening. Jingsong's reply described one model
 > > >>
 > > >> > >> > > > writer (parallel) -> aggregator (single-parallelism
 > > compaction planner)
 > > >>
 > > >> > >> > > ->
 > > >>
 > > >> > >> > > > compactor (parallel) -> global committer (single-parallelism)
 > > >>
 > > >> > >> > >
 > > >>
 > > >> > >> > >
 > > >>
 > > >> > >> > > My idea of the topology is very similar to the one outlined by
 > > Jinsong. The
 > > >>
 > > >> > >> > > compaction will happen in the committer operator.
 > > >>
 > > >> > >> > >
 > > >>
 > > >> > >> > > >
 > > >>
 > > >> > >> > > > In the Iceberg community, the following model has been
 > > discussed. It is
 > > >>
 > > >> > >> > > > better for Iceberg because it won't delay the data
 > > availability.
 > > >>
 > > >> > >> > > > writer (parallel) -> global committer for append (single
 > > parallelism) ->
 > > >>
 > > >> > >> > > > compactor (parallel) -> global committer for rewrite commit
 > > (single
 > > >>
 > > >> > >> > > > parallelism)
 > > >>
 > > >> > >> > >
 > > >>
 > > >> > >> > >
 > > >>
 > > >> > >> > > From a quick glimpse, it seems that the exact same topology is
 > > possible to
 > > >>
 > > >> > >> > > express with the committable aggregator, but this definitely
 > > depends on
 > > >>
 > > >> > >> > > the exact
 > > >>
 > > >> > >> > > setup.
 > > >>
 > > >> > >> > >
 > > >>
 > > >> > >> > > Best,
 > > >>
 > > >> > >> > > Fabian
 > > >>
 > > >> > >>
 > >


Re: Re: [DISCUSS] FLIP-191: Extend unified Sink interface to support small file compaction

Posted by Arvid Heise <ar...@apache.org>.
Just a quick amend: There will be no blocking exchange in the pre-writer
exchange for performance reasons.
After the writer, we have tiny data volume and are free to add as many as
we see necessary.

On Thu, Dec 16, 2021 at 11:18 AM Fabian Paul <fp...@apache.org> wrote:

> Hi Till,
>
> Good point, in the scenario with the blocking keyed exchange between
> the writer and committer my idea is to make the committer effectively
> the global committer. With Sink V2 there is no real difference anymore
> between the committer and global committer.
> You are right that everything after the committer would be part of the
> same failover region but we plan to insert a blocking exchange by
> default before all of the custom topologies.
>
> Best,
> Fabian
>
> On Thu, Dec 16, 2021 at 11:08 AM Till Rohrmann <tr...@apache.org>
> wrote:
> >
> > Hi Fabian,
> >
> > quick question on your comment 3. If there is a pipelined data exchange
> > with a keyBy between the writers/committers and the component that does
> the
> > global commit, then there will only be a single failover region. So is it
> > correct that you assumed blocking data exchanges for the scenario you
> > described?
> >
> > Cheers,
> > Till
> >
> > On Thu, Dec 16, 2021 at 9:23 AM Fabian Paul <fp...@apache.org> wrote:
> >
> > > Hi Yun,
> > >
> > > Thanks for your fast feedback. Let me clarify your points.
> > >
> > > 1. We solve it by using StreamExchangeMode.BATCH before any exchange.
> > > That obviously doesn’t help with lost TM but we would need to employ
> > > HA storage for that. Same issue as now and orthogonal.
> > >
> > > 2. Extending V1 with V2 or vice versa would require renames of methods
> > > (since return types are non-optional) and is making API changes. Even
> > > though Experimental, we want to give connector developers the
> > > opportunity to provide 1 implementation for all of Flink 1.X. We will
> > > offer an internal adapter from V1 to V2, 2 sinkTo , and internally
> > > just have one code-path.
> > >
> > > 3. DataStreamSink would act as a unified view on all the operators and
> > > update them all at once when using setParallelism and so on (setName
> > > and setUid will receive suffixes per operator).
> > > Iceberg actually has a different requirement: They want to have a
> > > committer with parallelism 1 but as a coordinator such that
> > > embarrassingly parallel pipelines have different fail-over regions. I
> > > was thinking that in this case, they need to implement a no-op
> > > committer (that just forwards the committables) and use a post-commit
> > > topology that achieves that.
> > > Another option is that they use the preCommit topology and insert a
> > > constant key-by that forwards all committables to a single committer.
> > > We are planning to provide building blocks for such pipelines as we
> > > go.
> > >
> > > Best,
> > > Fabian
> > >
> > > On Thu, Dec 16, 2021 at 5:50 AM Yun Gao <yu...@aliyun.com> wrote:
> > > >
> > > > Hi Fabian,
> > > >
> > > > Very thanks for the update! I think the latest version in general
> looks
> > > good from my side
> > > > and I think using separate feature interface would be much more easy
> to
> > > understand
> > > > and extend in the future. I have some pending issues on the details
> > > though:
> > > >
> > > > 1. The first one is if we could support end-to-end exactly-once with
> > > post-committing
> > > > topology in the batch mode ? Since for the batch mode, currently we
> > > could only commit
> > > >  all the transactions after the whole job is finished, otherwise if
> > > there are JM failover or the
> > > > writer / committer get restarted due to indeterminstic (A -> [B1,
> B2],
> > > A, B1 have finished and
> > > >  B2 failed, if -> is rebalance / random / rescale,  all of A, B1, B2
> > > would restarted), there might
> > > > be repeat records. Previously one possible thought is to move
> committer
> > > and global committer
> > > >  to the operator coordinator, but if it is a topology, we might need
> > > some other kind of solutions?
> > > >
> > > > 2. I also want to have a dobule confirmation with the compatibility:
> > > since the old sink is also named
> > > > with Sink, do we want to put the Sink v2 in a new package ? Besides,
> > > since we might want to keep
> > > > only have one `sinkTo(Sink<?> sink)` , perhaps we also need to make
> the
> > > Sink v1 to be a subclass of
> > > > Sink v2 and extends the stateful and two-phase-commit sinks, right?
> > > >
> > > > 3. I'd like also have a confirmation on ours thoughts with the
> > > `DataStreamSink` returned by the sinkTo method:
> > > > The main issue is how do we implement the method like
> `setParallelism`
> > > or `setMaxParallelism` since now the sink
> > > > would be translated to multiple transformations? perhaps we could
> make
> > > it the default values for all the transformations
> > > > for the sink? A related issue would be for iceberg sink, I think it
> > > would need to have only one committer to avoid the
> > > > competition of the optimistic locks (which would cause performance
> > > degradation), then it might need to have N writers
> > > > with 1 committers, to build such topology, perhaps we might need to
> add
> > > new methods to specify the parallelism of
> > > > the writers and committers separately?
> > > >
> > > > Best,
> > > > Yun
> > > >
> > > >
> > > > ------------------Original Mail ------------------
> > > > Sender:Fabian Paul <fp...@apache.org>
> > > > Send Date:Mon Dec 13 23:59:43 2021
> > > > Recipients:dev <de...@flink.apache.org>
> > > > Subject:Re: [DISCUSS] FLIP-191: Extend unified Sink interface to
> support
> > > small file compaction
> > > >>
> > > >> Hi all,
> > > >>
> > > >>
> > > >>
> > > >> After a lot of discussions with different, we received very fruitful
> > > >>
> > > >> feedback and reworked the ideas behind this FLIP. Initially, we had
> > > >>
> > > >> the impression that the compaction problem is solvable by a single
> > > >>
> > > >> topology that we can reuse across different sinks. We now have a
> > > >>
> > > >> better understanding that different external systems require
> different
> > > >>
> > > >> compaction mechanism i.e. Hive requires compaction before finally
> > > >>
> > > >> registering the file in the metastore or Iceberg compacts the files
> > > >>
> > > >> after they have been registered and just lazily compacts them.
> > > >>
> > > >>
> > > >>
> > > >> Considering all these different views we came up with a design that
> > > >>
> > > >> builds upon what @guowei.mgw@gmail.com and @yungao.gy@aliyun.com
> have
> > > >>
> > > >> proposed at the beginning. We allow inserting custom topologies
> before
> > > >>
> > > >> and after the SinkWriters and Committers. Furthermore, we do not see
> > > >>
> > > >> it as a downside. The Sink interfaces that will expose the
> DataStream
> > > >>
> > > >> to the user reside in flink-streaming-java in contrast to the basic
> > > >>
> > > >> Sink interfaces that reside fin flink-core deem it to be only used
> by
> > > >>
> > > >> expert users.
> > > >>
> > > >>
> > > >>
> > > >> Moreover, we also wanted to remove the global committer from the
> > > >>
> > > >> unified Sink interfaces and replace it with a custom post-commit
> > > >>
> > > >> topology. Unfortunately, we cannot do it without breaking the Sink
> > > >>
> > > >> interface since the GlobalCommittables are part of the parameterized
> > > >>
> > > >> Sink interface. Thus, we propose building a new Sink V2 interface
> > > >>
> > > >> consisting of composable interfaces that do not offer the
> > > >>
> > > >> GlobalCommitter anymore. We will implement a utility to extend a
> Sink
> > > >>
> > > >> with post topology that mimics the behavior of the GlobalCommitter.
> > > >>
> > > >> The new Sink V2 provides the same sort of methods as the Sink V1
> > > >>
> > > >> interface, so a migration of sinks that do not use the
> GlobalCommitter
> > > >>
> > > >> should be very easy.
> > > >>
> > > >> We plan to keep the existing Sink V1 interfaces to not break
> > > >>
> > > >> externally built sinks. As part of this FLIP, we migrate all the
> > > >>
> > > >> connectors inside of the main repository to the new Sink V2 API.
> > > >>
> > > >>
> > > >>
> > > >> The FLIP document is also updated and includes the proposed changes.
> > > >>
> > > >>
> > > >>
> > > >> Looking forward to your feedback,
> > > >>
> > > >> Fabian
> > > >>
> > > >>
> > > >>
> > > >>
> > >
> https://cwiki.apache.org/confluence/display/FLINK/FLIP-191%3A+Extend+unified+Sink+interface+to+support+small+file+compaction
> > > >>
> > > >>
> > > >>
> > > >>
> > > >>
> > > >> On Thu, Dec 2, 2021 at 10:15 AM Roman Khachatryan wrote:
> > > >>
> > > >> >
> > > >>
> > > >> > Thanks for clarifying (I was initially confused by merging state
> files
> > > >>
> > > >> > rather than output files).
> > > >>
> > > >> >
> > > >>
> > > >> > > At some point, Flink will definitely have some WAL adapter that
> can
> > > turn any sink into an exactly-once sink (with some caveats). For now,
> we
> > > keep that as an orthogonal solution as it has a rather high price
> (bursty
> > > workload with high latency). Ideally, we can keep the compaction
> > > asynchronously...
> > > >>
> > > >> >
> > > >>
> > > >> > Yes, that would be something like a WAL. I agree that it would
> have a
> > > >>
> > > >> > different set of trade-offs.
> > > >>
> > > >> >
> > > >>
> > > >> >
> > > >>
> > > >> > Regards,
> > > >>
> > > >> > Roman
> > > >>
> > > >> >
> > > >>
> > > >> > On Mon, Nov 29, 2021 at 3:33 PM Arvid Heise wrote:
> > > >>
> > > >> > >>
> > > >>
> > > >> > >> > One way to avoid write-read-merge is by wrapping SinkWriter
> with
> > > >>
> > > >> > >> > another one, which would buffer input elements in a temporary
> > > storage
> > > >>
> > > >> > >> > (e.g. local file) until a threshold is reached; after that,
> it
> > > would
> > > >>
> > > >> > >> > invoke the original SinkWriter. And if a checkpoint barrier
> > > comes in
> > > >>
> > > >> > >> > earlier, it would send written data to some aggregator.
> > > >>
> > > >> > >>
> > > >>
> > > >> > >> I think perhaps this seems to be a kind of WAL method? Namely
> we
> > > first
> > > >>
> > > >> > >> write the elements to some WAL logs and persist them on
> checkpoint
> > > >>
> > > >> > >> (in snapshot or remote FS), or we directly write WAL logs to
> the
> > > remote
> > > >>
> > > >> > >> FS eagerly.
> > > >>
> > > >> > >>
> > > >>
> > > >> > > At some point, Flink will definitely have some WAL adapter that
> can
> > > turn any sink into an exactly-once sink (with some caveats). For now,
> we
> > > keep that as an orthogonal solution as it has a rather high price
> (bursty
> > > workload with high latency). Ideally, we can keep the compaction
> > > asynchronously...
> > > >>
> > > >> > >
> > > >>
> > > >> > > On Mon, Nov 29, 2021 at 8:52 AM Yun Gao wrote:
> > > >>
> > > >> > >>
> > > >>
> > > >> > >> Hi,
> > > >>
> > > >> > >>
> > > >>
> > > >> > >> @Roman very sorry for the late response for a long time,
> > > >>
> > > >> > >>
> > > >>
> > > >> > >> > Merging artifacts from multiple checkpoints would apparently
> > > >>
> > > >> > >> require multiple concurrent checkpoints
> > > >>
> > > >> > >>
> > > >>
> > > >> > >> I think it might not need concurrent checkpoints: suppose some
> > > >>
> > > >> > >> operators (like the committer aggregator in the option 2)
> maintains
> > > >>
> > > >> > >> the list of files to merge, it could stores the lists of files
> to
> > > merge
> > > >>
> > > >> > >> in the states, then after several checkpoints are done and we
> have
> > > >>
> > > >> > >> enough files, we could merge all the files in the list.
> > > >>
> > > >> > >>
> > > >>
> > > >> > >> > Asynchronous merging in an aggregator would require some
> > > resolution
> > > >>
> > > >> > >> > logic on recovery, so that a merged artifact can be used if
> the
> > > >>
> > > >> > >> > original one was deleted. Otherwise, wouldn't recovery fail
> > > because
> > > >>
> > > >> > >> > some artifacts are missing?
> > > >>
> > > >> > >> > We could also defer deletion until the "compacted"
> checkpoint is
> > > >>
> > > >> > >> > subsumed - but isn't it too late, as it will be deleted
> anyways
> > > once
> > > >>
> > > >> > >> > subsumed?
> > > >>
> > > >> > >>
> > > >>
> > > >> > >> I think logically we could delete the original files once the
> > > "compacted" checkpoint
> > > >>
> > > >> > >> (which finish merging the compacted files and record it in the
> > > checkpoint) is completed
> > > >>
> > > >> > >> in all the options. If there are failover before we it, we
> could
> > > restart the merging and if
> > > >>
> > > >> > >> there are failover after it, we could have already recorded the
> > > files in the checkpoint.
> > > >>
> > > >> > >>
> > > >>
> > > >> > >> > One way to avoid write-read-merge is by wrapping SinkWriter
> with
> > > >>
> > > >> > >> > another one, which would buffer input elements in a temporary
> > > storage
> > > >>
> > > >> > >> > (e.g. local file) until a threshold is reached; after that,
> it
> > > would
> > > >>
> > > >> > >> > invoke the original SinkWriter. And if a checkpoint barrier
> > > comes in
> > > >>
> > > >> > >> > earlier, it would send written data to some aggregator.
> > > >>
> > > >> > >>
> > > >>
> > > >> > >> I think perhaps this seems to be a kind of WAL method? Namely
> we
> > > first
> > > >>
> > > >> > >> write the elements to some WAL logs and persist them on
> checkpoint
> > > >>
> > > >> > >> (in snapshot or remote FS), or we directly write WAL logs to
> the
> > > remote
> > > >>
> > > >> > >> FS eagerly.
> > > >>
> > > >> > >>
> > > >>
> > > >> > >> Sorry if I do not understand correctly somewhere.
> > > >>
> > > >> > >>
> > > >>
> > > >> > >> Best,
> > > >>
> > > >> > >> Yun
> > > >>
> > > >> > >>
> > > >>
> > > >> > >>
> > > >>
> > > >> > >>
> ------------------------------------------------------------------
> > > >>
> > > >> > >> From:Roman Khachatryan
> > > >>
> > > >> > >> Send Time:2021 Nov. 9 (Tue.) 22:03
> > > >>
> > > >> > >> To:dev
> > > >>
> > > >> > >> Subject:Re: [DISCUSS] FLIP-191: Extend unified Sink interface
> to
> > > support small file compaction
> > > >>
> > > >> > >>
> > > >>
> > > >> > >> Hi everyone,
> > > >>
> > > >> > >>
> > > >>
> > > >> > >> Thanks for the proposal and the discussion, I have some
> remarks:
> > > >>
> > > >> > >> (I'm not very familiar with the new Sink API but I thought
> about
> > > the
> > > >>
> > > >> > >> same problem in context of the changelog state backend)
> > > >>
> > > >> > >>
> > > >>
> > > >> > >> 1. Merging artifacts from multiple checkpoints would apparently
> > > >>
> > > >> > >> require multiple concurrent checkpoints (otherwise, a new
> > > checkpoint
> > > >>
> > > >> > >> won't be started before completing the previous one; and the
> > > previous
> > > >>
> > > >> > >> one can't be completed before durably storing the artifacts).
> > > However,
> > > >>
> > > >> > >> concurrent checkpoints are currently not supported with
> Unaligned
> > > >>
> > > >> > >> checkpoints (this is besides increasing e2e-latency).
> > > >>
> > > >> > >>
> > > >>
> > > >> > >> 2. Asynchronous merging in an aggregator would require some
> > > resolution
> > > >>
> > > >> > >> logic on recovery, so that a merged artifact can be used if the
> > > >>
> > > >> > >> original one was deleted. Otherwise, wouldn't recovery fail
> because
> > > >>
> > > >> > >> some artifacts are missing?
> > > >>
> > > >> > >> We could also defer deletion until the "compacted" checkpoint
> is
> > > >>
> > > >> > >> subsumed - but isn't it too late, as it will be deleted anyways
> > > once
> > > >>
> > > >> > >> subsumed?
> > > >>
> > > >> > >>
> > > >>
> > > >> > >> 3. Writing small files, then reading and merging them for
> *every*
> > > >>
> > > >> > >> checkpoint seems worse than only reading them on recovery. I
> guess
> > > I'm
> > > >>
> > > >> > >> missing some cases of reading, so to me it would make sense to
> > > mention
> > > >>
> > > >> > >> these cases explicitly in the FLIP motivation section.
> > > >>
> > > >> > >>
> > > >>
> > > >> > >> 4. One way to avoid write-read-merge is by wrapping SinkWriter
> with
> > > >>
> > > >> > >> another one, which would buffer input elements in a temporary
> > > storage
> > > >>
> > > >> > >> (e.g. local file) until a threshold is reached; after that, it
> > > would
> > > >>
> > > >> > >> invoke the original SinkWriter. And if a checkpoint barrier
> comes
> > > in
> > > >>
> > > >> > >> earlier, it would send written data to some aggregator. It will
> > > >>
> > > >> > >> increase checkpoint delay (async phase) compared to the current
> > > Flink;
> > > >>
> > > >> > >> but not compared to the write-read-merge solution, IIUC.
> > > >>
> > > >> > >> Then such "BufferingSinkWriters" could aggregate input elements
> > > from
> > > >>
> > > >> > >> each other, potentially recursively (I mean something like
> > > >>
> > > >> > >>
> > >
> https://cwiki.apache.org/confluence/download/attachments/173082889/DSTL-DFS-DAG.png
> > > >>
> > > >> > >> )
> > > >>
> > > >> > >>
> > > >>
> > > >> > >> 5. Reducing the number of files by reducing aggregator
> parallelism
> > > as
> > > >>
> > > >> > >> opposed to merging on reaching size threshold will likely be
> less
> > > >>
> > > >> > >> optimal and more difficult to configure. OTH, thresholds might
> be
> > > more
> > > >>
> > > >> > >> difficult to implement and (with recursive merging) would incur
> > > higher
> > > >>
> > > >> > >> latency. Maybe that's also something to decide explicitly or at
> > > least
> > > >>
> > > >> > >> mention in the FLIP.
> > > >>
> > > >> > >>
> > > >>
> > > >> > >>
> > > >>
> > > >> > >>
> > > >>
> > > >> > >> Regards,
> > > >>
> > > >> > >> Roman
> > > >>
> > > >> > >>
> > > >>
> > > >> > >>
> > > >>
> > > >> > >> On Tue, Nov 9, 2021 at 5:23 AM Reo Lei wrote:
> > > >>
> > > >> > >> >
> > > >>
> > > >> > >> > Hi Fabian,
> > > >>
> > > >> > >> >
> > > >>
> > > >> > >> > Thanks for drafting the FLIP and trying to support small file
> > > compaction. I
> > > >>
> > > >> > >> > think this feature is very urgent and valuable for users(at
> > > least for me).
> > > >>
> > > >> > >> >
> > > >>
> > > >> > >> > Currently I am trying to support streaming rewrite(compact)
> for
> > > Iceberg on
> > > >>
> > > >> > >> > PR#3323 . As Steven mentioned,
> > > >>
> > > >> > >> > Iceberg sink and compact data through the following steps:
> > > >>
> > > >> > >> > Step-1: Some parallel data writer(sinker) to write streaming
> > > data as files.
> > > >>
> > > >> > >> > Step-2: A single parallelism data files committer to commit
> the
> > > completed
> > > >>
> > > >> > >> > files as soon as possible to make them available.
> > > >>
> > > >> > >> > Step-3: Some parallel file rewriter(compactor) to collect
> > > committed files
> > > >>
> > > >> > >> > from multiple checkpoints, and rewriter(compact) them
> together
> > > once the
> > > >>
> > > >> > >> > total file size or number of files reach the threshold.
> > > >>
> > > >> > >> > Step-4: A single parallelism rewrite(compact) result
> committer
> > > to commit
> > > >>
> > > >> > >> > the rewritten(compacted) files to replace the old files and
> make
> > > them
> > > >>
> > > >> > >> > available.
> > > >>
> > > >> > >> >
> > > >>
> > > >> > >> >
> > > >>
> > > >> > >> > If Flink want to support small file compaction, some key
> point I
> > > think is
> > > >>
> > > >> > >> > necessary:
> > > >>
> > > >> > >> >
> > > >>
> > > >> > >> > 1, Compact files from multiple checkpoints.
> > > >>
> > > >> > >> > I totally agree with Jingsong, because completed file size
> > > usually could
> > > >>
> > > >> > >> > not reach the threshold in a single checkpoint. Especially
> for
> > > partitioned
> > > >>
> > > >> > >> > table, we need to compact the files of each partition, but
> > > usually the file
> > > >>
> > > >> > >> > size of each partition will be different and may not reach
> the
> > > merge
> > > >>
> > > >> > >> > threshold. If we compact these files, in a single checkpoint,
> > > regardless of
> > > >>
> > > >> > >> > whether the total file size reaches the threshold, then the
> > > value of
> > > >>
> > > >> > >> > compacting will be diminished and we will still get small
> files
> > > because
> > > >>
> > > >> > >> > these compacted files are not reach to target size. So we
> need
> > > the
> > > >>
> > > >> > >> > compactor to collect committed files from multiple
> checkpoints
> > > and compact
> > > >>
> > > >> > >> > them until they reach the threshold.
> > > >>
> > > >> > >> >
> > > >>
> > > >> > >> > 2, Separate write phase and compact phase.
> > > >>
> > > >> > >> > Users usually hope the data becomes available as soon as
> > > possible, and the
> > > >>
> > > >> > >> > end-to-end latency is very important. I think we need to
> > > separate the
> > > >>
> > > >> > >> > write and compact phase. For the write phase, there include
> the
> > > Step-1
> > > >>
> > > >> > >> > and Step-2, we sink data as file and commit it pre checkpoint
> > > and regardless
> > > >>
> > > >> > >> > of whether the file size it is. That could ensure the data
> will
> > > be
> > > >>
> > > >> > >> > available ASAP. For the compact phase, there include the
> Step-3
> > > >>
> > > >> > >> > and Step-4, the compactor should collect committed files from
> > > multiple
> > > >>
> > > >> > >> > checkpoints and compact them asynchronously once they reach
> the
> > > threshold,
> > > >>
> > > >> > >> > and the compact committer will commit the compaction result
> in
> > > the next
> > > >>
> > > >> > >> > checkpoint. We compact the committed files asynchronously
> > > because we don't
> > > >>
> > > >> > >> > want the compaction to affect the data sink or the whole
> > > pipeline.
> > > >>
> > > >> > >> >
> > > >>
> > > >> > >> > 3, Exactly once guarantee between write and compact phase.
> > > >>
> > > >> > >> > Once we separate write phase and compact phase, we need to
> > > consider
> > > >>
> > > >> > >> > how to guarantee
> > > >>
> > > >> > >> > the exact once semantic between two phases. We should not
> lose
> > > any data or
> > > >>
> > > >> > >> > files on the compactor(Step-3) in any case and cause the
> > > compaction result
> > > >>
> > > >> > >> > to be inconsistent with before. I think flink should provide
> an
> > > easy-to-use
> > > >>
> > > >> > >> > interface to make that easier.
> > > >>
> > > >> > >> >
> > > >>
> > > >> > >> > 4, Metadata operation and compaction result validation.
> > > >>
> > > >> > >> > In the compact phase, there may be not only compact files,
> but
> > > also a lot
> > > >>
> > > >> > >> > of metadata operations, such as the iceberg needing to
> > > read/write manifest
> > > >>
> > > >> > >> > and do MOR. And we need some interface to support users to do
> > > some
> > > >>
> > > >> > >> > validation of the compaction result. I think these points
> should
> > > be
> > > >>
> > > >> > >> > considered when we design the compaction API.
> > > >>
> > > >> > >> >
> > > >>
> > > >> > >> >
> > > >>
> > > >> > >> > Back to FLIP-191, option 1 looks very complicated while
> option 2
> > > is
> > > >>
> > > >> > >> > relatively simple, but neither of these two solutions
> separates
> > > the write
> > > >>
> > > >> > >> > phase from the compact phase. So I think we should consider
> the
> > > points I
> > > >>
> > > >> > >> > mentioned above. And if you have any other questions you can
> > > always feel
> > > >>
> > > >> > >> > free to reach out to me!
> > > >>
> > > >> > >> >
> > > >>
> > > >> > >> > BR,
> > > >>
> > > >> > >> > Reo
> > > >>
> > > >> > >> >
> > > >>
> > > >> > >> > Fabian Paul 于2021年11月8日周一 下午7:59写道:
> > > >>
> > > >> > >> >
> > > >>
> > > >> > >> > > Hi all,
> > > >>
> > > >> > >> > >
> > > >>
> > > >> > >> > > Thanks for the lively discussions. I am really excited to
> see
> > > so many
> > > >>
> > > >> > >> > > people
> > > >>
> > > >> > >> > > participating in this thread. It also underlines the need
> that
> > > many people
> > > >>
> > > >> > >> > > would
> > > >>
> > > >> > >> > > like to see a solution soon.
> > > >>
> > > >> > >> > >
> > > >>
> > > >> > >> > > I have updated the FLIP and removed the parallelism
> > > configuration because
> > > >>
> > > >> > >> > > it is
> > > >>
> > > >> > >> > > unnecessary since users can configure a constant exchange
> key
> > > to send all
> > > >>
> > > >> > >> > > committables to only one committable aggregator.
> > > >>
> > > >> > >> > >
> > > >>
> > > >> > >> > >
> > > >>
> > > >> > >> > > 1. Burden for developers w.r.t batch stream unification.
> > > >>
> > > >> > >> > >
> > > >>
> > > >> > >> > > @yun @guowei, from a theoretical point you are right about
> > > exposing the
> > > >>
> > > >> > >> > > DataStream
> > > >>
> > > >> > >> > > API in the sink users have the full power to write correct
> > > batch and
> > > >>
> > > >> > >> > > streaming
> > > >>
> > > >> > >> > > sinks. I think in reality a lot of users still struggle to
> > > build pipelines
> > > >>
> > > >> > >> > > with
> > > >>
> > > >> > >> > > i.e. the operator pipeline which works correct in streaming
> > > and batch mode.
> > > >>
> > > >> > >> > > Another problem I see is by exposing more deeper concepts
> is
> > > that we
> > > >>
> > > >> > >> > > cannot do
> > > >>
> > > >> > >> > > any optimization because we cannot reason about how sinks
> are
> > > built in the
> > > >>
> > > >> > >> > > future.
> > > >>
> > > >> > >> > >
> > > >>
> > > >> > >> > > We should also try to steer users towards using only
> > > `Functions` to give
> > > >>
> > > >> > >> > > us more
> > > >>
> > > >> > >> > > flexibility to swap the internal operator representation. I
> > > agree with
> > > >>
> > > >> > >> > > @yun we
> > > >>
> > > >> > >> > > should try to make the `ProcessFunction` more versatile to
> > > work on that
> > > >>
> > > >> > >> > > goal but
> > > >>
> > > >> > >> > > I see this as unrelated to the FLIP.
> > > >>
> > > >> > >> > >
> > > >>
> > > >> > >> > >
> > > >>
> > > >> > >> > > 2. Regarding Commit / Global commit
> > > >>
> > > >> > >> > >
> > > >>
> > > >> > >> > > I envision the global committer to be specific depending on
> > > the data lake
> > > >>
> > > >> > >> > > solution you want to write to. However, it is entirely
> > > orthogonal to the
> > > >>
> > > >> > >> > > compaction.
> > > >>
> > > >> > >> > > Currently, I do not expect any changes w.r.t the Global
> commit
> > > introduces
> > > >>
> > > >> > >> > > by
> > > >>
> > > >> > >> > > this FLIP.
> > > >>
> > > >> > >> > >
> > > >>
> > > >> > >> > >
> > > >>
> > > >> > >> > > 3. Regarding the case of trans-checkpoints merging
> > > >>
> > > >> > >> > >
> > > >>
> > > >> > >> > > @yun, as user, I would expect that if the committer
> receives
> > > in a
> > > >>
> > > >> > >> > > checkpoint files
> > > >>
> > > >> > >> > > to merge/commit that these are also finished when the
> > > checkpoint finishes.
> > > >>
> > > >> > >> > > I think all sinks rely on this principle currently i.e.,
> > > KafkaSink needs to
> > > >>
> > > >> > >> > > commit all open transactions until the next checkpoint can
> > > happen.
> > > >>
> > > >> > >> > >
> > > >>
> > > >> > >> > > Maybe in the future, we can somehow move the
> Committer#commit
> > > call to an
> > > >>
> > > >> > >> > > asynchronous execution, but we should discuss it as a
> separate
> > > thread.
> > > >>
> > > >> > >> > >
> > > >>
> > > >> > >> > > > We probably should first describe the different causes of
> > > small files and
> > > >>
> > > >> > >> > > > what problems was this proposal trying to solve. I wrote
> a
> > > data shuffling
> > > >>
> > > >> > >> > > > proposal [1] for Flink Iceberg sink (shared with Iceberg
> > > community [2]).
> > > >>
> > > >> > >> > > It
> > > >>
> > > >> > >> > > > can address small files problems due to skewed data
> > > distribution across
> > > >>
> > > >> > >> > > > Iceberg table partitions. Streaming shuffling before
> writers
> > > (to files)
> > > >>
> > > >> > >> > > is
> > > >>
> > > >> > >> > > > typically more efficient than post-write file compaction
> > > (which involves
> > > >>
> > > >> > >> > > > read-merge-write). It is usually cheaper to prevent a
> > > problem (small
> > > >>
> > > >> > >> > > files)
> > > >>
> > > >> > >> > > > than fixing it.
> > > >>
> > > >> > >> > >
> > > >>
> > > >> > >> > >
> > > >>
> > > >> > >> > > @steven you are raising a good point, although I think only
> > > using a
> > > >>
> > > >> > >> > > customizable
> > > >>
> > > >> > >> > > shuffle won't address the generation of small files. One
> > > assumption is that
> > > >>
> > > >> > >> > > at least the sink generates one file per subtask, which can
> > > already be too
> > > >>
> > > >> > >> > > many.
> > > >>
> > > >> > >> > > Another problem is that with low checkpointing intervals,
> the
> > > files do not
> > > >>
> > > >> > >> > > meet
> > > >>
> > > >> > >> > > the required size. The latter point is probably
> addressable by
> > > changing the
> > > >>
> > > >> > >> > > checkpoint interval, which might be inconvenient for some
> > > users.
> > > >>
> > > >> > >> > >
> > > >>
> > > >> > >> > > > The sink coordinator checkpoint problem (mentioned in
> option
> > > 1) would be
> > > >>
> > > >> > >> > > > great if Flink can address it. In the spirit of source
> > > >>
> > > >> > >> > > (enumerator-reader)
> > > >>
> > > >> > >> > > > and sink (writer-coordinator) duality, sink coordinator
> > > checkpoint should
> > > >>
> > > >> > >> > > > happen after the writer operator. This would be a natural
> > > fit to support
> > > >>
> > > >> > >> > > > global committer in FLIP-143. It is probably an
> orthogonal
> > > matter to this
> > > >>
> > > >> > >> > > > proposal.
> > > >>
> > > >> > >> > >
> > > >>
> > > >> > >> > >
> > > >>
> > > >> > >> > > To me the question here is what are the benefits of having
> a
> > > coordinator in
> > > >>
> > > >> > >> > > comparison to a global committer/aggregator operator.
> > > >>
> > > >> > >> > >
> > > >>
> > > >> > >> > > > Personally, I am usually in favor of keeping streaming
> > > ingestion (to data
> > > >>
> > > >> > >> > > > lake) relatively simple and stable. Also sometimes
> > > compaction and sorting
> > > >>
> > > >> > >> > > > are performed together in data rewrite maintenance jobs
> to
> > > improve read
> > > >>
> > > >> > >> > > > performance. In that case, the value of compacting (in
> Flink
> > > streaming
> > > >>
> > > >> > >> > > > ingestion) diminishes.
> > > >>
> > > >> > >> > >
> > > >>
> > > >> > >> > >
> > > >>
> > > >> > >> > > I agree it is always possible to have scheduled maintenance
> > > jobs keeping
> > > >>
> > > >> > >> > > care of
> > > >>
> > > >> > >> > > your data i.e., doing compaction. Unfortunately, the
> downside
> > > is that you
> > > >>
> > > >> > >> > > have to your data after it is already available for other
> > > downstream
> > > >>
> > > >> > >> > > consumers.
> > > >>
> > > >> > >> > > I guess this can lead to all kinds of visibility problems.
> I
> > > am also
> > > >>
> > > >> > >> > > surprised that
> > > >>
> > > >> > >> > > you personally are a fan of this approach and, on the other
> > > hand, are
> > > >>
> > > >> > >> > > developing
> > > >>
> > > >> > >> > > the Iceberg sink, which goes somewhat against your
> mentioned
> > > principle of
> > > >>
> > > >> > >> > > keeping
> > > >>
> > > >> > >> > > the sink simple.
> > > >>
> > > >> > >> > >
> > > >>
> > > >> > >> > > > Currently, it is unclear from the doc and this thread
> where
> > > the
> > > >>
> > > >> > >> > > compaction
> > > >>
> > > >> > >> > > > is actually happening. Jingsong's reply described one
> model
> > > >>
> > > >> > >> > > > writer (parallel) -> aggregator (single-parallelism
> > > compaction planner)
> > > >>
> > > >> > >> > > ->
> > > >>
> > > >> > >> > > > compactor (parallel) -> global committer
> (single-parallelism)
> > > >>
> > > >> > >> > >
> > > >>
> > > >> > >> > >
> > > >>
> > > >> > >> > > My idea of the topology is very similar to the one
> outlined by
> > > Jinsong. The
> > > >>
> > > >> > >> > > compaction will happen in the committer operator.
> > > >>
> > > >> > >> > >
> > > >>
> > > >> > >> > > >
> > > >>
> > > >> > >> > > > In the Iceberg community, the following model has been
> > > discussed. It is
> > > >>
> > > >> > >> > > > better for Iceberg because it won't delay the data
> > > availability.
> > > >>
> > > >> > >> > > > writer (parallel) -> global committer for append (single
> > > parallelism) ->
> > > >>
> > > >> > >> > > > compactor (parallel) -> global committer for rewrite
> commit
> > > (single
> > > >>
> > > >> > >> > > > parallelism)
> > > >>
> > > >> > >> > >
> > > >>
> > > >> > >> > >
> > > >>
> > > >> > >> > > From a quick glimpse, it seems that the exact same
> topology is
> > > possible to
> > > >>
> > > >> > >> > > express with the committable aggregator, but this
> definitely
> > > depends on
> > > >>
> > > >> > >> > > the exact
> > > >>
> > > >> > >> > > setup.
> > > >>
> > > >> > >> > >
> > > >>
> > > >> > >> > > Best,
> > > >>
> > > >> > >> > > Fabian
> > > >>
> > > >> > >>
> > >
>

Re: Re: [DISCUSS] FLIP-191: Extend unified Sink interface to support small file compaction

Posted by Fabian Paul <fp...@apache.org>.
Hi Till,

Good point, in the scenario with the blocking keyed exchange between
the writer and committer my idea is to make the committer effectively
the global committer. With Sink V2 there is no real difference anymore
between the committer and global committer.
You are right that everything after the committer would be part of the
same failover region but we plan to insert a blocking exchange by
default before all of the custom topologies.

Best,
Fabian

On Thu, Dec 16, 2021 at 11:08 AM Till Rohrmann <tr...@apache.org> wrote:
>
> Hi Fabian,
>
> quick question on your comment 3. If there is a pipelined data exchange
> with a keyBy between the writers/committers and the component that does the
> global commit, then there will only be a single failover region. So is it
> correct that you assumed blocking data exchanges for the scenario you
> described?
>
> Cheers,
> Till
>
> On Thu, Dec 16, 2021 at 9:23 AM Fabian Paul <fp...@apache.org> wrote:
>
> > Hi Yun,
> >
> > Thanks for your fast feedback. Let me clarify your points.
> >
> > 1. We solve it by using StreamExchangeMode.BATCH before any exchange.
> > That obviously doesn’t help with lost TM but we would need to employ
> > HA storage for that. Same issue as now and orthogonal.
> >
> > 2. Extending V1 with V2 or vice versa would require renames of methods
> > (since return types are non-optional) and is making API changes. Even
> > though Experimental, we want to give connector developers the
> > opportunity to provide 1 implementation for all of Flink 1.X. We will
> > offer an internal adapter from V1 to V2, 2 sinkTo , and internally
> > just have one code-path.
> >
> > 3. DataStreamSink would act as a unified view on all the operators and
> > update them all at once when using setParallelism and so on (setName
> > and setUid will receive suffixes per operator).
> > Iceberg actually has a different requirement: They want to have a
> > committer with parallelism 1 but as a coordinator such that
> > embarrassingly parallel pipelines have different fail-over regions. I
> > was thinking that in this case, they need to implement a no-op
> > committer (that just forwards the committables) and use a post-commit
> > topology that achieves that.
> > Another option is that they use the preCommit topology and insert a
> > constant key-by that forwards all committables to a single committer.
> > We are planning to provide building blocks for such pipelines as we
> > go.
> >
> > Best,
> > Fabian
> >
> > On Thu, Dec 16, 2021 at 5:50 AM Yun Gao <yu...@aliyun.com> wrote:
> > >
> > > Hi Fabian,
> > >
> > > Very thanks for the update! I think the latest version in general looks
> > good from my side
> > > and I think using separate feature interface would be much more easy to
> > understand
> > > and extend in the future. I have some pending issues on the details
> > though:
> > >
> > > 1. The first one is if we could support end-to-end exactly-once with
> > post-committing
> > > topology in the batch mode ? Since for the batch mode, currently we
> > could only commit
> > >  all the transactions after the whole job is finished, otherwise if
> > there are JM failover or the
> > > writer / committer get restarted due to indeterminstic (A -> [B1, B2],
> > A, B1 have finished and
> > >  B2 failed, if -> is rebalance / random / rescale,  all of A, B1, B2
> > would restarted), there might
> > > be repeat records. Previously one possible thought is to move committer
> > and global committer
> > >  to the operator coordinator, but if it is a topology, we might need
> > some other kind of solutions?
> > >
> > > 2. I also want to have a dobule confirmation with the compatibility:
> > since the old sink is also named
> > > with Sink, do we want to put the Sink v2 in a new package ? Besides,
> > since we might want to keep
> > > only have one `sinkTo(Sink<?> sink)` , perhaps we also need to make the
> > Sink v1 to be a subclass of
> > > Sink v2 and extends the stateful and two-phase-commit sinks, right?
> > >
> > > 3. I'd like also have a confirmation on ours thoughts with the
> > `DataStreamSink` returned by the sinkTo method:
> > > The main issue is how do we implement the method like `setParallelism`
> > or `setMaxParallelism` since now the sink
> > > would be translated to multiple transformations? perhaps we could make
> > it the default values for all the transformations
> > > for the sink? A related issue would be for iceberg sink, I think it
> > would need to have only one committer to avoid the
> > > competition of the optimistic locks (which would cause performance
> > degradation), then it might need to have N writers
> > > with 1 committers, to build such topology, perhaps we might need to add
> > new methods to specify the parallelism of
> > > the writers and committers separately?
> > >
> > > Best,
> > > Yun
> > >
> > >
> > > ------------------Original Mail ------------------
> > > Sender:Fabian Paul <fp...@apache.org>
> > > Send Date:Mon Dec 13 23:59:43 2021
> > > Recipients:dev <de...@flink.apache.org>
> > > Subject:Re: [DISCUSS] FLIP-191: Extend unified Sink interface to support
> > small file compaction
> > >>
> > >> Hi all,
> > >>
> > >>
> > >>
> > >> After a lot of discussions with different, we received very fruitful
> > >>
> > >> feedback and reworked the ideas behind this FLIP. Initially, we had
> > >>
> > >> the impression that the compaction problem is solvable by a single
> > >>
> > >> topology that we can reuse across different sinks. We now have a
> > >>
> > >> better understanding that different external systems require different
> > >>
> > >> compaction mechanism i.e. Hive requires compaction before finally
> > >>
> > >> registering the file in the metastore or Iceberg compacts the files
> > >>
> > >> after they have been registered and just lazily compacts them.
> > >>
> > >>
> > >>
> > >> Considering all these different views we came up with a design that
> > >>
> > >> builds upon what @guowei.mgw@gmail.com and @yungao.gy@aliyun.com have
> > >>
> > >> proposed at the beginning. We allow inserting custom topologies before
> > >>
> > >> and after the SinkWriters and Committers. Furthermore, we do not see
> > >>
> > >> it as a downside. The Sink interfaces that will expose the DataStream
> > >>
> > >> to the user reside in flink-streaming-java in contrast to the basic
> > >>
> > >> Sink interfaces that reside fin flink-core deem it to be only used by
> > >>
> > >> expert users.
> > >>
> > >>
> > >>
> > >> Moreover, we also wanted to remove the global committer from the
> > >>
> > >> unified Sink interfaces and replace it with a custom post-commit
> > >>
> > >> topology. Unfortunately, we cannot do it without breaking the Sink
> > >>
> > >> interface since the GlobalCommittables are part of the parameterized
> > >>
> > >> Sink interface. Thus, we propose building a new Sink V2 interface
> > >>
> > >> consisting of composable interfaces that do not offer the
> > >>
> > >> GlobalCommitter anymore. We will implement a utility to extend a Sink
> > >>
> > >> with post topology that mimics the behavior of the GlobalCommitter.
> > >>
> > >> The new Sink V2 provides the same sort of methods as the Sink V1
> > >>
> > >> interface, so a migration of sinks that do not use the GlobalCommitter
> > >>
> > >> should be very easy.
> > >>
> > >> We plan to keep the existing Sink V1 interfaces to not break
> > >>
> > >> externally built sinks. As part of this FLIP, we migrate all the
> > >>
> > >> connectors inside of the main repository to the new Sink V2 API.
> > >>
> > >>
> > >>
> > >> The FLIP document is also updated and includes the proposed changes.
> > >>
> > >>
> > >>
> > >> Looking forward to your feedback,
> > >>
> > >> Fabian
> > >>
> > >>
> > >>
> > >>
> > https://cwiki.apache.org/confluence/display/FLINK/FLIP-191%3A+Extend+unified+Sink+interface+to+support+small+file+compaction
> > >>
> > >>
> > >>
> > >>
> > >>
> > >> On Thu, Dec 2, 2021 at 10:15 AM Roman Khachatryan wrote:
> > >>
> > >> >
> > >>
> > >> > Thanks for clarifying (I was initially confused by merging state files
> > >>
> > >> > rather than output files).
> > >>
> > >> >
> > >>
> > >> > > At some point, Flink will definitely have some WAL adapter that can
> > turn any sink into an exactly-once sink (with some caveats). For now, we
> > keep that as an orthogonal solution as it has a rather high price (bursty
> > workload with high latency). Ideally, we can keep the compaction
> > asynchronously...
> > >>
> > >> >
> > >>
> > >> > Yes, that would be something like a WAL. I agree that it would have a
> > >>
> > >> > different set of trade-offs.
> > >>
> > >> >
> > >>
> > >> >
> > >>
> > >> > Regards,
> > >>
> > >> > Roman
> > >>
> > >> >
> > >>
> > >> > On Mon, Nov 29, 2021 at 3:33 PM Arvid Heise wrote:
> > >>
> > >> > >>
> > >>
> > >> > >> > One way to avoid write-read-merge is by wrapping SinkWriter with
> > >>
> > >> > >> > another one, which would buffer input elements in a temporary
> > storage
> > >>
> > >> > >> > (e.g. local file) until a threshold is reached; after that, it
> > would
> > >>
> > >> > >> > invoke the original SinkWriter. And if a checkpoint barrier
> > comes in
> > >>
> > >> > >> > earlier, it would send written data to some aggregator.
> > >>
> > >> > >>
> > >>
> > >> > >> I think perhaps this seems to be a kind of WAL method? Namely we
> > first
> > >>
> > >> > >> write the elements to some WAL logs and persist them on checkpoint
> > >>
> > >> > >> (in snapshot or remote FS), or we directly write WAL logs to the
> > remote
> > >>
> > >> > >> FS eagerly.
> > >>
> > >> > >>
> > >>
> > >> > > At some point, Flink will definitely have some WAL adapter that can
> > turn any sink into an exactly-once sink (with some caveats). For now, we
> > keep that as an orthogonal solution as it has a rather high price (bursty
> > workload with high latency). Ideally, we can keep the compaction
> > asynchronously...
> > >>
> > >> > >
> > >>
> > >> > > On Mon, Nov 29, 2021 at 8:52 AM Yun Gao wrote:
> > >>
> > >> > >>
> > >>
> > >> > >> Hi,
> > >>
> > >> > >>
> > >>
> > >> > >> @Roman very sorry for the late response for a long time,
> > >>
> > >> > >>
> > >>
> > >> > >> > Merging artifacts from multiple checkpoints would apparently
> > >>
> > >> > >> require multiple concurrent checkpoints
> > >>
> > >> > >>
> > >>
> > >> > >> I think it might not need concurrent checkpoints: suppose some
> > >>
> > >> > >> operators (like the committer aggregator in the option 2) maintains
> > >>
> > >> > >> the list of files to merge, it could stores the lists of files to
> > merge
> > >>
> > >> > >> in the states, then after several checkpoints are done and we have
> > >>
> > >> > >> enough files, we could merge all the files in the list.
> > >>
> > >> > >>
> > >>
> > >> > >> > Asynchronous merging in an aggregator would require some
> > resolution
> > >>
> > >> > >> > logic on recovery, so that a merged artifact can be used if the
> > >>
> > >> > >> > original one was deleted. Otherwise, wouldn't recovery fail
> > because
> > >>
> > >> > >> > some artifacts are missing?
> > >>
> > >> > >> > We could also defer deletion until the "compacted" checkpoint is
> > >>
> > >> > >> > subsumed - but isn't it too late, as it will be deleted anyways
> > once
> > >>
> > >> > >> > subsumed?
> > >>
> > >> > >>
> > >>
> > >> > >> I think logically we could delete the original files once the
> > "compacted" checkpoint
> > >>
> > >> > >> (which finish merging the compacted files and record it in the
> > checkpoint) is completed
> > >>
> > >> > >> in all the options. If there are failover before we it, we could
> > restart the merging and if
> > >>
> > >> > >> there are failover after it, we could have already recorded the
> > files in the checkpoint.
> > >>
> > >> > >>
> > >>
> > >> > >> > One way to avoid write-read-merge is by wrapping SinkWriter with
> > >>
> > >> > >> > another one, which would buffer input elements in a temporary
> > storage
> > >>
> > >> > >> > (e.g. local file) until a threshold is reached; after that, it
> > would
> > >>
> > >> > >> > invoke the original SinkWriter. And if a checkpoint barrier
> > comes in
> > >>
> > >> > >> > earlier, it would send written data to some aggregator.
> > >>
> > >> > >>
> > >>
> > >> > >> I think perhaps this seems to be a kind of WAL method? Namely we
> > first
> > >>
> > >> > >> write the elements to some WAL logs and persist them on checkpoint
> > >>
> > >> > >> (in snapshot or remote FS), or we directly write WAL logs to the
> > remote
> > >>
> > >> > >> FS eagerly.
> > >>
> > >> > >>
> > >>
> > >> > >> Sorry if I do not understand correctly somewhere.
> > >>
> > >> > >>
> > >>
> > >> > >> Best,
> > >>
> > >> > >> Yun
> > >>
> > >> > >>
> > >>
> > >> > >>
> > >>
> > >> > >> ------------------------------------------------------------------
> > >>
> > >> > >> From:Roman Khachatryan
> > >>
> > >> > >> Send Time:2021 Nov. 9 (Tue.) 22:03
> > >>
> > >> > >> To:dev
> > >>
> > >> > >> Subject:Re: [DISCUSS] FLIP-191: Extend unified Sink interface to
> > support small file compaction
> > >>
> > >> > >>
> > >>
> > >> > >> Hi everyone,
> > >>
> > >> > >>
> > >>
> > >> > >> Thanks for the proposal and the discussion, I have some remarks:
> > >>
> > >> > >> (I'm not very familiar with the new Sink API but I thought about
> > the
> > >>
> > >> > >> same problem in context of the changelog state backend)
> > >>
> > >> > >>
> > >>
> > >> > >> 1. Merging artifacts from multiple checkpoints would apparently
> > >>
> > >> > >> require multiple concurrent checkpoints (otherwise, a new
> > checkpoint
> > >>
> > >> > >> won't be started before completing the previous one; and the
> > previous
> > >>
> > >> > >> one can't be completed before durably storing the artifacts).
> > However,
> > >>
> > >> > >> concurrent checkpoints are currently not supported with Unaligned
> > >>
> > >> > >> checkpoints (this is besides increasing e2e-latency).
> > >>
> > >> > >>
> > >>
> > >> > >> 2. Asynchronous merging in an aggregator would require some
> > resolution
> > >>
> > >> > >> logic on recovery, so that a merged artifact can be used if the
> > >>
> > >> > >> original one was deleted. Otherwise, wouldn't recovery fail because
> > >>
> > >> > >> some artifacts are missing?
> > >>
> > >> > >> We could also defer deletion until the "compacted" checkpoint is
> > >>
> > >> > >> subsumed - but isn't it too late, as it will be deleted anyways
> > once
> > >>
> > >> > >> subsumed?
> > >>
> > >> > >>
> > >>
> > >> > >> 3. Writing small files, then reading and merging them for *every*
> > >>
> > >> > >> checkpoint seems worse than only reading them on recovery. I guess
> > I'm
> > >>
> > >> > >> missing some cases of reading, so to me it would make sense to
> > mention
> > >>
> > >> > >> these cases explicitly in the FLIP motivation section.
> > >>
> > >> > >>
> > >>
> > >> > >> 4. One way to avoid write-read-merge is by wrapping SinkWriter with
> > >>
> > >> > >> another one, which would buffer input elements in a temporary
> > storage
> > >>
> > >> > >> (e.g. local file) until a threshold is reached; after that, it
> > would
> > >>
> > >> > >> invoke the original SinkWriter. And if a checkpoint barrier comes
> > in
> > >>
> > >> > >> earlier, it would send written data to some aggregator. It will
> > >>
> > >> > >> increase checkpoint delay (async phase) compared to the current
> > Flink;
> > >>
> > >> > >> but not compared to the write-read-merge solution, IIUC.
> > >>
> > >> > >> Then such "BufferingSinkWriters" could aggregate input elements
> > from
> > >>
> > >> > >> each other, potentially recursively (I mean something like
> > >>
> > >> > >>
> > https://cwiki.apache.org/confluence/download/attachments/173082889/DSTL-DFS-DAG.png
> > >>
> > >> > >> )
> > >>
> > >> > >>
> > >>
> > >> > >> 5. Reducing the number of files by reducing aggregator parallelism
> > as
> > >>
> > >> > >> opposed to merging on reaching size threshold will likely be less
> > >>
> > >> > >> optimal and more difficult to configure. OTH, thresholds might be
> > more
> > >>
> > >> > >> difficult to implement and (with recursive merging) would incur
> > higher
> > >>
> > >> > >> latency. Maybe that's also something to decide explicitly or at
> > least
> > >>
> > >> > >> mention in the FLIP.
> > >>
> > >> > >>
> > >>
> > >> > >>
> > >>
> > >> > >>
> > >>
> > >> > >> Regards,
> > >>
> > >> > >> Roman
> > >>
> > >> > >>
> > >>
> > >> > >>
> > >>
> > >> > >> On Tue, Nov 9, 2021 at 5:23 AM Reo Lei wrote:
> > >>
> > >> > >> >
> > >>
> > >> > >> > Hi Fabian,
> > >>
> > >> > >> >
> > >>
> > >> > >> > Thanks for drafting the FLIP and trying to support small file
> > compaction. I
> > >>
> > >> > >> > think this feature is very urgent and valuable for users(at
> > least for me).
> > >>
> > >> > >> >
> > >>
> > >> > >> > Currently I am trying to support streaming rewrite(compact) for
> > Iceberg on
> > >>
> > >> > >> > PR#3323 . As Steven mentioned,
> > >>
> > >> > >> > Iceberg sink and compact data through the following steps:
> > >>
> > >> > >> > Step-1: Some parallel data writer(sinker) to write streaming
> > data as files.
> > >>
> > >> > >> > Step-2: A single parallelism data files committer to commit the
> > completed
> > >>
> > >> > >> > files as soon as possible to make them available.
> > >>
> > >> > >> > Step-3: Some parallel file rewriter(compactor) to collect
> > committed files
> > >>
> > >> > >> > from multiple checkpoints, and rewriter(compact) them together
> > once the
> > >>
> > >> > >> > total file size or number of files reach the threshold.
> > >>
> > >> > >> > Step-4: A single parallelism rewrite(compact) result committer
> > to commit
> > >>
> > >> > >> > the rewritten(compacted) files to replace the old files and make
> > them
> > >>
> > >> > >> > available.
> > >>
> > >> > >> >
> > >>
> > >> > >> >
> > >>
> > >> > >> > If Flink want to support small file compaction, some key point I
> > think is
> > >>
> > >> > >> > necessary:
> > >>
> > >> > >> >
> > >>
> > >> > >> > 1, Compact files from multiple checkpoints.
> > >>
> > >> > >> > I totally agree with Jingsong, because completed file size
> > usually could
> > >>
> > >> > >> > not reach the threshold in a single checkpoint. Especially for
> > partitioned
> > >>
> > >> > >> > table, we need to compact the files of each partition, but
> > usually the file
> > >>
> > >> > >> > size of each partition will be different and may not reach the
> > merge
> > >>
> > >> > >> > threshold. If we compact these files, in a single checkpoint,
> > regardless of
> > >>
> > >> > >> > whether the total file size reaches the threshold, then the
> > value of
> > >>
> > >> > >> > compacting will be diminished and we will still get small files
> > because
> > >>
> > >> > >> > these compacted files are not reach to target size. So we need
> > the
> > >>
> > >> > >> > compactor to collect committed files from multiple checkpoints
> > and compact
> > >>
> > >> > >> > them until they reach the threshold.
> > >>
> > >> > >> >
> > >>
> > >> > >> > 2, Separate write phase and compact phase.
> > >>
> > >> > >> > Users usually hope the data becomes available as soon as
> > possible, and the
> > >>
> > >> > >> > end-to-end latency is very important. I think we need to
> > separate the
> > >>
> > >> > >> > write and compact phase. For the write phase, there include the
> > Step-1
> > >>
> > >> > >> > and Step-2, we sink data as file and commit it pre checkpoint
> > and regardless
> > >>
> > >> > >> > of whether the file size it is. That could ensure the data will
> > be
> > >>
> > >> > >> > available ASAP. For the compact phase, there include the Step-3
> > >>
> > >> > >> > and Step-4, the compactor should collect committed files from
> > multiple
> > >>
> > >> > >> > checkpoints and compact them asynchronously once they reach the
> > threshold,
> > >>
> > >> > >> > and the compact committer will commit the compaction result in
> > the next
> > >>
> > >> > >> > checkpoint. We compact the committed files asynchronously
> > because we don't
> > >>
> > >> > >> > want the compaction to affect the data sink or the whole
> > pipeline.
> > >>
> > >> > >> >
> > >>
> > >> > >> > 3, Exactly once guarantee between write and compact phase.
> > >>
> > >> > >> > Once we separate write phase and compact phase, we need to
> > consider
> > >>
> > >> > >> > how to guarantee
> > >>
> > >> > >> > the exact once semantic between two phases. We should not lose
> > any data or
> > >>
> > >> > >> > files on the compactor(Step-3) in any case and cause the
> > compaction result
> > >>
> > >> > >> > to be inconsistent with before. I think flink should provide an
> > easy-to-use
> > >>
> > >> > >> > interface to make that easier.
> > >>
> > >> > >> >
> > >>
> > >> > >> > 4, Metadata operation and compaction result validation.
> > >>
> > >> > >> > In the compact phase, there may be not only compact files, but
> > also a lot
> > >>
> > >> > >> > of metadata operations, such as the iceberg needing to
> > read/write manifest
> > >>
> > >> > >> > and do MOR. And we need some interface to support users to do
> > some
> > >>
> > >> > >> > validation of the compaction result. I think these points should
> > be
> > >>
> > >> > >> > considered when we design the compaction API.
> > >>
> > >> > >> >
> > >>
> > >> > >> >
> > >>
> > >> > >> > Back to FLIP-191, option 1 looks very complicated while option 2
> > is
> > >>
> > >> > >> > relatively simple, but neither of these two solutions separates
> > the write
> > >>
> > >> > >> > phase from the compact phase. So I think we should consider the
> > points I
> > >>
> > >> > >> > mentioned above. And if you have any other questions you can
> > always feel
> > >>
> > >> > >> > free to reach out to me!
> > >>
> > >> > >> >
> > >>
> > >> > >> > BR,
> > >>
> > >> > >> > Reo
> > >>
> > >> > >> >
> > >>
> > >> > >> > Fabian Paul 于2021年11月8日周一 下午7:59写道:
> > >>
> > >> > >> >
> > >>
> > >> > >> > > Hi all,
> > >>
> > >> > >> > >
> > >>
> > >> > >> > > Thanks for the lively discussions. I am really excited to see
> > so many
> > >>
> > >> > >> > > people
> > >>
> > >> > >> > > participating in this thread. It also underlines the need that
> > many people
> > >>
> > >> > >> > > would
> > >>
> > >> > >> > > like to see a solution soon.
> > >>
> > >> > >> > >
> > >>
> > >> > >> > > I have updated the FLIP and removed the parallelism
> > configuration because
> > >>
> > >> > >> > > it is
> > >>
> > >> > >> > > unnecessary since users can configure a constant exchange key
> > to send all
> > >>
> > >> > >> > > committables to only one committable aggregator.
> > >>
> > >> > >> > >
> > >>
> > >> > >> > >
> > >>
> > >> > >> > > 1. Burden for developers w.r.t batch stream unification.
> > >>
> > >> > >> > >
> > >>
> > >> > >> > > @yun @guowei, from a theoretical point you are right about
> > exposing the
> > >>
> > >> > >> > > DataStream
> > >>
> > >> > >> > > API in the sink users have the full power to write correct
> > batch and
> > >>
> > >> > >> > > streaming
> > >>
> > >> > >> > > sinks. I think in reality a lot of users still struggle to
> > build pipelines
> > >>
> > >> > >> > > with
> > >>
> > >> > >> > > i.e. the operator pipeline which works correct in streaming
> > and batch mode.
> > >>
> > >> > >> > > Another problem I see is by exposing more deeper concepts is
> > that we
> > >>
> > >> > >> > > cannot do
> > >>
> > >> > >> > > any optimization because we cannot reason about how sinks are
> > built in the
> > >>
> > >> > >> > > future.
> > >>
> > >> > >> > >
> > >>
> > >> > >> > > We should also try to steer users towards using only
> > `Functions` to give
> > >>
> > >> > >> > > us more
> > >>
> > >> > >> > > flexibility to swap the internal operator representation. I
> > agree with
> > >>
> > >> > >> > > @yun we
> > >>
> > >> > >> > > should try to make the `ProcessFunction` more versatile to
> > work on that
> > >>
> > >> > >> > > goal but
> > >>
> > >> > >> > > I see this as unrelated to the FLIP.
> > >>
> > >> > >> > >
> > >>
> > >> > >> > >
> > >>
> > >> > >> > > 2. Regarding Commit / Global commit
> > >>
> > >> > >> > >
> > >>
> > >> > >> > > I envision the global committer to be specific depending on
> > the data lake
> > >>
> > >> > >> > > solution you want to write to. However, it is entirely
> > orthogonal to the
> > >>
> > >> > >> > > compaction.
> > >>
> > >> > >> > > Currently, I do not expect any changes w.r.t the Global commit
> > introduces
> > >>
> > >> > >> > > by
> > >>
> > >> > >> > > this FLIP.
> > >>
> > >> > >> > >
> > >>
> > >> > >> > >
> > >>
> > >> > >> > > 3. Regarding the case of trans-checkpoints merging
> > >>
> > >> > >> > >
> > >>
> > >> > >> > > @yun, as user, I would expect that if the committer receives
> > in a
> > >>
> > >> > >> > > checkpoint files
> > >>
> > >> > >> > > to merge/commit that these are also finished when the
> > checkpoint finishes.
> > >>
> > >> > >> > > I think all sinks rely on this principle currently i.e.,
> > KafkaSink needs to
> > >>
> > >> > >> > > commit all open transactions until the next checkpoint can
> > happen.
> > >>
> > >> > >> > >
> > >>
> > >> > >> > > Maybe in the future, we can somehow move the Committer#commit
> > call to an
> > >>
> > >> > >> > > asynchronous execution, but we should discuss it as a separate
> > thread.
> > >>
> > >> > >> > >
> > >>
> > >> > >> > > > We probably should first describe the different causes of
> > small files and
> > >>
> > >> > >> > > > what problems was this proposal trying to solve. I wrote a
> > data shuffling
> > >>
> > >> > >> > > > proposal [1] for Flink Iceberg sink (shared with Iceberg
> > community [2]).
> > >>
> > >> > >> > > It
> > >>
> > >> > >> > > > can address small files problems due to skewed data
> > distribution across
> > >>
> > >> > >> > > > Iceberg table partitions. Streaming shuffling before writers
> > (to files)
> > >>
> > >> > >> > > is
> > >>
> > >> > >> > > > typically more efficient than post-write file compaction
> > (which involves
> > >>
> > >> > >> > > > read-merge-write). It is usually cheaper to prevent a
> > problem (small
> > >>
> > >> > >> > > files)
> > >>
> > >> > >> > > > than fixing it.
> > >>
> > >> > >> > >
> > >>
> > >> > >> > >
> > >>
> > >> > >> > > @steven you are raising a good point, although I think only
> > using a
> > >>
> > >> > >> > > customizable
> > >>
> > >> > >> > > shuffle won't address the generation of small files. One
> > assumption is that
> > >>
> > >> > >> > > at least the sink generates one file per subtask, which can
> > already be too
> > >>
> > >> > >> > > many.
> > >>
> > >> > >> > > Another problem is that with low checkpointing intervals, the
> > files do not
> > >>
> > >> > >> > > meet
> > >>
> > >> > >> > > the required size. The latter point is probably addressable by
> > changing the
> > >>
> > >> > >> > > checkpoint interval, which might be inconvenient for some
> > users.
> > >>
> > >> > >> > >
> > >>
> > >> > >> > > > The sink coordinator checkpoint problem (mentioned in option
> > 1) would be
> > >>
> > >> > >> > > > great if Flink can address it. In the spirit of source
> > >>
> > >> > >> > > (enumerator-reader)
> > >>
> > >> > >> > > > and sink (writer-coordinator) duality, sink coordinator
> > checkpoint should
> > >>
> > >> > >> > > > happen after the writer operator. This would be a natural
> > fit to support
> > >>
> > >> > >> > > > global committer in FLIP-143. It is probably an orthogonal
> > matter to this
> > >>
> > >> > >> > > > proposal.
> > >>
> > >> > >> > >
> > >>
> > >> > >> > >
> > >>
> > >> > >> > > To me the question here is what are the benefits of having a
> > coordinator in
> > >>
> > >> > >> > > comparison to a global committer/aggregator operator.
> > >>
> > >> > >> > >
> > >>
> > >> > >> > > > Personally, I am usually in favor of keeping streaming
> > ingestion (to data
> > >>
> > >> > >> > > > lake) relatively simple and stable. Also sometimes
> > compaction and sorting
> > >>
> > >> > >> > > > are performed together in data rewrite maintenance jobs to
> > improve read
> > >>
> > >> > >> > > > performance. In that case, the value of compacting (in Flink
> > streaming
> > >>
> > >> > >> > > > ingestion) diminishes.
> > >>
> > >> > >> > >
> > >>
> > >> > >> > >
> > >>
> > >> > >> > > I agree it is always possible to have scheduled maintenance
> > jobs keeping
> > >>
> > >> > >> > > care of
> > >>
> > >> > >> > > your data i.e., doing compaction. Unfortunately, the downside
> > is that you
> > >>
> > >> > >> > > have to your data after it is already available for other
> > downstream
> > >>
> > >> > >> > > consumers.
> > >>
> > >> > >> > > I guess this can lead to all kinds of visibility problems. I
> > am also
> > >>
> > >> > >> > > surprised that
> > >>
> > >> > >> > > you personally are a fan of this approach and, on the other
> > hand, are
> > >>
> > >> > >> > > developing
> > >>
> > >> > >> > > the Iceberg sink, which goes somewhat against your mentioned
> > principle of
> > >>
> > >> > >> > > keeping
> > >>
> > >> > >> > > the sink simple.
> > >>
> > >> > >> > >
> > >>
> > >> > >> > > > Currently, it is unclear from the doc and this thread where
> > the
> > >>
> > >> > >> > > compaction
> > >>
> > >> > >> > > > is actually happening. Jingsong's reply described one model
> > >>
> > >> > >> > > > writer (parallel) -> aggregator (single-parallelism
> > compaction planner)
> > >>
> > >> > >> > > ->
> > >>
> > >> > >> > > > compactor (parallel) -> global committer (single-parallelism)
> > >>
> > >> > >> > >
> > >>
> > >> > >> > >
> > >>
> > >> > >> > > My idea of the topology is very similar to the one outlined by
> > Jinsong. The
> > >>
> > >> > >> > > compaction will happen in the committer operator.
> > >>
> > >> > >> > >
> > >>
> > >> > >> > > >
> > >>
> > >> > >> > > > In the Iceberg community, the following model has been
> > discussed. It is
> > >>
> > >> > >> > > > better for Iceberg because it won't delay the data
> > availability.
> > >>
> > >> > >> > > > writer (parallel) -> global committer for append (single
> > parallelism) ->
> > >>
> > >> > >> > > > compactor (parallel) -> global committer for rewrite commit
> > (single
> > >>
> > >> > >> > > > parallelism)
> > >>
> > >> > >> > >
> > >>
> > >> > >> > >
> > >>
> > >> > >> > > From a quick glimpse, it seems that the exact same topology is
> > possible to
> > >>
> > >> > >> > > express with the committable aggregator, but this definitely
> > depends on
> > >>
> > >> > >> > > the exact
> > >>
> > >> > >> > > setup.
> > >>
> > >> > >> > >
> > >>
> > >> > >> > > Best,
> > >>
> > >> > >> > > Fabian
> > >>
> > >> > >>
> >

Re: Re: [DISCUSS] FLIP-191: Extend unified Sink interface to support small file compaction

Posted by Till Rohrmann <tr...@apache.org>.
Hi Fabian,

quick question on your comment 3. If there is a pipelined data exchange
with a keyBy between the writers/committers and the component that does the
global commit, then there will only be a single failover region. So is it
correct that you assumed blocking data exchanges for the scenario you
described?

Cheers,
Till

On Thu, Dec 16, 2021 at 9:23 AM Fabian Paul <fp...@apache.org> wrote:

> Hi Yun,
>
> Thanks for your fast feedback. Let me clarify your points.
>
> 1. We solve it by using StreamExchangeMode.BATCH before any exchange.
> That obviously doesn’t help with lost TM but we would need to employ
> HA storage for that. Same issue as now and orthogonal.
>
> 2. Extending V1 with V2 or vice versa would require renames of methods
> (since return types are non-optional) and is making API changes. Even
> though Experimental, we want to give connector developers the
> opportunity to provide 1 implementation for all of Flink 1.X. We will
> offer an internal adapter from V1 to V2, 2 sinkTo , and internally
> just have one code-path.
>
> 3. DataStreamSink would act as a unified view on all the operators and
> update them all at once when using setParallelism and so on (setName
> and setUid will receive suffixes per operator).
> Iceberg actually has a different requirement: They want to have a
> committer with parallelism 1 but as a coordinator such that
> embarrassingly parallel pipelines have different fail-over regions. I
> was thinking that in this case, they need to implement a no-op
> committer (that just forwards the committables) and use a post-commit
> topology that achieves that.
> Another option is that they use the preCommit topology and insert a
> constant key-by that forwards all committables to a single committer.
> We are planning to provide building blocks for such pipelines as we
> go.
>
> Best,
> Fabian
>
> On Thu, Dec 16, 2021 at 5:50 AM Yun Gao <yu...@aliyun.com> wrote:
> >
> > Hi Fabian,
> >
> > Very thanks for the update! I think the latest version in general looks
> good from my side
> > and I think using separate feature interface would be much more easy to
> understand
> > and extend in the future. I have some pending issues on the details
> though:
> >
> > 1. The first one is if we could support end-to-end exactly-once with
> post-committing
> > topology in the batch mode ? Since for the batch mode, currently we
> could only commit
> >  all the transactions after the whole job is finished, otherwise if
> there are JM failover or the
> > writer / committer get restarted due to indeterminstic (A -> [B1, B2],
> A, B1 have finished and
> >  B2 failed, if -> is rebalance / random / rescale,  all of A, B1, B2
> would restarted), there might
> > be repeat records. Previously one possible thought is to move committer
> and global committer
> >  to the operator coordinator, but if it is a topology, we might need
> some other kind of solutions?
> >
> > 2. I also want to have a dobule confirmation with the compatibility:
> since the old sink is also named
> > with Sink, do we want to put the Sink v2 in a new package ? Besides,
> since we might want to keep
> > only have one `sinkTo(Sink<?> sink)` , perhaps we also need to make the
> Sink v1 to be a subclass of
> > Sink v2 and extends the stateful and two-phase-commit sinks, right?
> >
> > 3. I'd like also have a confirmation on ours thoughts with the
> `DataStreamSink` returned by the sinkTo method:
> > The main issue is how do we implement the method like `setParallelism`
> or `setMaxParallelism` since now the sink
> > would be translated to multiple transformations? perhaps we could make
> it the default values for all the transformations
> > for the sink? A related issue would be for iceberg sink, I think it
> would need to have only one committer to avoid the
> > competition of the optimistic locks (which would cause performance
> degradation), then it might need to have N writers
> > with 1 committers, to build such topology, perhaps we might need to add
> new methods to specify the parallelism of
> > the writers and committers separately?
> >
> > Best,
> > Yun
> >
> >
> > ------------------Original Mail ------------------
> > Sender:Fabian Paul <fp...@apache.org>
> > Send Date:Mon Dec 13 23:59:43 2021
> > Recipients:dev <de...@flink.apache.org>
> > Subject:Re: [DISCUSS] FLIP-191: Extend unified Sink interface to support
> small file compaction
> >>
> >> Hi all,
> >>
> >>
> >>
> >> After a lot of discussions with different, we received very fruitful
> >>
> >> feedback and reworked the ideas behind this FLIP. Initially, we had
> >>
> >> the impression that the compaction problem is solvable by a single
> >>
> >> topology that we can reuse across different sinks. We now have a
> >>
> >> better understanding that different external systems require different
> >>
> >> compaction mechanism i.e. Hive requires compaction before finally
> >>
> >> registering the file in the metastore or Iceberg compacts the files
> >>
> >> after they have been registered and just lazily compacts them.
> >>
> >>
> >>
> >> Considering all these different views we came up with a design that
> >>
> >> builds upon what @guowei.mgw@gmail.com and @yungao.gy@aliyun.com have
> >>
> >> proposed at the beginning. We allow inserting custom topologies before
> >>
> >> and after the SinkWriters and Committers. Furthermore, we do not see
> >>
> >> it as a downside. The Sink interfaces that will expose the DataStream
> >>
> >> to the user reside in flink-streaming-java in contrast to the basic
> >>
> >> Sink interfaces that reside fin flink-core deem it to be only used by
> >>
> >> expert users.
> >>
> >>
> >>
> >> Moreover, we also wanted to remove the global committer from the
> >>
> >> unified Sink interfaces and replace it with a custom post-commit
> >>
> >> topology. Unfortunately, we cannot do it without breaking the Sink
> >>
> >> interface since the GlobalCommittables are part of the parameterized
> >>
> >> Sink interface. Thus, we propose building a new Sink V2 interface
> >>
> >> consisting of composable interfaces that do not offer the
> >>
> >> GlobalCommitter anymore. We will implement a utility to extend a Sink
> >>
> >> with post topology that mimics the behavior of the GlobalCommitter.
> >>
> >> The new Sink V2 provides the same sort of methods as the Sink V1
> >>
> >> interface, so a migration of sinks that do not use the GlobalCommitter
> >>
> >> should be very easy.
> >>
> >> We plan to keep the existing Sink V1 interfaces to not break
> >>
> >> externally built sinks. As part of this FLIP, we migrate all the
> >>
> >> connectors inside of the main repository to the new Sink V2 API.
> >>
> >>
> >>
> >> The FLIP document is also updated and includes the proposed changes.
> >>
> >>
> >>
> >> Looking forward to your feedback,
> >>
> >> Fabian
> >>
> >>
> >>
> >>
> https://cwiki.apache.org/confluence/display/FLINK/FLIP-191%3A+Extend+unified+Sink+interface+to+support+small+file+compaction
> >>
> >>
> >>
> >>
> >>
> >> On Thu, Dec 2, 2021 at 10:15 AM Roman Khachatryan wrote:
> >>
> >> >
> >>
> >> > Thanks for clarifying (I was initially confused by merging state files
> >>
> >> > rather than output files).
> >>
> >> >
> >>
> >> > > At some point, Flink will definitely have some WAL adapter that can
> turn any sink into an exactly-once sink (with some caveats). For now, we
> keep that as an orthogonal solution as it has a rather high price (bursty
> workload with high latency). Ideally, we can keep the compaction
> asynchronously...
> >>
> >> >
> >>
> >> > Yes, that would be something like a WAL. I agree that it would have a
> >>
> >> > different set of trade-offs.
> >>
> >> >
> >>
> >> >
> >>
> >> > Regards,
> >>
> >> > Roman
> >>
> >> >
> >>
> >> > On Mon, Nov 29, 2021 at 3:33 PM Arvid Heise wrote:
> >>
> >> > >>
> >>
> >> > >> > One way to avoid write-read-merge is by wrapping SinkWriter with
> >>
> >> > >> > another one, which would buffer input elements in a temporary
> storage
> >>
> >> > >> > (e.g. local file) until a threshold is reached; after that, it
> would
> >>
> >> > >> > invoke the original SinkWriter. And if a checkpoint barrier
> comes in
> >>
> >> > >> > earlier, it would send written data to some aggregator.
> >>
> >> > >>
> >>
> >> > >> I think perhaps this seems to be a kind of WAL method? Namely we
> first
> >>
> >> > >> write the elements to some WAL logs and persist them on checkpoint
> >>
> >> > >> (in snapshot or remote FS), or we directly write WAL logs to the
> remote
> >>
> >> > >> FS eagerly.
> >>
> >> > >>
> >>
> >> > > At some point, Flink will definitely have some WAL adapter that can
> turn any sink into an exactly-once sink (with some caveats). For now, we
> keep that as an orthogonal solution as it has a rather high price (bursty
> workload with high latency). Ideally, we can keep the compaction
> asynchronously...
> >>
> >> > >
> >>
> >> > > On Mon, Nov 29, 2021 at 8:52 AM Yun Gao wrote:
> >>
> >> > >>
> >>
> >> > >> Hi,
> >>
> >> > >>
> >>
> >> > >> @Roman very sorry for the late response for a long time,
> >>
> >> > >>
> >>
> >> > >> > Merging artifacts from multiple checkpoints would apparently
> >>
> >> > >> require multiple concurrent checkpoints
> >>
> >> > >>
> >>
> >> > >> I think it might not need concurrent checkpoints: suppose some
> >>
> >> > >> operators (like the committer aggregator in the option 2) maintains
> >>
> >> > >> the list of files to merge, it could stores the lists of files to
> merge
> >>
> >> > >> in the states, then after several checkpoints are done and we have
> >>
> >> > >> enough files, we could merge all the files in the list.
> >>
> >> > >>
> >>
> >> > >> > Asynchronous merging in an aggregator would require some
> resolution
> >>
> >> > >> > logic on recovery, so that a merged artifact can be used if the
> >>
> >> > >> > original one was deleted. Otherwise, wouldn't recovery fail
> because
> >>
> >> > >> > some artifacts are missing?
> >>
> >> > >> > We could also defer deletion until the "compacted" checkpoint is
> >>
> >> > >> > subsumed - but isn't it too late, as it will be deleted anyways
> once
> >>
> >> > >> > subsumed?
> >>
> >> > >>
> >>
> >> > >> I think logically we could delete the original files once the
> "compacted" checkpoint
> >>
> >> > >> (which finish merging the compacted files and record it in the
> checkpoint) is completed
> >>
> >> > >> in all the options. If there are failover before we it, we could
> restart the merging and if
> >>
> >> > >> there are failover after it, we could have already recorded the
> files in the checkpoint.
> >>
> >> > >>
> >>
> >> > >> > One way to avoid write-read-merge is by wrapping SinkWriter with
> >>
> >> > >> > another one, which would buffer input elements in a temporary
> storage
> >>
> >> > >> > (e.g. local file) until a threshold is reached; after that, it
> would
> >>
> >> > >> > invoke the original SinkWriter. And if a checkpoint barrier
> comes in
> >>
> >> > >> > earlier, it would send written data to some aggregator.
> >>
> >> > >>
> >>
> >> > >> I think perhaps this seems to be a kind of WAL method? Namely we
> first
> >>
> >> > >> write the elements to some WAL logs and persist them on checkpoint
> >>
> >> > >> (in snapshot or remote FS), or we directly write WAL logs to the
> remote
> >>
> >> > >> FS eagerly.
> >>
> >> > >>
> >>
> >> > >> Sorry if I do not understand correctly somewhere.
> >>
> >> > >>
> >>
> >> > >> Best,
> >>
> >> > >> Yun
> >>
> >> > >>
> >>
> >> > >>
> >>
> >> > >> ------------------------------------------------------------------
> >>
> >> > >> From:Roman Khachatryan
> >>
> >> > >> Send Time:2021 Nov. 9 (Tue.) 22:03
> >>
> >> > >> To:dev
> >>
> >> > >> Subject:Re: [DISCUSS] FLIP-191: Extend unified Sink interface to
> support small file compaction
> >>
> >> > >>
> >>
> >> > >> Hi everyone,
> >>
> >> > >>
> >>
> >> > >> Thanks for the proposal and the discussion, I have some remarks:
> >>
> >> > >> (I'm not very familiar with the new Sink API but I thought about
> the
> >>
> >> > >> same problem in context of the changelog state backend)
> >>
> >> > >>
> >>
> >> > >> 1. Merging artifacts from multiple checkpoints would apparently
> >>
> >> > >> require multiple concurrent checkpoints (otherwise, a new
> checkpoint
> >>
> >> > >> won't be started before completing the previous one; and the
> previous
> >>
> >> > >> one can't be completed before durably storing the artifacts).
> However,
> >>
> >> > >> concurrent checkpoints are currently not supported with Unaligned
> >>
> >> > >> checkpoints (this is besides increasing e2e-latency).
> >>
> >> > >>
> >>
> >> > >> 2. Asynchronous merging in an aggregator would require some
> resolution
> >>
> >> > >> logic on recovery, so that a merged artifact can be used if the
> >>
> >> > >> original one was deleted. Otherwise, wouldn't recovery fail because
> >>
> >> > >> some artifacts are missing?
> >>
> >> > >> We could also defer deletion until the "compacted" checkpoint is
> >>
> >> > >> subsumed - but isn't it too late, as it will be deleted anyways
> once
> >>
> >> > >> subsumed?
> >>
> >> > >>
> >>
> >> > >> 3. Writing small files, then reading and merging them for *every*
> >>
> >> > >> checkpoint seems worse than only reading them on recovery. I guess
> I'm
> >>
> >> > >> missing some cases of reading, so to me it would make sense to
> mention
> >>
> >> > >> these cases explicitly in the FLIP motivation section.
> >>
> >> > >>
> >>
> >> > >> 4. One way to avoid write-read-merge is by wrapping SinkWriter with
> >>
> >> > >> another one, which would buffer input elements in a temporary
> storage
> >>
> >> > >> (e.g. local file) until a threshold is reached; after that, it
> would
> >>
> >> > >> invoke the original SinkWriter. And if a checkpoint barrier comes
> in
> >>
> >> > >> earlier, it would send written data to some aggregator. It will
> >>
> >> > >> increase checkpoint delay (async phase) compared to the current
> Flink;
> >>
> >> > >> but not compared to the write-read-merge solution, IIUC.
> >>
> >> > >> Then such "BufferingSinkWriters" could aggregate input elements
> from
> >>
> >> > >> each other, potentially recursively (I mean something like
> >>
> >> > >>
> https://cwiki.apache.org/confluence/download/attachments/173082889/DSTL-DFS-DAG.png
> >>
> >> > >> )
> >>
> >> > >>
> >>
> >> > >> 5. Reducing the number of files by reducing aggregator parallelism
> as
> >>
> >> > >> opposed to merging on reaching size threshold will likely be less
> >>
> >> > >> optimal and more difficult to configure. OTH, thresholds might be
> more
> >>
> >> > >> difficult to implement and (with recursive merging) would incur
> higher
> >>
> >> > >> latency. Maybe that's also something to decide explicitly or at
> least
> >>
> >> > >> mention in the FLIP.
> >>
> >> > >>
> >>
> >> > >>
> >>
> >> > >>
> >>
> >> > >> Regards,
> >>
> >> > >> Roman
> >>
> >> > >>
> >>
> >> > >>
> >>
> >> > >> On Tue, Nov 9, 2021 at 5:23 AM Reo Lei wrote:
> >>
> >> > >> >
> >>
> >> > >> > Hi Fabian,
> >>
> >> > >> >
> >>
> >> > >> > Thanks for drafting the FLIP and trying to support small file
> compaction. I
> >>
> >> > >> > think this feature is very urgent and valuable for users(at
> least for me).
> >>
> >> > >> >
> >>
> >> > >> > Currently I am trying to support streaming rewrite(compact) for
> Iceberg on
> >>
> >> > >> > PR#3323 . As Steven mentioned,
> >>
> >> > >> > Iceberg sink and compact data through the following steps:
> >>
> >> > >> > Step-1: Some parallel data writer(sinker) to write streaming
> data as files.
> >>
> >> > >> > Step-2: A single parallelism data files committer to commit the
> completed
> >>
> >> > >> > files as soon as possible to make them available.
> >>
> >> > >> > Step-3: Some parallel file rewriter(compactor) to collect
> committed files
> >>
> >> > >> > from multiple checkpoints, and rewriter(compact) them together
> once the
> >>
> >> > >> > total file size or number of files reach the threshold.
> >>
> >> > >> > Step-4: A single parallelism rewrite(compact) result committer
> to commit
> >>
> >> > >> > the rewritten(compacted) files to replace the old files and make
> them
> >>
> >> > >> > available.
> >>
> >> > >> >
> >>
> >> > >> >
> >>
> >> > >> > If Flink want to support small file compaction, some key point I
> think is
> >>
> >> > >> > necessary:
> >>
> >> > >> >
> >>
> >> > >> > 1, Compact files from multiple checkpoints.
> >>
> >> > >> > I totally agree with Jingsong, because completed file size
> usually could
> >>
> >> > >> > not reach the threshold in a single checkpoint. Especially for
> partitioned
> >>
> >> > >> > table, we need to compact the files of each partition, but
> usually the file
> >>
> >> > >> > size of each partition will be different and may not reach the
> merge
> >>
> >> > >> > threshold. If we compact these files, in a single checkpoint,
> regardless of
> >>
> >> > >> > whether the total file size reaches the threshold, then the
> value of
> >>
> >> > >> > compacting will be diminished and we will still get small files
> because
> >>
> >> > >> > these compacted files are not reach to target size. So we need
> the
> >>
> >> > >> > compactor to collect committed files from multiple checkpoints
> and compact
> >>
> >> > >> > them until they reach the threshold.
> >>
> >> > >> >
> >>
> >> > >> > 2, Separate write phase and compact phase.
> >>
> >> > >> > Users usually hope the data becomes available as soon as
> possible, and the
> >>
> >> > >> > end-to-end latency is very important. I think we need to
> separate the
> >>
> >> > >> > write and compact phase. For the write phase, there include the
> Step-1
> >>
> >> > >> > and Step-2, we sink data as file and commit it pre checkpoint
> and regardless
> >>
> >> > >> > of whether the file size it is. That could ensure the data will
> be
> >>
> >> > >> > available ASAP. For the compact phase, there include the Step-3
> >>
> >> > >> > and Step-4, the compactor should collect committed files from
> multiple
> >>
> >> > >> > checkpoints and compact them asynchronously once they reach the
> threshold,
> >>
> >> > >> > and the compact committer will commit the compaction result in
> the next
> >>
> >> > >> > checkpoint. We compact the committed files asynchronously
> because we don't
> >>
> >> > >> > want the compaction to affect the data sink or the whole
> pipeline.
> >>
> >> > >> >
> >>
> >> > >> > 3, Exactly once guarantee between write and compact phase.
> >>
> >> > >> > Once we separate write phase and compact phase, we need to
> consider
> >>
> >> > >> > how to guarantee
> >>
> >> > >> > the exact once semantic between two phases. We should not lose
> any data or
> >>
> >> > >> > files on the compactor(Step-3) in any case and cause the
> compaction result
> >>
> >> > >> > to be inconsistent with before. I think flink should provide an
> easy-to-use
> >>
> >> > >> > interface to make that easier.
> >>
> >> > >> >
> >>
> >> > >> > 4, Metadata operation and compaction result validation.
> >>
> >> > >> > In the compact phase, there may be not only compact files, but
> also a lot
> >>
> >> > >> > of metadata operations, such as the iceberg needing to
> read/write manifest
> >>
> >> > >> > and do MOR. And we need some interface to support users to do
> some
> >>
> >> > >> > validation of the compaction result. I think these points should
> be
> >>
> >> > >> > considered when we design the compaction API.
> >>
> >> > >> >
> >>
> >> > >> >
> >>
> >> > >> > Back to FLIP-191, option 1 looks very complicated while option 2
> is
> >>
> >> > >> > relatively simple, but neither of these two solutions separates
> the write
> >>
> >> > >> > phase from the compact phase. So I think we should consider the
> points I
> >>
> >> > >> > mentioned above. And if you have any other questions you can
> always feel
> >>
> >> > >> > free to reach out to me!
> >>
> >> > >> >
> >>
> >> > >> > BR,
> >>
> >> > >> > Reo
> >>
> >> > >> >
> >>
> >> > >> > Fabian Paul 于2021年11月8日周一 下午7:59写道:
> >>
> >> > >> >
> >>
> >> > >> > > Hi all,
> >>
> >> > >> > >
> >>
> >> > >> > > Thanks for the lively discussions. I am really excited to see
> so many
> >>
> >> > >> > > people
> >>
> >> > >> > > participating in this thread. It also underlines the need that
> many people
> >>
> >> > >> > > would
> >>
> >> > >> > > like to see a solution soon.
> >>
> >> > >> > >
> >>
> >> > >> > > I have updated the FLIP and removed the parallelism
> configuration because
> >>
> >> > >> > > it is
> >>
> >> > >> > > unnecessary since users can configure a constant exchange key
> to send all
> >>
> >> > >> > > committables to only one committable aggregator.
> >>
> >> > >> > >
> >>
> >> > >> > >
> >>
> >> > >> > > 1. Burden for developers w.r.t batch stream unification.
> >>
> >> > >> > >
> >>
> >> > >> > > @yun @guowei, from a theoretical point you are right about
> exposing the
> >>
> >> > >> > > DataStream
> >>
> >> > >> > > API in the sink users have the full power to write correct
> batch and
> >>
> >> > >> > > streaming
> >>
> >> > >> > > sinks. I think in reality a lot of users still struggle to
> build pipelines
> >>
> >> > >> > > with
> >>
> >> > >> > > i.e. the operator pipeline which works correct in streaming
> and batch mode.
> >>
> >> > >> > > Another problem I see is by exposing more deeper concepts is
> that we
> >>
> >> > >> > > cannot do
> >>
> >> > >> > > any optimization because we cannot reason about how sinks are
> built in the
> >>
> >> > >> > > future.
> >>
> >> > >> > >
> >>
> >> > >> > > We should also try to steer users towards using only
> `Functions` to give
> >>
> >> > >> > > us more
> >>
> >> > >> > > flexibility to swap the internal operator representation. I
> agree with
> >>
> >> > >> > > @yun we
> >>
> >> > >> > > should try to make the `ProcessFunction` more versatile to
> work on that
> >>
> >> > >> > > goal but
> >>
> >> > >> > > I see this as unrelated to the FLIP.
> >>
> >> > >> > >
> >>
> >> > >> > >
> >>
> >> > >> > > 2. Regarding Commit / Global commit
> >>
> >> > >> > >
> >>
> >> > >> > > I envision the global committer to be specific depending on
> the data lake
> >>
> >> > >> > > solution you want to write to. However, it is entirely
> orthogonal to the
> >>
> >> > >> > > compaction.
> >>
> >> > >> > > Currently, I do not expect any changes w.r.t the Global commit
> introduces
> >>
> >> > >> > > by
> >>
> >> > >> > > this FLIP.
> >>
> >> > >> > >
> >>
> >> > >> > >
> >>
> >> > >> > > 3. Regarding the case of trans-checkpoints merging
> >>
> >> > >> > >
> >>
> >> > >> > > @yun, as user, I would expect that if the committer receives
> in a
> >>
> >> > >> > > checkpoint files
> >>
> >> > >> > > to merge/commit that these are also finished when the
> checkpoint finishes.
> >>
> >> > >> > > I think all sinks rely on this principle currently i.e.,
> KafkaSink needs to
> >>
> >> > >> > > commit all open transactions until the next checkpoint can
> happen.
> >>
> >> > >> > >
> >>
> >> > >> > > Maybe in the future, we can somehow move the Committer#commit
> call to an
> >>
> >> > >> > > asynchronous execution, but we should discuss it as a separate
> thread.
> >>
> >> > >> > >
> >>
> >> > >> > > > We probably should first describe the different causes of
> small files and
> >>
> >> > >> > > > what problems was this proposal trying to solve. I wrote a
> data shuffling
> >>
> >> > >> > > > proposal [1] for Flink Iceberg sink (shared with Iceberg
> community [2]).
> >>
> >> > >> > > It
> >>
> >> > >> > > > can address small files problems due to skewed data
> distribution across
> >>
> >> > >> > > > Iceberg table partitions. Streaming shuffling before writers
> (to files)
> >>
> >> > >> > > is
> >>
> >> > >> > > > typically more efficient than post-write file compaction
> (which involves
> >>
> >> > >> > > > read-merge-write). It is usually cheaper to prevent a
> problem (small
> >>
> >> > >> > > files)
> >>
> >> > >> > > > than fixing it.
> >>
> >> > >> > >
> >>
> >> > >> > >
> >>
> >> > >> > > @steven you are raising a good point, although I think only
> using a
> >>
> >> > >> > > customizable
> >>
> >> > >> > > shuffle won't address the generation of small files. One
> assumption is that
> >>
> >> > >> > > at least the sink generates one file per subtask, which can
> already be too
> >>
> >> > >> > > many.
> >>
> >> > >> > > Another problem is that with low checkpointing intervals, the
> files do not
> >>
> >> > >> > > meet
> >>
> >> > >> > > the required size. The latter point is probably addressable by
> changing the
> >>
> >> > >> > > checkpoint interval, which might be inconvenient for some
> users.
> >>
> >> > >> > >
> >>
> >> > >> > > > The sink coordinator checkpoint problem (mentioned in option
> 1) would be
> >>
> >> > >> > > > great if Flink can address it. In the spirit of source
> >>
> >> > >> > > (enumerator-reader)
> >>
> >> > >> > > > and sink (writer-coordinator) duality, sink coordinator
> checkpoint should
> >>
> >> > >> > > > happen after the writer operator. This would be a natural
> fit to support
> >>
> >> > >> > > > global committer in FLIP-143. It is probably an orthogonal
> matter to this
> >>
> >> > >> > > > proposal.
> >>
> >> > >> > >
> >>
> >> > >> > >
> >>
> >> > >> > > To me the question here is what are the benefits of having a
> coordinator in
> >>
> >> > >> > > comparison to a global committer/aggregator operator.
> >>
> >> > >> > >
> >>
> >> > >> > > > Personally, I am usually in favor of keeping streaming
> ingestion (to data
> >>
> >> > >> > > > lake) relatively simple and stable. Also sometimes
> compaction and sorting
> >>
> >> > >> > > > are performed together in data rewrite maintenance jobs to
> improve read
> >>
> >> > >> > > > performance. In that case, the value of compacting (in Flink
> streaming
> >>
> >> > >> > > > ingestion) diminishes.
> >>
> >> > >> > >
> >>
> >> > >> > >
> >>
> >> > >> > > I agree it is always possible to have scheduled maintenance
> jobs keeping
> >>
> >> > >> > > care of
> >>
> >> > >> > > your data i.e., doing compaction. Unfortunately, the downside
> is that you
> >>
> >> > >> > > have to your data after it is already available for other
> downstream
> >>
> >> > >> > > consumers.
> >>
> >> > >> > > I guess this can lead to all kinds of visibility problems. I
> am also
> >>
> >> > >> > > surprised that
> >>
> >> > >> > > you personally are a fan of this approach and, on the other
> hand, are
> >>
> >> > >> > > developing
> >>
> >> > >> > > the Iceberg sink, which goes somewhat against your mentioned
> principle of
> >>
> >> > >> > > keeping
> >>
> >> > >> > > the sink simple.
> >>
> >> > >> > >
> >>
> >> > >> > > > Currently, it is unclear from the doc and this thread where
> the
> >>
> >> > >> > > compaction
> >>
> >> > >> > > > is actually happening. Jingsong's reply described one model
> >>
> >> > >> > > > writer (parallel) -> aggregator (single-parallelism
> compaction planner)
> >>
> >> > >> > > ->
> >>
> >> > >> > > > compactor (parallel) -> global committer (single-parallelism)
> >>
> >> > >> > >
> >>
> >> > >> > >
> >>
> >> > >> > > My idea of the topology is very similar to the one outlined by
> Jinsong. The
> >>
> >> > >> > > compaction will happen in the committer operator.
> >>
> >> > >> > >
> >>
> >> > >> > > >
> >>
> >> > >> > > > In the Iceberg community, the following model has been
> discussed. It is
> >>
> >> > >> > > > better for Iceberg because it won't delay the data
> availability.
> >>
> >> > >> > > > writer (parallel) -> global committer for append (single
> parallelism) ->
> >>
> >> > >> > > > compactor (parallel) -> global committer for rewrite commit
> (single
> >>
> >> > >> > > > parallelism)
> >>
> >> > >> > >
> >>
> >> > >> > >
> >>
> >> > >> > > From a quick glimpse, it seems that the exact same topology is
> possible to
> >>
> >> > >> > > express with the committable aggregator, but this definitely
> depends on
> >>
> >> > >> > > the exact
> >>
> >> > >> > > setup.
> >>
> >> > >> > >
> >>
> >> > >> > > Best,
> >>
> >> > >> > > Fabian
> >>
> >> > >>
>

Re: Re: [DISCUSS] FLIP-191: Extend unified Sink interface to support small file compaction

Posted by Fabian Paul <fp...@apache.org>.
Hi Yun,

Thanks for your fast feedback. Let me clarify your points.

1. We solve it by using StreamExchangeMode.BATCH before any exchange.
That obviously doesn’t help with lost TM but we would need to employ
HA storage for that. Same issue as now and orthogonal.

2. Extending V1 with V2 or vice versa would require renames of methods
(since return types are non-optional) and is making API changes. Even
though Experimental, we want to give connector developers the
opportunity to provide 1 implementation for all of Flink 1.X. We will
offer an internal adapter from V1 to V2, 2 sinkTo , and internally
just have one code-path.

3. DataStreamSink would act as a unified view on all the operators and
update them all at once when using setParallelism and so on (setName
and setUid will receive suffixes per operator).
Iceberg actually has a different requirement: They want to have a
committer with parallelism 1 but as a coordinator such that
embarrassingly parallel pipelines have different fail-over regions. I
was thinking that in this case, they need to implement a no-op
committer (that just forwards the committables) and use a post-commit
topology that achieves that.
Another option is that they use the preCommit topology and insert a
constant key-by that forwards all committables to a single committer.
We are planning to provide building blocks for such pipelines as we
go.

Best,
Fabian

On Thu, Dec 16, 2021 at 5:50 AM Yun Gao <yu...@aliyun.com> wrote:
>
> Hi Fabian,
>
> Very thanks for the update! I think the latest version in general looks good from my side
> and I think using separate feature interface would be much more easy to understand
> and extend in the future. I have some pending issues on the details though:
>
> 1. The first one is if we could support end-to-end exactly-once with post-committing
> topology in the batch mode ? Since for the batch mode, currently we could only commit
>  all the transactions after the whole job is finished, otherwise if there are JM failover or the
> writer / committer get restarted due to indeterminstic (A -> [B1, B2], A, B1 have finished and
>  B2 failed, if -> is rebalance / random / rescale,  all of A, B1, B2 would restarted), there might
> be repeat records. Previously one possible thought is to move committer and global committer
>  to the operator coordinator, but if it is a topology, we might need some other kind of solutions?
>
> 2. I also want to have a dobule confirmation with the compatibility: since the old sink is also named
> with Sink, do we want to put the Sink v2 in a new package ? Besides, since we might want to keep
> only have one `sinkTo(Sink<?> sink)` , perhaps we also need to make the Sink v1 to be a subclass of
> Sink v2 and extends the stateful and two-phase-commit sinks, right?
>
> 3. I'd like also have a confirmation on ours thoughts with the `DataStreamSink` returned by the sinkTo method:
> The main issue is how do we implement the method like `setParallelism` or `setMaxParallelism` since now the sink
> would be translated to multiple transformations? perhaps we could make it the default values for all the transformations
> for the sink? A related issue would be for iceberg sink, I think it would need to have only one committer to avoid the
> competition of the optimistic locks (which would cause performance degradation), then it might need to have N writers
> with 1 committers, to build such topology, perhaps we might need to add new methods to specify the parallelism of
> the writers and committers separately?
>
> Best,
> Yun
>
>
> ------------------Original Mail ------------------
> Sender:Fabian Paul <fp...@apache.org>
> Send Date:Mon Dec 13 23:59:43 2021
> Recipients:dev <de...@flink.apache.org>
> Subject:Re: [DISCUSS] FLIP-191: Extend unified Sink interface to support small file compaction
>>
>> Hi all,
>>
>>
>>
>> After a lot of discussions with different, we received very fruitful
>>
>> feedback and reworked the ideas behind this FLIP. Initially, we had
>>
>> the impression that the compaction problem is solvable by a single
>>
>> topology that we can reuse across different sinks. We now have a
>>
>> better understanding that different external systems require different
>>
>> compaction mechanism i.e. Hive requires compaction before finally
>>
>> registering the file in the metastore or Iceberg compacts the files
>>
>> after they have been registered and just lazily compacts them.
>>
>>
>>
>> Considering all these different views we came up with a design that
>>
>> builds upon what @guowei.mgw@gmail.com and @yungao.gy@aliyun.com have
>>
>> proposed at the beginning. We allow inserting custom topologies before
>>
>> and after the SinkWriters and Committers. Furthermore, we do not see
>>
>> it as a downside. The Sink interfaces that will expose the DataStream
>>
>> to the user reside in flink-streaming-java in contrast to the basic
>>
>> Sink interfaces that reside fin flink-core deem it to be only used by
>>
>> expert users.
>>
>>
>>
>> Moreover, we also wanted to remove the global committer from the
>>
>> unified Sink interfaces and replace it with a custom post-commit
>>
>> topology. Unfortunately, we cannot do it without breaking the Sink
>>
>> interface since the GlobalCommittables are part of the parameterized
>>
>> Sink interface. Thus, we propose building a new Sink V2 interface
>>
>> consisting of composable interfaces that do not offer the
>>
>> GlobalCommitter anymore. We will implement a utility to extend a Sink
>>
>> with post topology that mimics the behavior of the GlobalCommitter.
>>
>> The new Sink V2 provides the same sort of methods as the Sink V1
>>
>> interface, so a migration of sinks that do not use the GlobalCommitter
>>
>> should be very easy.
>>
>> We plan to keep the existing Sink V1 interfaces to not break
>>
>> externally built sinks. As part of this FLIP, we migrate all the
>>
>> connectors inside of the main repository to the new Sink V2 API.
>>
>>
>>
>> The FLIP document is also updated and includes the proposed changes.
>>
>>
>>
>> Looking forward to your feedback,
>>
>> Fabian
>>
>>
>>
>> https://cwiki.apache.org/confluence/display/FLINK/FLIP-191%3A+Extend+unified+Sink+interface+to+support+small+file+compaction
>>
>>
>>
>>
>>
>> On Thu, Dec 2, 2021 at 10:15 AM Roman Khachatryan wrote:
>>
>> >
>>
>> > Thanks for clarifying (I was initially confused by merging state files
>>
>> > rather than output files).
>>
>> >
>>
>> > > At some point, Flink will definitely have some WAL adapter that can turn any sink into an exactly-once sink (with some caveats). For now, we keep that as an orthogonal solution as it has a rather high price (bursty workload with high latency). Ideally, we can keep the compaction asynchronously...
>>
>> >
>>
>> > Yes, that would be something like a WAL. I agree that it would have a
>>
>> > different set of trade-offs.
>>
>> >
>>
>> >
>>
>> > Regards,
>>
>> > Roman
>>
>> >
>>
>> > On Mon, Nov 29, 2021 at 3:33 PM Arvid Heise wrote:
>>
>> > >>
>>
>> > >> > One way to avoid write-read-merge is by wrapping SinkWriter with
>>
>> > >> > another one, which would buffer input elements in a temporary storage
>>
>> > >> > (e.g. local file) until a threshold is reached; after that, it would
>>
>> > >> > invoke the original SinkWriter. And if a checkpoint barrier comes in
>>
>> > >> > earlier, it would send written data to some aggregator.
>>
>> > >>
>>
>> > >> I think perhaps this seems to be a kind of WAL method? Namely we first
>>
>> > >> write the elements to some WAL logs and persist them on checkpoint
>>
>> > >> (in snapshot or remote FS), or we directly write WAL logs to the remote
>>
>> > >> FS eagerly.
>>
>> > >>
>>
>> > > At some point, Flink will definitely have some WAL adapter that can turn any sink into an exactly-once sink (with some caveats). For now, we keep that as an orthogonal solution as it has a rather high price (bursty workload with high latency). Ideally, we can keep the compaction asynchronously...
>>
>> > >
>>
>> > > On Mon, Nov 29, 2021 at 8:52 AM Yun Gao wrote:
>>
>> > >>
>>
>> > >> Hi,
>>
>> > >>
>>
>> > >> @Roman very sorry for the late response for a long time,
>>
>> > >>
>>
>> > >> > Merging artifacts from multiple checkpoints would apparently
>>
>> > >> require multiple concurrent checkpoints
>>
>> > >>
>>
>> > >> I think it might not need concurrent checkpoints: suppose some
>>
>> > >> operators (like the committer aggregator in the option 2) maintains
>>
>> > >> the list of files to merge, it could stores the lists of files to merge
>>
>> > >> in the states, then after several checkpoints are done and we have
>>
>> > >> enough files, we could merge all the files in the list.
>>
>> > >>
>>
>> > >> > Asynchronous merging in an aggregator would require some resolution
>>
>> > >> > logic on recovery, so that a merged artifact can be used if the
>>
>> > >> > original one was deleted. Otherwise, wouldn't recovery fail because
>>
>> > >> > some artifacts are missing?
>>
>> > >> > We could also defer deletion until the "compacted" checkpoint is
>>
>> > >> > subsumed - but isn't it too late, as it will be deleted anyways once
>>
>> > >> > subsumed?
>>
>> > >>
>>
>> > >> I think logically we could delete the original files once the "compacted" checkpoint
>>
>> > >> (which finish merging the compacted files and record it in the checkpoint) is completed
>>
>> > >> in all the options. If there are failover before we it, we could restart the merging and if
>>
>> > >> there are failover after it, we could have already recorded the files in the checkpoint.
>>
>> > >>
>>
>> > >> > One way to avoid write-read-merge is by wrapping SinkWriter with
>>
>> > >> > another one, which would buffer input elements in a temporary storage
>>
>> > >> > (e.g. local file) until a threshold is reached; after that, it would
>>
>> > >> > invoke the original SinkWriter. And if a checkpoint barrier comes in
>>
>> > >> > earlier, it would send written data to some aggregator.
>>
>> > >>
>>
>> > >> I think perhaps this seems to be a kind of WAL method? Namely we first
>>
>> > >> write the elements to some WAL logs and persist them on checkpoint
>>
>> > >> (in snapshot or remote FS), or we directly write WAL logs to the remote
>>
>> > >> FS eagerly.
>>
>> > >>
>>
>> > >> Sorry if I do not understand correctly somewhere.
>>
>> > >>
>>
>> > >> Best,
>>
>> > >> Yun
>>
>> > >>
>>
>> > >>
>>
>> > >> ------------------------------------------------------------------
>>
>> > >> From:Roman Khachatryan
>>
>> > >> Send Time:2021 Nov. 9 (Tue.) 22:03
>>
>> > >> To:dev
>>
>> > >> Subject:Re: [DISCUSS] FLIP-191: Extend unified Sink interface to support small file compaction
>>
>> > >>
>>
>> > >> Hi everyone,
>>
>> > >>
>>
>> > >> Thanks for the proposal and the discussion, I have some remarks:
>>
>> > >> (I'm not very familiar with the new Sink API but I thought about the
>>
>> > >> same problem in context of the changelog state backend)
>>
>> > >>
>>
>> > >> 1. Merging artifacts from multiple checkpoints would apparently
>>
>> > >> require multiple concurrent checkpoints (otherwise, a new checkpoint
>>
>> > >> won't be started before completing the previous one; and the previous
>>
>> > >> one can't be completed before durably storing the artifacts). However,
>>
>> > >> concurrent checkpoints are currently not supported with Unaligned
>>
>> > >> checkpoints (this is besides increasing e2e-latency).
>>
>> > >>
>>
>> > >> 2. Asynchronous merging in an aggregator would require some resolution
>>
>> > >> logic on recovery, so that a merged artifact can be used if the
>>
>> > >> original one was deleted. Otherwise, wouldn't recovery fail because
>>
>> > >> some artifacts are missing?
>>
>> > >> We could also defer deletion until the "compacted" checkpoint is
>>
>> > >> subsumed - but isn't it too late, as it will be deleted anyways once
>>
>> > >> subsumed?
>>
>> > >>
>>
>> > >> 3. Writing small files, then reading and merging them for *every*
>>
>> > >> checkpoint seems worse than only reading them on recovery. I guess I'm
>>
>> > >> missing some cases of reading, so to me it would make sense to mention
>>
>> > >> these cases explicitly in the FLIP motivation section.
>>
>> > >>
>>
>> > >> 4. One way to avoid write-read-merge is by wrapping SinkWriter with
>>
>> > >> another one, which would buffer input elements in a temporary storage
>>
>> > >> (e.g. local file) until a threshold is reached; after that, it would
>>
>> > >> invoke the original SinkWriter. And if a checkpoint barrier comes in
>>
>> > >> earlier, it would send written data to some aggregator. It will
>>
>> > >> increase checkpoint delay (async phase) compared to the current Flink;
>>
>> > >> but not compared to the write-read-merge solution, IIUC.
>>
>> > >> Then such "BufferingSinkWriters" could aggregate input elements from
>>
>> > >> each other, potentially recursively (I mean something like
>>
>> > >> https://cwiki.apache.org/confluence/download/attachments/173082889/DSTL-DFS-DAG.png
>>
>> > >> )
>>
>> > >>
>>
>> > >> 5. Reducing the number of files by reducing aggregator parallelism as
>>
>> > >> opposed to merging on reaching size threshold will likely be less
>>
>> > >> optimal and more difficult to configure. OTH, thresholds might be more
>>
>> > >> difficult to implement and (with recursive merging) would incur higher
>>
>> > >> latency. Maybe that's also something to decide explicitly or at least
>>
>> > >> mention in the FLIP.
>>
>> > >>
>>
>> > >>
>>
>> > >>
>>
>> > >> Regards,
>>
>> > >> Roman
>>
>> > >>
>>
>> > >>
>>
>> > >> On Tue, Nov 9, 2021 at 5:23 AM Reo Lei wrote:
>>
>> > >> >
>>
>> > >> > Hi Fabian,
>>
>> > >> >
>>
>> > >> > Thanks for drafting the FLIP and trying to support small file compaction. I
>>
>> > >> > think this feature is very urgent and valuable for users(at least for me).
>>
>> > >> >
>>
>> > >> > Currently I am trying to support streaming rewrite(compact) for Iceberg on
>>
>> > >> > PR#3323 . As Steven mentioned,
>>
>> > >> > Iceberg sink and compact data through the following steps:
>>
>> > >> > Step-1: Some parallel data writer(sinker) to write streaming data as files.
>>
>> > >> > Step-2: A single parallelism data files committer to commit the completed
>>
>> > >> > files as soon as possible to make them available.
>>
>> > >> > Step-3: Some parallel file rewriter(compactor) to collect committed files
>>
>> > >> > from multiple checkpoints, and rewriter(compact) them together once the
>>
>> > >> > total file size or number of files reach the threshold.
>>
>> > >> > Step-4: A single parallelism rewrite(compact) result committer to commit
>>
>> > >> > the rewritten(compacted) files to replace the old files and make them
>>
>> > >> > available.
>>
>> > >> >
>>
>> > >> >
>>
>> > >> > If Flink want to support small file compaction, some key point I think is
>>
>> > >> > necessary:
>>
>> > >> >
>>
>> > >> > 1, Compact files from multiple checkpoints.
>>
>> > >> > I totally agree with Jingsong, because completed file size usually could
>>
>> > >> > not reach the threshold in a single checkpoint. Especially for partitioned
>>
>> > >> > table, we need to compact the files of each partition, but usually the file
>>
>> > >> > size of each partition will be different and may not reach the merge
>>
>> > >> > threshold. If we compact these files, in a single checkpoint, regardless of
>>
>> > >> > whether the total file size reaches the threshold, then the value of
>>
>> > >> > compacting will be diminished and we will still get small files because
>>
>> > >> > these compacted files are not reach to target size. So we need the
>>
>> > >> > compactor to collect committed files from multiple checkpoints and compact
>>
>> > >> > them until they reach the threshold.
>>
>> > >> >
>>
>> > >> > 2, Separate write phase and compact phase.
>>
>> > >> > Users usually hope the data becomes available as soon as possible, and the
>>
>> > >> > end-to-end latency is very important. I think we need to separate the
>>
>> > >> > write and compact phase. For the write phase, there include the Step-1
>>
>> > >> > and Step-2, we sink data as file and commit it pre checkpoint and regardless
>>
>> > >> > of whether the file size it is. That could ensure the data will be
>>
>> > >> > available ASAP. For the compact phase, there include the Step-3
>>
>> > >> > and Step-4, the compactor should collect committed files from multiple
>>
>> > >> > checkpoints and compact them asynchronously once they reach the threshold,
>>
>> > >> > and the compact committer will commit the compaction result in the next
>>
>> > >> > checkpoint. We compact the committed files asynchronously because we don't
>>
>> > >> > want the compaction to affect the data sink or the whole pipeline.
>>
>> > >> >
>>
>> > >> > 3, Exactly once guarantee between write and compact phase.
>>
>> > >> > Once we separate write phase and compact phase, we need to consider
>>
>> > >> > how to guarantee
>>
>> > >> > the exact once semantic between two phases. We should not lose any data or
>>
>> > >> > files on the compactor(Step-3) in any case and cause the compaction result
>>
>> > >> > to be inconsistent with before. I think flink should provide an easy-to-use
>>
>> > >> > interface to make that easier.
>>
>> > >> >
>>
>> > >> > 4, Metadata operation and compaction result validation.
>>
>> > >> > In the compact phase, there may be not only compact files, but also a lot
>>
>> > >> > of metadata operations, such as the iceberg needing to read/write manifest
>>
>> > >> > and do MOR. And we need some interface to support users to do some
>>
>> > >> > validation of the compaction result. I think these points should be
>>
>> > >> > considered when we design the compaction API.
>>
>> > >> >
>>
>> > >> >
>>
>> > >> > Back to FLIP-191, option 1 looks very complicated while option 2 is
>>
>> > >> > relatively simple, but neither of these two solutions separates the write
>>
>> > >> > phase from the compact phase. So I think we should consider the points I
>>
>> > >> > mentioned above. And if you have any other questions you can always feel
>>
>> > >> > free to reach out to me!
>>
>> > >> >
>>
>> > >> > BR,
>>
>> > >> > Reo
>>
>> > >> >
>>
>> > >> > Fabian Paul 于2021年11月8日周一 下午7:59写道:
>>
>> > >> >
>>
>> > >> > > Hi all,
>>
>> > >> > >
>>
>> > >> > > Thanks for the lively discussions. I am really excited to see so many
>>
>> > >> > > people
>>
>> > >> > > participating in this thread. It also underlines the need that many people
>>
>> > >> > > would
>>
>> > >> > > like to see a solution soon.
>>
>> > >> > >
>>
>> > >> > > I have updated the FLIP and removed the parallelism configuration because
>>
>> > >> > > it is
>>
>> > >> > > unnecessary since users can configure a constant exchange key to send all
>>
>> > >> > > committables to only one committable aggregator.
>>
>> > >> > >
>>
>> > >> > >
>>
>> > >> > > 1. Burden for developers w.r.t batch stream unification.
>>
>> > >> > >
>>
>> > >> > > @yun @guowei, from a theoretical point you are right about exposing the
>>
>> > >> > > DataStream
>>
>> > >> > > API in the sink users have the full power to write correct batch and
>>
>> > >> > > streaming
>>
>> > >> > > sinks. I think in reality a lot of users still struggle to build pipelines
>>
>> > >> > > with
>>
>> > >> > > i.e. the operator pipeline which works correct in streaming and batch mode.
>>
>> > >> > > Another problem I see is by exposing more deeper concepts is that we
>>
>> > >> > > cannot do
>>
>> > >> > > any optimization because we cannot reason about how sinks are built in the
>>
>> > >> > > future.
>>
>> > >> > >
>>
>> > >> > > We should also try to steer users towards using only `Functions` to give
>>
>> > >> > > us more
>>
>> > >> > > flexibility to swap the internal operator representation. I agree with
>>
>> > >> > > @yun we
>>
>> > >> > > should try to make the `ProcessFunction` more versatile to work on that
>>
>> > >> > > goal but
>>
>> > >> > > I see this as unrelated to the FLIP.
>>
>> > >> > >
>>
>> > >> > >
>>
>> > >> > > 2. Regarding Commit / Global commit
>>
>> > >> > >
>>
>> > >> > > I envision the global committer to be specific depending on the data lake
>>
>> > >> > > solution you want to write to. However, it is entirely orthogonal to the
>>
>> > >> > > compaction.
>>
>> > >> > > Currently, I do not expect any changes w.r.t the Global commit introduces
>>
>> > >> > > by
>>
>> > >> > > this FLIP.
>>
>> > >> > >
>>
>> > >> > >
>>
>> > >> > > 3. Regarding the case of trans-checkpoints merging
>>
>> > >> > >
>>
>> > >> > > @yun, as user, I would expect that if the committer receives in a
>>
>> > >> > > checkpoint files
>>
>> > >> > > to merge/commit that these are also finished when the checkpoint finishes.
>>
>> > >> > > I think all sinks rely on this principle currently i.e., KafkaSink needs to
>>
>> > >> > > commit all open transactions until the next checkpoint can happen.
>>
>> > >> > >
>>
>> > >> > > Maybe in the future, we can somehow move the Committer#commit call to an
>>
>> > >> > > asynchronous execution, but we should discuss it as a separate thread.
>>
>> > >> > >
>>
>> > >> > > > We probably should first describe the different causes of small files and
>>
>> > >> > > > what problems was this proposal trying to solve. I wrote a data shuffling
>>
>> > >> > > > proposal [1] for Flink Iceberg sink (shared with Iceberg community [2]).
>>
>> > >> > > It
>>
>> > >> > > > can address small files problems due to skewed data distribution across
>>
>> > >> > > > Iceberg table partitions. Streaming shuffling before writers (to files)
>>
>> > >> > > is
>>
>> > >> > > > typically more efficient than post-write file compaction (which involves
>>
>> > >> > > > read-merge-write). It is usually cheaper to prevent a problem (small
>>
>> > >> > > files)
>>
>> > >> > > > than fixing it.
>>
>> > >> > >
>>
>> > >> > >
>>
>> > >> > > @steven you are raising a good point, although I think only using a
>>
>> > >> > > customizable
>>
>> > >> > > shuffle won't address the generation of small files. One assumption is that
>>
>> > >> > > at least the sink generates one file per subtask, which can already be too
>>
>> > >> > > many.
>>
>> > >> > > Another problem is that with low checkpointing intervals, the files do not
>>
>> > >> > > meet
>>
>> > >> > > the required size. The latter point is probably addressable by changing the
>>
>> > >> > > checkpoint interval, which might be inconvenient for some users.
>>
>> > >> > >
>>
>> > >> > > > The sink coordinator checkpoint problem (mentioned in option 1) would be
>>
>> > >> > > > great if Flink can address it. In the spirit of source
>>
>> > >> > > (enumerator-reader)
>>
>> > >> > > > and sink (writer-coordinator) duality, sink coordinator checkpoint should
>>
>> > >> > > > happen after the writer operator. This would be a natural fit to support
>>
>> > >> > > > global committer in FLIP-143. It is probably an orthogonal matter to this
>>
>> > >> > > > proposal.
>>
>> > >> > >
>>
>> > >> > >
>>
>> > >> > > To me the question here is what are the benefits of having a coordinator in
>>
>> > >> > > comparison to a global committer/aggregator operator.
>>
>> > >> > >
>>
>> > >> > > > Personally, I am usually in favor of keeping streaming ingestion (to data
>>
>> > >> > > > lake) relatively simple and stable. Also sometimes compaction and sorting
>>
>> > >> > > > are performed together in data rewrite maintenance jobs to improve read
>>
>> > >> > > > performance. In that case, the value of compacting (in Flink streaming
>>
>> > >> > > > ingestion) diminishes.
>>
>> > >> > >
>>
>> > >> > >
>>
>> > >> > > I agree it is always possible to have scheduled maintenance jobs keeping
>>
>> > >> > > care of
>>
>> > >> > > your data i.e., doing compaction. Unfortunately, the downside is that you
>>
>> > >> > > have to your data after it is already available for other downstream
>>
>> > >> > > consumers.
>>
>> > >> > > I guess this can lead to all kinds of visibility problems. I am also
>>
>> > >> > > surprised that
>>
>> > >> > > you personally are a fan of this approach and, on the other hand, are
>>
>> > >> > > developing
>>
>> > >> > > the Iceberg sink, which goes somewhat against your mentioned principle of
>>
>> > >> > > keeping
>>
>> > >> > > the sink simple.
>>
>> > >> > >
>>
>> > >> > > > Currently, it is unclear from the doc and this thread where the
>>
>> > >> > > compaction
>>
>> > >> > > > is actually happening. Jingsong's reply described one model
>>
>> > >> > > > writer (parallel) -> aggregator (single-parallelism compaction planner)
>>
>> > >> > > ->
>>
>> > >> > > > compactor (parallel) -> global committer (single-parallelism)
>>
>> > >> > >
>>
>> > >> > >
>>
>> > >> > > My idea of the topology is very similar to the one outlined by Jinsong. The
>>
>> > >> > > compaction will happen in the committer operator.
>>
>> > >> > >
>>
>> > >> > > >
>>
>> > >> > > > In the Iceberg community, the following model has been discussed. It is
>>
>> > >> > > > better for Iceberg because it won't delay the data availability.
>>
>> > >> > > > writer (parallel) -> global committer for append (single parallelism) ->
>>
>> > >> > > > compactor (parallel) -> global committer for rewrite commit (single
>>
>> > >> > > > parallelism)
>>
>> > >> > >
>>
>> > >> > >
>>
>> > >> > > From a quick glimpse, it seems that the exact same topology is possible to
>>
>> > >> > > express with the committable aggregator, but this definitely depends on
>>
>> > >> > > the exact
>>
>> > >> > > setup.
>>
>> > >> > >
>>
>> > >> > > Best,
>>
>> > >> > > Fabian
>>
>> > >>

Re: Re: [DISCUSS] FLIP-191: Extend unified Sink interface to support small file compaction

Posted by Yun Gao <yu...@aliyun.com.INVALID>.
Hi Fabian,

Very thanks for the update! I think the latest version in general looks good from my side
and I think using separate feature interface would be much more easy to understand 
and extend in the future. I have some pending issues on the details though:

1. The first one is if we could support end-to-end exactly-once with post-committing
topology in the batch mode ? Since for the batch mode, currently we could only commit
 all the transactions after the whole job is finished, otherwise if there are JM failover or the 
writer / committer get restarted due to indeterminstic (A -> [B1, B2], A, B1 have finished and
 B2 failed, if -> is rebalance / random / rescale,  all of A, B1, B2 would restarted), there might 
be repeat records. Previously one possible thought is to move committer and global committer
 to the operator coordinator, but if it is a topology, we might need some other kind of solutions?

2. I also want to have a dobule confirmation with the compatibility: since the old sink is also named 
with Sink, do we want to put the Sink v2 in a new package ? Besides, since we might want to keep 
only have one `sinkTo(Sink<?> sink)` , perhaps we also need to make the Sink v1 to be a subclass of 
Sink v2 and extends the stateful and two-phase-commit sinks, right? 

3. I'd like also have a confirmation on ours thoughts with the `DataStreamSink` returned by the sinkTo method: 
The main issue is how do we implement the method like `setParallelism` or `setMaxParallelism` since now the sink 
would be translated to multiple transformations? perhaps we could make it the default values for all the transformations 
for the sink? A related issue would be for iceberg sink, I think it would need to have only one committer to avoid the 
competition of the optimistic locks (which would cause performance degradation), then it might need to have N writers 
with 1 committers, to build such topology, perhaps we might need to add new methods to specify the parallelism of 
the writers and committers separately? 

Best,
Yun



 ------------------Original Mail ------------------
Sender:Fabian Paul <fp...@apache.org>
Send Date:Mon Dec 13 23:59:43 2021
Recipients:dev <de...@flink.apache.org>
Subject:Re: [DISCUSS] FLIP-191: Extend unified Sink interface to support small file compaction
Hi all,



After a lot of discussions with different, we received very fruitful

feedback and reworked the ideas behind this FLIP. Initially, we had

the impression that the compaction problem is solvable by a single

topology that we can reuse across different sinks. We now have a

better understanding that different external systems require different

compaction mechanism i.e. Hive requires compaction before finally

registering the file in the metastore or Iceberg compacts the files

after they have been registered and just lazily compacts them.



Considering all these different views we came up with a design that

builds upon what @guowei.mgw@gmail.com and @yungao.gy@aliyun.com have

proposed at the beginning. We allow inserting custom topologies before

and after the SinkWriters and Committers. Furthermore, we do not see

it as a downside. The Sink interfaces that will expose the DataStream

to the user reside in flink-streaming-java in contrast to the basic

Sink interfaces that reside fin flink-core deem it to be only used by

expert users.



Moreover, we also wanted to remove the global committer from the

unified Sink interfaces and replace it with a custom post-commit

topology. Unfortunately, we cannot do it without breaking the Sink

interface since the GlobalCommittables are part of the parameterized

Sink interface. Thus, we propose building a new Sink V2 interface

consisting of composable interfaces that do not offer the

GlobalCommitter anymore. We will implement a utility to extend a Sink

with post topology that mimics the behavior of the GlobalCommitter.

The new Sink V2 provides the same sort of methods as the Sink V1

interface, so a migration of sinks that do not use the GlobalCommitter

should be very easy.

We plan to keep the existing Sink V1 interfaces to not break

externally built sinks. As part of this FLIP, we migrate all the

connectors inside of the main repository to the new Sink V2 API.



The FLIP document is also updated and includes the proposed changes.



Looking forward to your feedback,

Fabian



https://cwiki.apache.org/confluence/display/FLINK/FLIP-191%3A+Extend+unified+Sink+interface+to+support+small+file+compaction





On Thu, Dec 2, 2021 at 10:15 AM Roman Khachatryan  wrote:

>

> Thanks for clarifying (I was initially confused by merging state files

> rather than output files).

>

> > At some point, Flink will definitely have some WAL adapter that can turn any sink into an exactly-once sink (with some caveats). For now, we keep that as an orthogonal solution as it has a rather high price (bursty workload with high latency). Ideally, we can keep the compaction asynchronously...

>

> Yes, that would be something like a WAL. I agree that it would have a

> different set of trade-offs.

>

>

> Regards,

> Roman

>

> On Mon, Nov 29, 2021 at 3:33 PM Arvid Heise  wrote:

> >>

> >> > One way to avoid write-read-merge is by wrapping SinkWriter with

> >> > another one, which would buffer input elements in a temporary storage

> >> > (e.g. local file) until a threshold is reached; after that, it would

> >> > invoke the original SinkWriter. And if a checkpoint barrier comes in

> >> > earlier, it would send written data to some aggregator.

> >>

> >> I think perhaps this seems to be a kind of WAL method? Namely we first

> >> write the elements to some WAL logs and persist them on checkpoint

> >> (in snapshot or remote FS), or we directly write WAL logs to the remote

> >> FS eagerly.

> >>

> > At some point, Flink will definitely have some WAL adapter that can turn any sink into an exactly-once sink (with some caveats). For now, we keep that as an orthogonal solution as it has a rather high price (bursty workload with high latency). Ideally, we can keep the compaction asynchronously...

> >

> > On Mon, Nov 29, 2021 at 8:52 AM Yun Gao  wrote:

> >>

> >> Hi,

> >>

> >> @Roman very sorry for the late response for a long time,

> >>

> >> > Merging artifacts from multiple checkpoints would apparently

> >> require multiple concurrent checkpoints

> >>

> >> I think it might not need concurrent checkpoints: suppose some

> >> operators (like the committer aggregator in the option 2) maintains

> >> the list of files to merge, it could stores the lists of files to merge

> >> in the states, then after several checkpoints are done and we have

> >> enough files, we could merge all the files in the list.

> >>

> >> > Asynchronous merging in an aggregator would require some resolution

> >> > logic on recovery, so that a merged artifact can be used if the

> >> > original one was deleted. Otherwise, wouldn't recovery fail because

> >> > some artifacts are missing?

> >> > We could also defer deletion until the "compacted" checkpoint is

> >> > subsumed - but isn't it too late, as it will be deleted anyways once

> >> > subsumed?

> >>

> >> I think logically we could delete the original files once the "compacted" checkpoint

> >> (which finish merging the compacted files and record it in the checkpoint) is completed

> >> in all the options. If there are failover before we it, we could restart the merging and if

> >> there are failover after it, we could have already recorded the files in the checkpoint.

> >>

> >> > One way to avoid write-read-merge is by wrapping SinkWriter with

> >> > another one, which would buffer input elements in a temporary storage

> >> > (e.g. local file) until a threshold is reached; after that, it would

> >> > invoke the original SinkWriter. And if a checkpoint barrier comes in

> >> > earlier, it would send written data to some aggregator.

> >>

> >> I think perhaps this seems to be a kind of WAL method? Namely we first

> >> write the elements to some WAL logs and persist them on checkpoint

> >> (in snapshot or remote FS), or we directly write WAL logs to the remote

> >> FS eagerly.

> >>

> >> Sorry if I do not understand correctly somewhere.

> >>

> >> Best,

> >> Yun

> >>

> >>

> >> ------------------------------------------------------------------

> >> From:Roman Khachatryan 

> >> Send Time:2021 Nov. 9 (Tue.) 22:03

> >> To:dev 

> >> Subject:Re: [DISCUSS] FLIP-191: Extend unified Sink interface to support small file compaction

> >>

> >> Hi everyone,

> >>

> >> Thanks for the proposal and the discussion, I have some remarks:

> >> (I'm not very familiar with the new Sink API but I thought about the

> >> same problem in context of the changelog state backend)

> >>

> >> 1. Merging artifacts from multiple checkpoints would apparently

> >> require multiple concurrent checkpoints (otherwise, a new checkpoint

> >> won't be started before completing the previous one; and the previous

> >> one can't be completed before durably storing the artifacts). However,

> >> concurrent checkpoints are currently not supported with Unaligned

> >> checkpoints (this is besides increasing e2e-latency).

> >>

> >> 2. Asynchronous merging in an aggregator would require some resolution

> >> logic on recovery, so that a merged artifact can be used if the

> >> original one was deleted. Otherwise, wouldn't recovery fail because

> >> some artifacts are missing?

> >> We could also defer deletion until the "compacted" checkpoint is

> >> subsumed - but isn't it too late, as it will be deleted anyways once

> >> subsumed?

> >>

> >> 3. Writing small files, then reading and merging them for *every*

> >> checkpoint seems worse than only reading them on recovery. I guess I'm

> >> missing some cases of reading, so to me it would make sense to mention

> >> these cases explicitly in the FLIP motivation section.

> >>

> >> 4. One way to avoid write-read-merge is by wrapping SinkWriter with

> >> another one, which would buffer input elements in a temporary storage

> >> (e.g. local file) until a threshold is reached; after that, it would

> >> invoke the original SinkWriter. And if a checkpoint barrier comes in

> >> earlier, it would send written data to some aggregator. It will

> >> increase checkpoint delay (async phase) compared to the current Flink;

> >> but not compared to the write-read-merge solution, IIUC.

> >> Then such "BufferingSinkWriters" could aggregate input elements from

> >> each other, potentially recursively (I mean something like

> >> https://cwiki.apache.org/confluence/download/attachments/173082889/DSTL-DFS-DAG.png

> >> )

> >>

> >> 5. Reducing the number of files by reducing aggregator parallelism as

> >> opposed to merging on reaching size threshold will likely be less

> >> optimal and more difficult to configure. OTH, thresholds might be more

> >> difficult to implement and (with recursive merging) would incur higher

> >> latency. Maybe that's also something to decide explicitly or at least

> >> mention in the FLIP.

> >>

> >>

> >>

> >> Regards,

> >> Roman

> >>

> >>

> >> On Tue, Nov 9, 2021 at 5:23 AM Reo Lei  wrote:

> >> >

> >> > Hi Fabian,

> >> >

> >> > Thanks for drafting the FLIP and trying to support small file compaction. I

> >> > think this feature is very urgent and valuable for users(at least for me).

> >> >

> >> > Currently I am trying to support streaming rewrite(compact) for Iceberg on

> >> > PR#3323 . As Steven mentioned,

> >> > Iceberg sink and compact data through the following steps:

> >> > Step-1: Some parallel data writer(sinker) to write streaming data as files.

> >> > Step-2: A single parallelism data files committer to commit the completed

> >> > files as soon as possible to make them available.

> >> > Step-3: Some parallel file rewriter(compactor) to collect committed files

> >> > from multiple checkpoints, and rewriter(compact) them together once the

> >> > total file size or number of files reach the threshold.

> >> > Step-4: A single parallelism rewrite(compact) result committer to commit

> >> > the rewritten(compacted) files to replace the old files and make them

> >> > available.

> >> >

> >> >

> >> > If Flink want to support small file compaction, some key point I think is

> >> > necessary:

> >> >

> >> > 1, Compact files from multiple checkpoints.

> >> > I totally agree with Jingsong, because completed file size usually could

> >> > not reach the threshold in a single checkpoint. Especially for partitioned

> >> > table, we need to compact the files of each partition, but usually the file

> >> > size of each partition will be different and may not reach the merge

> >> > threshold. If we compact these files, in a single checkpoint, regardless of

> >> > whether the total file size reaches the threshold, then the value of

> >> > compacting will be diminished and we will still get small files because

> >> > these compacted files are not reach to target size. So we need the

> >> > compactor to collect committed files from multiple checkpoints and compact

> >> > them until they reach the threshold.

> >> >

> >> > 2, Separate write phase and compact phase.

> >> > Users usually hope the data becomes available as soon as possible, and the

> >> > end-to-end latency is very important. I think we need to separate the

> >> > write and compact phase. For the write phase, there include the Step-1

> >> > and Step-2, we sink data as file and commit it pre checkpoint and regardless

> >> > of whether the file size it is. That could ensure the data will be

> >> > available ASAP. For the compact phase, there include the Step-3

> >> > and Step-4, the compactor should collect committed files from multiple

> >> > checkpoints and compact them asynchronously once they reach the threshold,

> >> > and the compact committer will commit the compaction result in the next

> >> > checkpoint. We compact the committed files asynchronously because we don't

> >> > want the compaction to affect the data sink or the whole pipeline.

> >> >

> >> > 3, Exactly once guarantee between write and compact phase.

> >> > Once we separate write phase and compact phase, we need to consider

> >> > how to guarantee

> >> > the exact once semantic between two phases. We should not lose any data or

> >> > files on the compactor(Step-3) in any case and cause the compaction result

> >> > to be inconsistent with before. I think flink should provide an easy-to-use

> >> > interface to make that easier.

> >> >

> >> > 4, Metadata operation and compaction result validation.

> >> > In the compact phase, there may be not only compact files, but also a lot

> >> > of metadata operations, such as the iceberg needing to read/write manifest

> >> > and do MOR. And we need some interface to support users to do some

> >> > validation of the compaction result. I think these points should be

> >> > considered when we design the compaction API.

> >> >

> >> >

> >> > Back to FLIP-191, option 1 looks very complicated while option 2 is

> >> > relatively simple, but neither of these two solutions separates the write

> >> > phase from the compact phase. So I think we should consider the points I

> >> > mentioned above. And if you have any other questions you can always feel

> >> > free to reach out to me!

> >> >

> >> > BR,

> >> > Reo

> >> >

> >> > Fabian Paul  于2021年11月8日周一 下午7:59写道:

> >> >

> >> > > Hi all,

> >> > >

> >> > > Thanks for the lively discussions. I am really excited to see so many

> >> > > people

> >> > > participating in this thread. It also underlines the need that many people

> >> > > would

> >> > > like to see a solution soon.

> >> > >

> >> > > I have updated the FLIP and removed the parallelism configuration because

> >> > > it is

> >> > > unnecessary since users can configure a constant exchange key to send all

> >> > > committables to only one committable aggregator.

> >> > >

> >> > >

> >> > > 1. Burden for developers w.r.t batch stream unification.

> >> > >

> >> > > @yun @guowei, from a theoretical point you are right about exposing the

> >> > > DataStream

> >> > > API in the sink users have the full power to write correct batch and

> >> > > streaming

> >> > > sinks. I think in reality a lot of users still struggle to build pipelines

> >> > > with

> >> > > i.e. the operator pipeline which works correct in streaming and batch mode.

> >> > > Another problem I see is by exposing more deeper concepts is that we

> >> > > cannot do

> >> > > any optimization because we cannot reason about how sinks are built in the

> >> > > future.

> >> > >

> >> > > We should also try to steer users towards using only `Functions` to give

> >> > > us more

> >> > > flexibility to swap the internal operator representation. I agree with

> >> > > @yun we

> >> > > should try to make the `ProcessFunction` more versatile to work on that

> >> > > goal but

> >> > > I see this as unrelated to the FLIP.

> >> > >

> >> > >

> >> > > 2. Regarding Commit / Global commit

> >> > >

> >> > > I envision the global committer to be specific depending on the data lake

> >> > > solution you want to write to. However, it is entirely orthogonal to the

> >> > > compaction.

> >> > > Currently, I do not expect any changes w.r.t the Global commit introduces

> >> > > by

> >> > > this FLIP.

> >> > >

> >> > >

> >> > > 3. Regarding the case of trans-checkpoints merging

> >> > >

> >> > > @yun, as user, I would expect that if the committer receives in a

> >> > > checkpoint files

> >> > > to merge/commit that these are also finished when the checkpoint finishes.

> >> > > I think all sinks rely on this principle currently i.e., KafkaSink needs to

> >> > > commit all open transactions until the next checkpoint can happen.

> >> > >

> >> > > Maybe in the future, we can somehow move the Committer#commit call to an

> >> > > asynchronous execution, but we should discuss it as a separate thread.

> >> > >

> >> > > > We probably should first describe the different causes of small files and

> >> > > > what problems was this proposal trying to solve. I wrote a data shuffling

> >> > > > proposal [1] for Flink Iceberg sink (shared with Iceberg community [2]).

> >> > > It

> >> > > > can address small files problems due to skewed data distribution across

> >> > > > Iceberg table partitions. Streaming shuffling before writers (to files)

> >> > > is

> >> > > > typically more efficient than post-write file compaction (which involves

> >> > > > read-merge-write). It is usually cheaper to prevent a problem (small

> >> > > files)

> >> > > > than fixing it.

> >> > >

> >> > >

> >> > > @steven you are raising a good point, although I think only using a

> >> > > customizable

> >> > > shuffle won't address the generation of small files. One assumption is that

> >> > > at least the sink generates one file per subtask, which can already be too

> >> > > many.

> >> > > Another problem is that with low checkpointing intervals, the files do not

> >> > > meet

> >> > > the required size. The latter point is probably addressable by changing the

> >> > > checkpoint interval, which might be inconvenient for some users.

> >> > >

> >> > > > The sink coordinator checkpoint problem (mentioned in option 1) would be

> >> > > > great if Flink can address it. In the spirit of source

> >> > > (enumerator-reader)

> >> > > > and sink (writer-coordinator) duality, sink coordinator checkpoint should

> >> > > > happen after the writer operator. This would be a natural fit to support

> >> > > > global committer in FLIP-143. It is probably an orthogonal matter to this

> >> > > > proposal.

> >> > >

> >> > >

> >> > > To me the question here is what are the benefits of having a coordinator in

> >> > > comparison to a global committer/aggregator operator.

> >> > >

> >> > > > Personally, I am usually in favor of keeping streaming ingestion (to data

> >> > > > lake) relatively simple and stable. Also sometimes compaction and sorting

> >> > > > are performed together in data rewrite maintenance jobs to improve read

> >> > > > performance. In that case, the value of compacting (in Flink streaming

> >> > > > ingestion) diminishes.

> >> > >

> >> > >

> >> > > I agree it is always possible to have scheduled maintenance jobs keeping

> >> > > care of

> >> > > your data i.e., doing compaction. Unfortunately, the downside is that you

> >> > > have to your data after it is already available for other downstream

> >> > > consumers.

> >> > > I guess this can lead to all kinds of visibility problems. I am also

> >> > > surprised that

> >> > > you personally are a fan of this approach and, on the other hand, are

> >> > > developing

> >> > > the Iceberg sink, which goes somewhat against your mentioned principle of

> >> > > keeping

> >> > > the sink simple.

> >> > >

> >> > > > Currently, it is unclear from the doc and this thread where the

> >> > > compaction

> >> > > > is actually happening. Jingsong's reply described one model

> >> > > > writer (parallel) -> aggregator (single-parallelism compaction planner)

> >> > > ->

> >> > > > compactor (parallel) -> global committer (single-parallelism)

> >> > >

> >> > >

> >> > > My idea of the topology is very similar to the one outlined by Jinsong. The

> >> > > compaction will happen in the committer operator.

> >> > >

> >> > > >

> >> > > > In the Iceberg community, the following model has been discussed. It is

> >> > > > better for Iceberg because it won't delay the data availability.

> >> > > > writer (parallel) -> global committer for append (single parallelism) ->

> >> > > > compactor (parallel) -> global committer for rewrite commit (single

> >> > > > parallelism)

> >> > >

> >> > >

> >> > > From a quick glimpse, it seems that the exact same topology is possible to

> >> > > express with the committable aggregator, but this definitely depends on

> >> > > the exact

> >> > > setup.

> >> > >

> >> > > Best,

> >> > > Fabian

> >>

Re: [DISCUSS] FLIP-191: Extend unified Sink interface to support small file compaction

Posted by Fabian Paul <fp...@apache.org>.
Hi all,

After a lot of discussions with different, we received very fruitful
feedback and reworked the ideas behind this FLIP. Initially, we had
the impression that the compaction problem is solvable by a single
topology that we can reuse across different sinks. We now have a
better understanding that different external systems require different
compaction mechanism i.e. Hive requires compaction before finally
registering the file in the metastore or Iceberg compacts the files
after they have been registered and just lazily compacts them.

Considering all these different views we came up with a design that
builds upon what @guowei.mgw@gmail.com and @yungao.gy@aliyun.com have
proposed at the beginning. We allow inserting custom topologies before
and after the SinkWriters and Committers. Furthermore, we do not see
it as a downside. The Sink interfaces that will expose the DataStream
to the user reside in flink-streaming-java in contrast to the basic
Sink interfaces that reside fin flink-core deem it to be only used by
expert users.

Moreover, we also wanted to remove the global committer from the
unified Sink interfaces and replace it with a custom post-commit
topology. Unfortunately, we cannot do it without breaking the Sink
interface since the GlobalCommittables are part of the parameterized
Sink interface. Thus, we propose building a new Sink V2 interface
consisting of composable interfaces that do not offer the
GlobalCommitter anymore. We will implement a utility to extend a Sink
with post topology that mimics the behavior of the GlobalCommitter.
The new Sink V2 provides the same sort of methods as the Sink V1
interface, so a migration of sinks that do not use the GlobalCommitter
should be very easy.
We plan to keep the existing Sink V1 interfaces to not break
externally built sinks. As part of this FLIP, we migrate all the
connectors inside of the main repository to the new Sink V2 API.

The FLIP document is also updated and includes the proposed changes.

Looking forward to your feedback,
Fabian

https://cwiki.apache.org/confluence/display/FLINK/FLIP-191%3A+Extend+unified+Sink+interface+to+support+small+file+compaction


On Thu, Dec 2, 2021 at 10:15 AM Roman Khachatryan <ro...@apache.org> wrote:
>
> Thanks for clarifying (I was initially confused by merging state files
> rather than output files).
>
> > At some point, Flink will definitely have some WAL adapter that can turn any sink into an exactly-once sink (with some caveats). For now, we keep that as an orthogonal solution as it has a rather high price (bursty workload with high latency). Ideally, we can keep the compaction asynchronously...
>
> Yes, that would be something like a WAL. I agree that it would have a
> different set of trade-offs.
>
>
> Regards,
> Roman
>
> On Mon, Nov 29, 2021 at 3:33 PM Arvid Heise <ar...@apache.org> wrote:
> >>
> >> > One way to avoid write-read-merge is by wrapping SinkWriter with
> >> > another one, which would buffer input elements in a temporary storage
> >> > (e.g. local file) until a threshold is reached; after that, it would
> >> > invoke the original SinkWriter. And if a checkpoint barrier comes in
> >> > earlier, it would send written data to some aggregator.
> >>
> >> I think perhaps this seems to be a kind of WAL method? Namely we first
> >> write the elements to some WAL logs and persist them on checkpoint
> >> (in snapshot or remote FS), or we directly write WAL logs to the remote
> >> FS eagerly.
> >>
> > At some point, Flink will definitely have some WAL adapter that can turn any sink into an exactly-once sink (with some caveats). For now, we keep that as an orthogonal solution as it has a rather high price (bursty workload with high latency). Ideally, we can keep the compaction asynchronously...
> >
> > On Mon, Nov 29, 2021 at 8:52 AM Yun Gao <yu...@aliyun.com.invalid> wrote:
> >>
> >> Hi,
> >>
> >> @Roman very sorry for the late response for a long time,
> >>
> >> > Merging artifacts from multiple checkpoints would apparently
> >> require multiple concurrent checkpoints
> >>
> >> I think it might not need concurrent checkpoints: suppose some
> >> operators (like the committer aggregator in the option 2) maintains
> >> the list of files to merge, it could stores the lists of files to merge
> >> in the states, then after several checkpoints are done and we have
> >> enough files, we could merge all the files in the list.
> >>
> >> > Asynchronous merging in an aggregator would require some resolution
> >> > logic on recovery, so that a merged artifact can be used if the
> >> > original one was deleted. Otherwise, wouldn't recovery fail because
> >> > some artifacts are missing?
> >> > We could also defer deletion until the "compacted" checkpoint is
> >> > subsumed - but isn't it too late, as it will be deleted anyways once
> >> > subsumed?
> >>
> >> I think logically we could delete the original files once the "compacted" checkpoint
> >> (which finish merging the compacted files and record it in the checkpoint) is completed
> >> in all the options. If there are failover before we it, we could restart the merging and if
> >> there are failover after it, we could have already recorded the files in the checkpoint.
> >>
> >> > One way to avoid write-read-merge is by wrapping SinkWriter with
> >> > another one, which would buffer input elements in a temporary storage
> >> > (e.g. local file) until a threshold is reached; after that, it would
> >> > invoke the original SinkWriter. And if a checkpoint barrier comes in
> >> > earlier, it would send written data to some aggregator.
> >>
> >> I think perhaps this seems to be a kind of WAL method? Namely we first
> >> write the elements to some WAL logs and persist them on checkpoint
> >> (in snapshot or remote FS), or we directly write WAL logs to the remote
> >> FS eagerly.
> >>
> >> Sorry if I do not understand correctly somewhere.
> >>
> >> Best,
> >> Yun
> >>
> >>
> >> ------------------------------------------------------------------
> >> From:Roman Khachatryan <ro...@apache.org>
> >> Send Time:2021 Nov. 9 (Tue.) 22:03
> >> To:dev <de...@flink.apache.org>
> >> Subject:Re: [DISCUSS] FLIP-191: Extend unified Sink interface to support small file compaction
> >>
> >> Hi everyone,
> >>
> >> Thanks for the proposal and the discussion, I have some remarks:
> >> (I'm not very familiar with the new Sink API but I thought about the
> >> same problem in context of the changelog state backend)
> >>
> >> 1. Merging artifacts from multiple checkpoints would apparently
> >> require multiple concurrent checkpoints (otherwise, a new checkpoint
> >> won't be started before completing the previous one; and the previous
> >> one can't be completed before durably storing the artifacts). However,
> >> concurrent checkpoints are currently not supported with Unaligned
> >> checkpoints (this is besides increasing e2e-latency).
> >>
> >> 2. Asynchronous merging in an aggregator would require some resolution
> >> logic on recovery, so that a merged artifact can be used if the
> >> original one was deleted. Otherwise, wouldn't recovery fail because
> >> some artifacts are missing?
> >> We could also defer deletion until the "compacted" checkpoint is
> >> subsumed - but isn't it too late, as it will be deleted anyways once
> >> subsumed?
> >>
> >> 3. Writing small files, then reading and merging them for *every*
> >> checkpoint seems worse than only reading them on recovery. I guess I'm
> >> missing some cases of reading, so to me it would make sense to mention
> >> these cases explicitly in the FLIP motivation section.
> >>
> >> 4. One way to avoid write-read-merge is by wrapping SinkWriter with
> >> another one, which would buffer input elements in a temporary storage
> >> (e.g. local file) until a threshold is reached; after that, it would
> >> invoke the original SinkWriter. And if a checkpoint barrier comes in
> >> earlier, it would send written data to some aggregator. It will
> >> increase checkpoint delay (async phase) compared to the current Flink;
> >> but not compared to the write-read-merge solution, IIUC.
> >> Then such "BufferingSinkWriters" could aggregate input elements from
> >> each other, potentially recursively (I mean something like
> >> https://cwiki.apache.org/confluence/download/attachments/173082889/DSTL-DFS-DAG.png
> >> )
> >>
> >> 5. Reducing the number of files by reducing aggregator parallelism as
> >> opposed to merging on reaching size threshold will likely be less
> >> optimal and more difficult to configure. OTH, thresholds might be more
> >> difficult to implement and (with recursive merging) would incur higher
> >> latency. Maybe that's also something to decide explicitly or at least
> >> mention in the FLIP.
> >>
> >>
> >>
> >> Regards,
> >> Roman
> >>
> >>
> >> On Tue, Nov 9, 2021 at 5:23 AM Reo Lei <le...@gmail.com> wrote:
> >> >
> >> > Hi Fabian,
> >> >
> >> > Thanks for drafting the FLIP and trying to support small file compaction. I
> >> > think this feature is very urgent and valuable for users(at least for me).
> >> >
> >> > Currently I am trying to support streaming rewrite(compact) for Iceberg on
> >> > PR#3323 <https://github.com/apache/iceberg/pull/3323>. As Steven mentioned,
> >> > Iceberg sink and compact data through the following steps:
> >> > Step-1: Some parallel data writer(sinker) to write streaming data as files.
> >> > Step-2: A single parallelism data files committer to commit the completed
> >> > files as soon as possible to make them available.
> >> > Step-3: Some parallel file rewriter(compactor) to collect committed files
> >> > from multiple checkpoints, and rewriter(compact) them together once the
> >> > total file size or number of files reach the threshold.
> >> > Step-4: A single parallelism rewrite(compact) result committer to commit
> >> > the rewritten(compacted) files to replace the old files and make them
> >> > available.
> >> >
> >> >
> >> > If Flink want to support small file compaction, some key point I think is
> >> > necessary:
> >> >
> >> > 1, Compact files from multiple checkpoints.
> >> > I totally agree with Jingsong, because completed file size usually could
> >> > not reach the threshold in a single checkpoint. Especially for partitioned
> >> > table, we need to compact the files of each partition, but usually the file
> >> > size of each partition will be different and may not reach the merge
> >> > threshold. If we compact these files, in a single checkpoint, regardless of
> >> > whether the total file size reaches the threshold, then the value of
> >> > compacting will be diminished and we will still get small files because
> >> > these compacted files are not reach to target size. So we need the
> >> > compactor to collect committed files from multiple checkpoints and compact
> >> > them until they reach the threshold.
> >> >
> >> > 2, Separate write phase and compact phase.
> >> > Users usually hope the data becomes available as soon as possible, and the
> >> >  end-to-end latency is very important. I think we need to separate the
> >> > write and compact phase. For the write phase, there include the Step-1
> >> > and Step-2, we sink data as file and commit it pre checkpoint and regardless
> >> > of whether the file size it is. That could ensure the data will be
> >> > available ASAP. For the compact phase, there include the Step-3
> >> > and Step-4,  the compactor should collect committed files from multiple
> >> > checkpoints and compact them asynchronously once they reach the threshold,
> >> > and the compact committer will commit the  compaction result in the next
> >> > checkpoint. We compact the committed files asynchronously because we don't
> >> > want the compaction to affect the data sink or the whole pipeline.
> >> >
> >> > 3, Exactly once guarantee between write and compact phase.
> >> > Once we separate write phase and compact phase, we need to consider
> >> > how to guarantee
> >> > the exact once semantic between two phases. We should not lose any data or
> >> > files on the compactor(Step-3) in any case and cause the compaction result
> >> > to be inconsistent with before. I think flink should provide an easy-to-use
> >> > interface to make that easier.
> >> >
> >> > 4, Metadata operation and  compaction result validation.
> >> > In the compact phase, there may be not only compact files, but also a lot
> >> > of metadata operations, such as the iceberg needing to read/write manifest
> >> > and do MOR. And we need some interface to support users to do some
> >> > validation of the compaction result. I think these points should be
> >> > considered when we design the compaction API.
> >> >
> >> >
> >> > Back to FLIP-191, option 1 looks very complicated while option 2 is
> >> > relatively simple, but neither of these two solutions separates the write
> >> > phase from the compact phase. So I think we should consider the points I
> >> > mentioned above. And if you have any other questions you can always feel
> >> > free to reach out to me!
> >> >
> >> > BR,
> >> > Reo
> >> >
> >> > Fabian Paul <fa...@ververica.com> 于2021年11月8日周一 下午7:59写道:
> >> >
> >> > > Hi all,
> >> > >
> >> > > Thanks for the lively discussions. I am really excited to see so many
> >> > > people
> >> > > participating in this thread. It also underlines the need that many people
> >> > > would
> >> > > like to see a solution soon.
> >> > >
> >> > > I have updated the FLIP and removed the parallelism configuration because
> >> > > it is
> >> > > unnecessary since users can configure a constant exchange key to send all
> >> > > committables to only one committable aggregator.
> >> > >
> >> > >
> >> > > 1. Burden for developers w.r.t batch stream unification.
> >> > >
> >> > > @yun @guowei, from a theoretical point you are right about exposing the
> >> > > DataStream
> >> > > API in the sink users have the full power to write correct batch and
> >> > > streaming
> >> > > sinks. I think in reality a lot of users still struggle to build pipelines
> >> > > with
> >> > > i.e. the operator pipeline which works correct in streaming and batch mode.
> >> > > Another problem I see is by exposing more deeper concepts is that we
> >> > > cannot do
> >> > > any optimization because we cannot reason about how sinks are built in the
> >> > > future.
> >> > >
> >> > > We should also try to steer users towards using only `Functions` to give
> >> > > us more
> >> > > flexibility to swap the internal operator representation. I agree with
> >> > > @yun we
> >> > > should try to make the `ProcessFunction` more versatile to work on that
> >> > > goal but
> >> > > I see this as unrelated to the FLIP.
> >> > >
> >> > >
> >> > > 2. Regarding Commit / Global commit
> >> > >
> >> > > I envision the global committer to be specific depending on the data lake
> >> > > solution you want to write to. However, it is entirely orthogonal to the
> >> > > compaction.
> >> > > Currently, I do not expect any changes w.r.t the Global commit introduces
> >> > > by
> >> > > this FLIP.
> >> > >
> >> > >
> >> > > 3. Regarding the case of trans-checkpoints merging
> >> > >
> >> > > @yun, as user, I would expect that if the committer receives in a
> >> > > checkpoint files
> >> > > to merge/commit that these are also finished when the checkpoint finishes.
> >> > > I think all sinks rely on this principle currently i.e., KafkaSink needs to
> >> > > commit all open transactions until the next checkpoint can happen.
> >> > >
> >> > > Maybe in the future, we can somehow move the Committer#commit call to an
> >> > > asynchronous execution, but we should discuss it as a separate thread.
> >> > >
> >> > > > We probably should first describe the different causes of small files and
> >> > > > what problems was this proposal trying to solve. I wrote a data shuffling
> >> > > > proposal [1] for Flink Iceberg sink (shared with Iceberg community [2]).
> >> > > It
> >> > > > can address small files problems due to skewed data distribution across
> >> > > > Iceberg table partitions. Streaming shuffling before writers (to files)
> >> > > is
> >> > > > typically more efficient than post-write file compaction (which involves
> >> > > > read-merge-write). It is usually cheaper to prevent a problem (small
> >> > > files)
> >> > > > than fixing it.
> >> > >
> >> > >
> >> > > @steven you are raising a good point, although I think only using a
> >> > > customizable
> >> > > shuffle won't address the generation of small files. One assumption is that
> >> > > at least the sink generates one file per subtask, which can already be too
> >> > > many.
> >> > > Another problem is that with low checkpointing intervals, the files do not
> >> > > meet
> >> > > the required size. The latter point is probably addressable by changing the
> >> > > checkpoint interval, which might be inconvenient for some users.
> >> > >
> >> > > > The sink coordinator checkpoint problem (mentioned in option 1) would be
> >> > > > great if Flink can address it. In the spirit of source
> >> > > (enumerator-reader)
> >> > > > and sink (writer-coordinator) duality, sink coordinator checkpoint should
> >> > > > happen after the writer operator. This would be a natural fit to support
> >> > > > global committer in FLIP-143. It is probably an orthogonal matter to this
> >> > > > proposal.
> >> > >
> >> > >
> >> > > To me the question here is what are the benefits of having a coordinator in
> >> > > comparison to a global committer/aggregator operator.
> >> > >
> >> > > > Personally, I am usually in favor of keeping streaming ingestion (to data
> >> > > > lake) relatively simple and stable. Also sometimes compaction and sorting
> >> > > > are performed together in data rewrite maintenance jobs to improve read
> >> > > > performance. In that case, the value of compacting (in Flink streaming
> >> > > > ingestion) diminishes.
> >> > >
> >> > >
> >> > > I agree it is always possible to have scheduled maintenance jobs keeping
> >> > > care of
> >> > > your data i.e., doing compaction. Unfortunately, the downside is that you
> >> > > have to your data after it is already available for other downstream
> >> > > consumers.
> >> > > I guess this can lead to all kinds of visibility problems. I am also
> >> > > surprised that
> >> > > you personally are a fan of this approach and, on the other hand, are
> >> > > developing
> >> > > the Iceberg sink, which goes somewhat against your mentioned principle of
> >> > > keeping
> >> > > the sink simple.
> >> > >
> >> > > > Currently, it is unclear from the doc and this thread where the
> >> > > compaction
> >> > > > is actually happening. Jingsong's reply described one model
> >> > > > writer (parallel) -> aggregator (single-parallelism compaction planner)
> >> > > ->
> >> > > > compactor (parallel) -> global committer (single-parallelism)
> >> > >
> >> > >
> >> > > My idea of the topology is very similar to the one outlined by Jinsong. The
> >> > > compaction will happen in the committer operator.
> >> > >
> >> > > >
> >> > > > In the Iceberg community, the following model has been discussed. It is
> >> > > > better for Iceberg because it won't delay the data availability.
> >> > > > writer (parallel) -> global committer for append (single parallelism) ->
> >> > > > compactor (parallel) -> global committer for rewrite commit (single
> >> > > > parallelism)
> >> > >
> >> > >
> >> > > From a quick glimpse, it seems that the exact same topology is possible to
> >> > > express with the committable aggregator, but this definitely depends on
> >> > > the exact
> >> > > setup.
> >> > >
> >> > > Best,
> >> > > Fabian
> >>