-
Notifications
You must be signed in to change notification settings - Fork 486
[flink] Flink sink support hash by bucket id for PrimaryKey Table #579
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
Merged
Merged
Changes from all commits
Commits
File filter
Filter by extension
Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
There are no files selected for viewing
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
40 changes: 40 additions & 0 deletions
40
...-flink/fluss-flink-common/src/main/java/com/alibaba/fluss/flink/sink/ChannelComputer.java
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,40 @@ | ||
| /* | ||
| * Copyright (c) 2025 Alibaba Group Holding Ltd. | ||
| * | ||
| * Licensed under the Apache License, Version 2.0 (the "License"); | ||
| * you may not use this file except in compliance with the License. | ||
| * You may obtain a copy of the License at | ||
| * | ||
| * http://www.apache.org/licenses/LICENSE-2.0 | ||
| * | ||
| * Unless required by applicable law or agreed to in writing, software | ||
| * distributed under the License is distributed on an "AS IS" BASIS, | ||
| * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. | ||
| * See the License for the specific language governing permissions and | ||
| * limitations under the License. | ||
| */ | ||
|
|
||
| package com.alibaba.fluss.flink.sink; | ||
|
|
||
| import java.io.Serializable; | ||
|
|
||
| /** | ||
| * A utility class to compute which downstream channel a given record should be sent to before flink | ||
| * sink. | ||
| * | ||
| * @param <T> type of record | ||
| */ | ||
| public interface ChannelComputer<T> extends Serializable { | ||
| void setup(int numChannels); | ||
|
|
||
| int channel(T record); | ||
|
|
||
| static int select(String partitionName, int bucket, int numChannels) { | ||
| int startChannel = Math.abs(partitionName.hashCode()) % numChannels; | ||
| return (startChannel + bucket) % numChannels; | ||
| } | ||
|
|
||
| static int select(int bucket, int numChannels) { | ||
| return bucket % numChannels; | ||
| } | ||
| } |
110 changes: 110 additions & 0 deletions
110
...-flink-common/src/main/java/com/alibaba/fluss/flink/sink/FlinkRowDataChannelComputer.java
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,110 @@ | ||
| /* | ||
| * Copyright (c) 2025 Alibaba Group Holding Ltd. | ||
| * | ||
| * Licensed under the Apache License, Version 2.0 (the "License"); | ||
| * you may not use this file except in compliance with the License. | ||
| * You may obtain a copy of the License at | ||
| * | ||
| * http://www.apache.org/licenses/LICENSE-2.0 | ||
| * | ||
| * Unless required by applicable law or agreed to in writing, software | ||
| * distributed under the License is distributed on an "AS IS" BASIS, | ||
| * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. | ||
| * See the License for the specific language governing permissions and | ||
| * limitations under the License. | ||
| */ | ||
|
|
||
| package com.alibaba.fluss.flink.sink; | ||
|
|
||
| import com.alibaba.fluss.annotation.VisibleForTesting; | ||
| import com.alibaba.fluss.bucketing.BucketingFunction; | ||
| import com.alibaba.fluss.client.table.getter.PartitionGetter; | ||
| import com.alibaba.fluss.flink.row.FlinkAsFlussRow; | ||
| import com.alibaba.fluss.metadata.DataLakeFormat; | ||
| import com.alibaba.fluss.row.encode.KeyEncoder; | ||
| import com.alibaba.fluss.types.RowType; | ||
|
|
||
| import org.apache.flink.table.data.RowData; | ||
|
|
||
| import javax.annotation.Nullable; | ||
|
|
||
| import java.util.List; | ||
|
|
||
| import static com.alibaba.fluss.utils.Preconditions.checkNotNull; | ||
|
|
||
| /** {@link ChannelComputer} for flink {@link RowData}. */ | ||
| public class FlinkRowDataChannelComputer implements ChannelComputer<RowData> { | ||
|
|
||
| private static final long serialVersionUID = 1L; | ||
|
|
||
| private final @Nullable DataLakeFormat lakeFormat; | ||
| private final int numBucket; | ||
| private final RowType flussRowType; | ||
| private final List<String> bucketKeys; | ||
| private final List<String> partitionKeys; | ||
|
|
||
| private transient int numChannels; | ||
| private transient BucketingFunction bucketingFunction; | ||
| private transient KeyEncoder bucketKeyEncoder; | ||
| private transient boolean combineShuffleWithPartitionName; | ||
| private transient @Nullable PartitionGetter partitionGetter; | ||
|
|
||
| public FlinkRowDataChannelComputer( | ||
| RowType flussRowType, | ||
| List<String> bucketKeys, | ||
| List<String> partitionKeys, | ||
| @Nullable DataLakeFormat lakeFormat, | ||
| int numBucket) { | ||
| this.flussRowType = flussRowType; | ||
| this.bucketKeys = bucketKeys; | ||
| this.partitionKeys = partitionKeys; | ||
| this.lakeFormat = lakeFormat; | ||
| this.numBucket = numBucket; | ||
| } | ||
|
|
||
| @Override | ||
| public void setup(int numChannels) { | ||
| this.numChannels = numChannels; | ||
| this.bucketingFunction = BucketingFunction.of(lakeFormat); | ||
| this.bucketKeyEncoder = KeyEncoder.of(flussRowType, bucketKeys, lakeFormat); | ||
| if (partitionKeys.isEmpty()) { | ||
| this.partitionGetter = null; | ||
| } else { | ||
| this.partitionGetter = new PartitionGetter(flussRowType, partitionKeys); | ||
| } | ||
|
|
||
| // Only when partition keys exist and the Flink job parallelism and the bucket number are | ||
| // not divisible, then we need to include the partition name as part of the shuffle key. | ||
| // This approach can help avoid the possible data skew. For example, if bucket number is 3 | ||
| // and task parallelism is 2, it is highly possible that data shuffle becomes uneven. For | ||
| // instance, in task1, it might have 'partition0-bucket0', 'partition1-bucket0', | ||
| // 'partition0-bucket2', and 'partition1-bucket2', whereas in task2, it would only have | ||
| // 'partition0-bucket1' and 'partition1-bucket1'. As partition number increases, this | ||
| // situation becomes even more severe. | ||
| this.combineShuffleWithPartitionName = | ||
| partitionGetter != null && numBucket % numChannels != 0; | ||
| } | ||
|
|
||
| @Override | ||
| public int channel(RowData record) { | ||
| FlinkAsFlussRow row = new FlinkAsFlussRow().replace(record); | ||
| int bucketId = bucketingFunction.bucketing(bucketKeyEncoder.encodeKey(row), numBucket); | ||
| if (!combineShuffleWithPartitionName) { | ||
| return ChannelComputer.select(bucketId, numChannels); | ||
| } else { | ||
| checkNotNull(partitionGetter, "partitionGetter is null"); | ||
| String partitionName = partitionGetter.getPartition(row); | ||
| return ChannelComputer.select(partitionName, bucketId, numChannels); | ||
| } | ||
| } | ||
|
|
||
| @Override | ||
| public String toString() { | ||
| return "BUCKET_SHUFFLE"; | ||
| } | ||
|
|
||
| @VisibleForTesting | ||
| boolean isCombineShuffleWithPartitionName() { | ||
| return combineShuffleWithPartitionName; | ||
| } | ||
| } | ||
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Oops, something went wrong.
Add this suggestion to a batch that can be applied as a single commit.
This suggestion is invalid because no changes were made to the code.
Suggestions cannot be applied while the pull request is closed.
Suggestions cannot be applied while viewing a subset of changes.
Only one suggestion per line can be applied in a batch.
Add this suggestion to a batch that can be applied as a single commit.
Applying suggestions on deleted lines is not supported.
You must change the existing code in this line in order to create a valid suggestion.
Outdated suggestions cannot be applied.
This suggestion has been applied or marked resolved.
Suggestions cannot be applied from pending reviews.
Suggestions cannot be applied on multi-line comments.
Suggestions cannot be applied while the pull request is queued to merge.
Suggestion cannot be applied right now. Please check back later.
Uh oh!
There was an error while loading. Please reload this page.