Good news. There is now a pull request available:
https://github.com/apache/iceberg/pull/12424

-Max

On Thu, Nov 21, 2024 at 4:36 PM Ferenc Csaky <ferenc.cs...@pm.me.invalid> wrote:
>
> Hello devs,
>
> +1 from my side, as I look things from the Flink perspective. The Flink 
> mailing list thread Peter
> linked in his previous message already has more supporters who are agreeing 
> this feature
> would be pretty helpful regarding CDC tasks as well.
>
> Multiple users (including us) are looking forward to this feature. Looking 
> forward to any input
> from the Iceberg side.
>
> Best,
> Ferenc
> On Thursday, November 21st, 2024 at 15:11, Péter Váry 
> <peter.vary.apa...@gmail.com> wrote:
>
> Many of the Flink users support the Dynamic Sink.
> See: https://lists.apache.org/thread/khw0z63n34cmh2nrzrx7j9bdmzz861lb
>
> Any comments from the Iceberg community side?
>
> Jean-Baptiste Onofré <j...@nanthrax.net> ezt írta (időpont: 2024. nov. 13., 
> Sze, 14:06):
>>
>> Thanks for the proposal!
>> I will take a look asap.
>>
>> Regards
>> JB
>>
>> On Mon, Nov 11, 2024 at 6:32 PM Péter Váry <peter.vary.apa...@gmail.com> 
>> wrote:
>> >
>> > Hi Team,
>> >
>> > With Max Michels, we started to work on enhancing the current Iceberg Sink 
>> > to allow inserting evolving records into a changing table.
>> > See: 
>> > https://docs.google.com/document/d/1R3NZmi65S4lwnmNjH4gLCuXZbgvZV5GNrQKJ5NYdO9s
>> > Created the project to follow the lifecycle of the proposal: 
>> > https://github.com/orgs/apache/projects/429
>> >
>> > From the abstract:
>> > ---------
>> > Flink Iceberg connector sink is the tool to write data to an Iceberg table 
>> > from a continuous Flink stream. The current Sink implementations emphasize 
>> > throughput over flexibility. The main limiting factor is that the Iceberg 
>> > Flink Sink requires static table structure. The table, the schema, the 
>> > partitioning specification need to be constant. If one of the previous 
>> > things changes the Flink Job needs to be restarted. This allows using 
>> > optimal record serialization and good performance, but real life use-cases 
>> > need to work around this limitation when the underlying table has changed. 
>> > We need to provide a tool to accommodate these changes.
>> > [..]
>> > The following typical use cases are considered during this design:
>> > - Incoming Avro records schema changes (new columns are added, or other 
>> > backward compatible changes happen). The Flink job is expected to update 
>> > the table schema dynamically, and continue to ingest data with the new and 
>> > the old schema without a job restart.
>> > - Incoming records define the target Iceberg table dynamically. The Flink 
>> > job is expected to create the new table(s) and continue writing to them 
>> > without a job restart.
>> > - The partitioning schema of the table changes. The Flink job is expected 
>> > to update the specification and continue writing to the target table 
>> > without a job restart.
>> > ---------
>> >
>> > If you have any questions, ideas, suggestions please let us know here, or 
>> > in comments on the document.
>> >
>> > Thanks,
>> > Peter
>
>

Reply via email to