Skip to content

Conversation

@pvary
Copy link
Contributor

@pvary pvary commented Sep 26, 2023

Summary

The Flink community created a new Sink specification in FLIP-143 with the explicit goal to guarantee the unified handling of the bounded and unbounded data streams. Later it was enhanced in FLIP-191 so there is a well defined place to execute small files compaction. The deprecation of the old SinkFunction is postponed to somewhere around Flink 2.0 based on the discussion on the dev mailing list , so the migration is not extremely urgent, but having the possibility to use the PostCommitTopology to execute the compaction of the small files could provide immediate benefits for the users of the Iceberg-Flink integration.

Previous work

There is an existing Iceberg PR #4904 for the Sink migration by Liwei Li (https://github.com/hililiwei) and Kyle Bendickson (https://github.com/kbendick) with the related documentation which is authored by the same team. The discussion there is stuck, and the PR has been out of date for almost a year now. The current proposal builds heavily on their work and wants to keep them as the co-authors for the proposed change.

To start the discussion, I have created the following document.
https://docs.google.com/document/d/1K1M4wb9r_Tr-SDsUvqLyBaI5F14eRcqe3-ud6io0Da0/edit?usp=sharing

I propose the following timeline:

  1. Review the design document
  2. Update the PR
  3. PR reviews
  4. Merge the PR to the Iceberg source
  5. Restart the discussion about the missing features in the Flink community by creating a FLIP
  6. Discuss/review/merge the relevant Flink changes
  7. Release the Flink changes
  8. Create a PR in the Iceberg repo to start using the new Flink features
  9. Merge the Iceberg PR
  10. Be happy 😀

@pvary
Copy link
Contributor Author

pvary commented Sep 26, 2023

CC: @hililiwei, @stevenzwu, @chenjunjiedada, @gyfora - you might be interested in this

.defaultValue(false)
.withDescription("Use the FLIP-27 based Iceberg source implementation.");

public static final ConfigOption<Boolean> TABLE_EXEC_ICEBERG_USE_FLIP143_SINK =
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I think this config name is not very good. We should not include "flip" in it.
How about calling use-v2-sink?

* #init(IcebergFilesCommitterMetrics)} is idempotent, so it could be called multiple times to
* accommodate the missing init feature in the {@link Committer}.
*/
public class CommonCommitter implements Serializable {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Similar to the Sink base class maybe this should be called CommitterBase ?

Comment on lines +75 to +95
SinkBuilder builder() {
return builder;
}

FlinkWriteConf flinkWriteConf() {
return flinkWriteConf;
}

Map<String, String> writeProperties() {
return writeProperties;
}

SerializableSupplier<Table> tableSupplier() {
return tableSupplier;
}

RowType flinkRowType() {
return flinkRowType;
}

List<Integer> equalityFieldIds() {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Why not simply protected / package private fields and access them directly to reduce boilerplate?

Comment on lines +180 to +182
.setParallelism(1)
.setMaxParallelism(1)
.global();
Copy link
Contributor

@gyfora gyfora Oct 4, 2023

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

SetP/MaxP could be replaced by .forceNonParallel()

prefixIfNotNull(uidPrefix, initTable.name() + "-" + sinkId + "-pre-commit-topology"),
typeInformation,
new SinkV2Aggregator(commonCommitter))
.uid(prefixIfNotNull(uidPrefix, "pre-commit-topology"))
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Maybe a better name/uid would be ...-write-result-aggregator in case we add more things here later and it's also more descriptive

TypeInformation<CommittableMessage<SinkV2Committable>> typeInformation =
CommittableMessageTypeInfo.of(this::getCommittableSerializer);
return writeResults
.global()
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

We don't really need global() as the parallelism is forced to be 1

@github-actions
Copy link

This pull request has been marked as stale due to 30 days of inactivity. It will be closed in 1 week if no further activity occurs. If you think that’s incorrect or this pull request requires a review, please simply write any comment. If closed, you can revive the PR at any time and @mention a reviewer or discuss it on the dev@iceberg.apache.org list. Thank you for your contributions.

@github-actions github-actions bot added the stale label Sep 20, 2024
@github-actions
Copy link

This pull request has been closed due to lack of activity. This is not a judgement on the merit of the PR in any way. It is just a way of keeping the PR queue manageable. If you think that is incorrect, or the pull request requires review, you can revive the PR at any time.

@github-actions github-actions bot closed this Sep 27, 2024
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Projects

None yet

Development

Successfully merging this pull request may close these issues.

3 participants