-
Notifications
You must be signed in to change notification settings - Fork 3k
Flink: port range distribution to v2 iceberg sink #12071
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Flink: port range distribution to v2 iceberg sink #12071
Conversation
|
Hey @rodmeneses! Thanks for porting this feature over. I'll have a look shortly. |
mxm
left a comment
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
LGTM
|
@pvary Can you take a look as well? |
flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergSink.java
Outdated
Show resolved
Hide resolved
...link/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2DistributionMode.java
Outdated
Show resolved
Hide resolved
...link/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2DistributionMode.java
Show resolved
Hide resolved
flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergSink.java
Outdated
Show resolved
Hide resolved
|
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. |
|
keeping it alive |
|
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. |
|
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. |
|
Hi team, how is the progress on this PR? I really need this feature in SinkV2. |
|
reopened the PR. @rodmeneses please update when the it is ready for review. right now, it is marked as draft. |
|
+1 it would be nice to follow up with this. If @rodmeneses is busy, maybe @Guosmilesmile could also take this one? |
|
Hi @stevenzwu thanks for reopening this. I will try to finish it this coming week. I think it needs only to port some fixes recently made in the FlinkSink RANGE distribution mode, as well as address some reviews. |
c44ba98 to
21c628a
Compare
| } | ||
| } | ||
|
|
||
| private DataStream<RowData> distributeDataStreamByNoneDistributionMode( |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Breaking distributeDataStream into smaller functions, each one for a given distributionMode. due to:
Cyclomatic Complexity is 13 (max allowed is 12). [CyclomaticComplexity]
This way, it is also clear what each distributionMode needs as function parameters, i.e:
distributeDataStreamByNoneDistributionMode -> (DataStream<RowData> input, Schema schema)distributeDataStreamByHashDistributionMode -> (DataStream<RowData> input, Schema schema, PartitionSpec spec)distributeDataStreamByRangeDistributionMode->
DataStream input, Schema schema, PartitionSpec spec, SortOrder sortOrderParam`
Which brings nice information about what each distributionMode needs for its internal calculation
stevenzwu
left a comment
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
@rodmeneses please mark the PR ready for view when ready. right now, it is still a draft.
One uber comment: it has been a long time since the initial creation of this PR. there were some changes in FlinkSink. please do another path and make sure the drift has been ported here.
also, there are some old comments from Peter not addressed. but looks like those problem might also exist in the v1 sink test too.
flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergSink.java
Show resolved
Hide resolved
| + "and table is unpartitioned"); | ||
| return input; | ||
| } else { | ||
| if (BucketPartitionerUtil.hasOneBucketField(spec)) { |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
can you check with FlinkSink code again? this code has been removed/reverted there.
|
|
||
| return shuffleStream | ||
| .partitionCustom(new RangePartitioner(schema, sortOrder), r -> r) | ||
| .filter(StatisticsOrRecord::hasRecord) |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
there has been changes from when this PR was initially created. please re-sync with FlinkSink
82e7d90 to
de8ea5e
Compare
| } | ||
|
|
||
| @TestTemplate | ||
| void testJobNoneDistributeMode() throws Exception { |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
All these methods were incorrectly copied here in previous commit and they are duplicates of the ones on TestFlinkIcebergSinkV2DistributionMode
So I'm removing them
de8ea5e to
d2b4b78
Compare
| PartitionSpec partitionSpec, | ||
| SortOrder sortOrderParam) { | ||
|
|
||
| int writerParallelism = |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
this logic should be applied to the writer parallelism for v2 sink.
if write parallelism is not configured, the v1 sink default the writer parallelism to the input parallelism to promote chaining. Want to confirm if that is the case for v2 sink? from reading the code, I thought v2 sink will default the writer parallelism to default job parallelism?
// Note that IcebergSink internally consists o multiple operators (like writer, committer,
// aggregator).
// The following parallelism will be propagated to all of the above operators.
if (sink.flinkWriteConf.writeParallelism() != null) {
rowDataDataStreamSink.setParallelism(sink.flinkWriteConf.writeParallelism());
}
technically, if this is a behavior change problem for v2 sink, it is not caused by this PR. but it is critical that the same writer parallelism is used by the shuffle operator to properly range partition the data to downstream writer tasks. That is why in the v1 FlinkSink, you can see writerParallelism is computed once and pass to two methods.
int writerParallelism =
flinkWriteConf.writeParallelism() == null
? rowDataInput.getParallelism()
: flinkWriteConf.writeParallelism();
// Distribute the records from input data stream based on the write.distribution-mode and
// equality fields.
DataStream<RowData> distributeStream =
distributeDataStream(rowDataInput, equalityFieldIds, flinkRowType, writerParallelism);
// Add parallel writers that append rows to files
SingleOutputStreamOperator<FlinkWriteResult> writerStream =
appendWriter(distributeStream, flinkRowType, equalityFieldIds, writerParallelism);
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Hi @stevenzwu
Thanks for your comment.
We have the same logic in the beginning of the distributeDataStreamByRangeDistributionMode method:
int writerParallelism =
flinkWriteConf.writeParallelism() == null
? input.getParallelism()
: flinkWriteConf.writeParallelism();
So I think that for range partitioning the behavior should be the same. What do you think ?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
sorry, I wasn't very clear earlier. The v2 sink writer parallelism selection is different than the v1 sink. It doesn't use input parallelism if write parallelism is not configured explicitly.
@Override
public Builder writeParallelism(int newWriteParallelism) {
writeOptions.put(
FlinkWriteOptions.WRITE_PARALLELISM.key(), Integer.toString(newWriteParallelism));
return this;
}
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Hi @stevenzwu . Thanks for the clarification. I think I know what you mean. On FlinkSink, even without talking about RANGE distribution, the parallelism of the v1 sink by default will be the same as the input source parallelism.
This is a good approach, because it encourage chaining.
However, we dont have that logic on the IcebergSink v2. I think we should have the same logic there. I could do that in another PR and the follow up with this one. What do you think? Thanks
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
sure. you can follow up with a new PR for the writer parallelism fix.
can you resolve the conflict? then we can merge this.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
will do!
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Hi @stevenzwu I have rebased and this is ready! Thanks
Tagging @mxm and @pvary as well
ce63639 to
1dcf734
Compare
1dcf734 to
2721fbe
Compare
mxm
left a comment
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
LGTM. This should be ported to 2.0 and 1.19 once merged.
|
thanks @rodmeneses for the contribution and @mxm @pvary for the review |
This PR ports the RANGE distribution mode on the
FlinkSinkto the newIcebergSinkbased on the Flink V2 sink interface.cc: @stevenzwu @mxm @pvary @Guosmilesmile