stevenzwu commented on code in PR #7077:
URL: https://github.com/apache/iceberg/pull/7077#discussion_r1157557073
##########
flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/FlinkWriteConf.java:
##########
@@ -153,16 +167,89 @@ public String orcCompressionStrategy() {
.parse();
}
- public DistributionMode distributionMode() {
Review Comment:
I understand the motivation. but it makes the `FlinkWriteConf` not a simple
config hierarchy resolution class. it puts more complex logic in the conf
class. hence I am not sure if this is better or not.
##########
flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkSink.java:
##########
@@ -484,63 +484,23 @@ private DataStream<RowData> distributeDataStream(
PartitionSpec partitionSpec,
Schema iSchema,
RowType flinkRowType) {
- DistributionMode writeMode = flinkWriteConf.distributionMode();
+ DistributionMode writeMode =
+ flinkWriteConf.distributionMode(equalityFieldIds,
equalityFieldColumns);
LOG.info("Write distribution mode is '{}'", writeMode.modeName());
switch (writeMode) {
case NONE:
- if (equalityFieldIds.isEmpty()) {
- return input;
- } else {
- LOG.info("Distribute rows by equality fields, because there are
equality fields set");
- return input.keyBy(
- new EqualityFieldKeySelector(iSchema, flinkRowType,
equalityFieldIds));
- }
+ return input;
case HASH:
- if (equalityFieldIds.isEmpty()) {
- if (partitionSpec.isUnpartitioned()) {
- LOG.warn(
- "Fallback to use 'none' distribution mode, because there are
no equality fields set "
- + "and table is unpartitioned");
- return input;
- } else {
- return input.keyBy(new PartitionKeySelector(partitionSpec,
iSchema, flinkRowType));
- }
- } else {
- if (partitionSpec.isUnpartitioned()) {
- LOG.info(
- "Distribute rows by equality fields, because there are
equality fields set "
- + "and table is unpartitioned");
- return input.keyBy(
- new EqualityFieldKeySelector(iSchema, flinkRowType,
equalityFieldIds));
- } else {
- for (PartitionField partitionField : partitionSpec.fields()) {
- Preconditions.checkState(
- equalityFieldIds.contains(partitionField.sourceId()),
- "In 'hash' distribution mode with equality fields set,
partition field '%s' "
- + "should be included in equality fields: '%s'",
- partitionField,
- equalityFieldColumns);
- }
- return input.keyBy(new PartitionKeySelector(partitionSpec,
iSchema, flinkRowType));
- }
- }
-
- case RANGE:
- if (equalityFieldIds.isEmpty()) {
- LOG.warn(
- "Fallback to use 'none' distribution mode, because there are
no equality fields set "
- + "and {}=range is not supported yet in flink",
- WRITE_DISTRIBUTION_MODE);
- return input;
- } else {
+ if (partitionSpec.isUnpartitioned()) {
Review Comment:
is this correct? in the old logic, if it is `RANGE` and `equalityFieldIds`
is not empty, we will use `EqualityFieldKeySelector`. that doesn't seem to be
the case with this change.
--
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
To unsubscribe, e-mail: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]