Skip to content

Conversation

@garyli1019
Copy link
Member

@garyli1019 garyli1019 commented Jan 24, 2022

What is the purpose of the pull request

This pull request is to implement RFC-35(https://cwiki.apache.org/confluence/display/HUDI/RFC-35%3A+Make+Flink+MOR+table+writing+streaming+friendly)

Brief change log

  • Implement bucket index for Flink writer pipeline
  • Implement BucketStreamWriteFunction to write log file in a stream like fashion

Verify this pull request

This change added tests and can be verified as follows:

  • Added IT test for bucket stream write function.

Committer checklist

  • Has a corresponding JIRA in PR title & commit

  • Commit message is descriptive of the change

  • CI is green

  • Necessary doc changes done or have another open PR

  • For large changes, please consider breaking it into sub-tasks under an umbrella JIRA.

@garyli1019 garyli1019 marked this pull request as ready for review January 24, 2022 12:05
@garyli1019 garyli1019 requested a review from danny0405 January 24, 2022 12:06
@garyli1019
Copy link
Member Author

cc: @yihua FYI

@minchowang
Copy link
Contributor

Wow~ Great

@garyli1019
Copy link
Member Author

@hudi-bot run azure

@garyli1019 garyli1019 changed the title [RFC-35] Make Flink writer stream friendly [HUDI-2450] RFC-35 Make Flink writer stream friendly Jan 25, 2022
@garyli1019 garyli1019 changed the title [HUDI-2450] RFC-35 Make Flink writer stream friendly [HUDI-3315] RFC-35 Make Flink writer stream friendly Jan 25, 2022
@vinothchandar vinothchandar added the rfc Request for comments label Jan 26, 2022
public Boolean buffer(HoodieRecord<T> record, String instantTime, HoodieFlinkTable<T> table) {
final HoodieRecordLocation loc = record.getCurrentLocation();
final String fileID = loc.getFileId();
final String partitionPath = record.getPartitionPath();
Copy link
Contributor

Choose a reason for hiding this comment

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

Can we find a way to move the buffering logic into the write function ?

Copy link
Member Author

Choose a reason for hiding this comment

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

IMO we need to move the buffering logic into hoodie common client, then other engine could reuse it(a streaming API later). Kafka connect sink is also looking for a streaming way to write. Is there any advantage of putting the buffering logic into the write function that I am not aware of?

import org.apache.hudi.common.engine.HoodieEngineContext;
import org.apache.hudi.common.model.HoodieRecordPayload;
import org.apache.hudi.common.model.WriteOperationType;
import org.apache.hudi.config.HoodieWriteConfig;
Copy link
Contributor

Choose a reason for hiding this comment

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

Guess this class can be avoided if we move the buffering logic into the write function ?

.key(HoodieIndexConfig.BUCKET_INDEX_NUM_BUCKETS.key())
.intType()
.defaultValue(256) // default 256 buckets per partition
.withDescription("Hudi bucket number per partition. Only affected if using Hudi bucket index.");
Copy link
Contributor

Choose a reason for hiding this comment

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

Is there any reason the default value is 256 here, seems to generate many small files for small data sets.

Copy link
Member Author

Choose a reason for hiding this comment

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

sure, will change it into a smaller number

Copy link
Contributor

Choose a reason for hiding this comment

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

If people change the BUCKET_INDEX_NUM_BUCKETS or the write function parallelism, does the hash index still work ?

pipeline = Pipelines.hoodieStreamWrite(conf, parallelism, hoodieRecordDataStream);
if (OptionsResolver.isBucketIndexTable(conf)) {
if (!OptionsResolver.isMorTable(conf)) {
throw new HoodieNotSupportedException("Bucket index only support MOR table type.");
Copy link
Contributor

Choose a reason for hiding this comment

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

We can move the whole if ... else ... code block into the Pipelines.hoodieStreamWrite function.

Copy link
Member Author

Choose a reason for hiding this comment

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

sure

@vinothchandar
Copy link
Member

cc @yihua could you also please review this from the angle of making the write client abstractions more friendly

@yihua
Copy link
Contributor

yihua commented Feb 3, 2022

cc @yihua could you also please review this from the angle of making the write client abstractions more friendly

I'll review this.

@garyli1019
Copy link
Member Author

@hudi-bot run azure

@garyli1019
Copy link
Member Author

@hudi-bot run azure

@garyli1019
Copy link
Member Author

As discussed with @danny0405 , the changes in write client will be not included in this PR, because it will make the write client looks ugly. We will include those once we have a streaming API. So this PR will only include the bucket index for Flink writer. cc: @yihua

@garyli1019
Copy link
Member Author

@minihippo would you review this PR if you have time. Thanks~

@garyli1019
Copy link
Member Author

@hudi-bot run azure

@garyli1019
Copy link
Member Author

@hudi-bot run azure

@garyli1019 garyli1019 changed the title [HUDI-3315] RFC-35 Make Flink writer stream friendly [HUDI-3315] RFC-35 Part-1 Support bucket index in Flink writer Feb 22, 2022
Copy link
Contributor

@danny0405 danny0405 left a comment

Choose a reason for hiding this comment

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

+1, thanks for the contribution, i have left some comments ~


public static final ConfigOption<String> INDEX_KEY_FIELD = ConfigOptions
.key(HoodieIndexConfig.BUCKET_INDEX_HASH_FIELD.key())
.stringType()
Copy link
Contributor

Choose a reason for hiding this comment

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

Does this key must be same same with the primary key ? Because all the changes of a key must belong to one data bucket.

Copy link
Member Author

Choose a reason for hiding this comment

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

this could be a subset of primary keys. e.g. primary key could be "id1,id2", this index key could be either "id1" "id2" "id1,id2".

} else {
LOG.info(String.format("Adding fileID %s to the partition bucket %s.", fileID, partitionBucketId));
bucketToFileIDMap.put(partitionBucketId, fileID);
}
Copy link
Contributor

Choose a reason for hiding this comment

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

The bucketToFileIDMap seems never be cleared, is there possibility that this map be put into the state ?

Copy link
Member Author

Choose a reason for hiding this comment

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

yes, this could be store in the state, but we need to think about the consistency issue between the state and actual file system view. Any diff could lead to incorrect data.

Copy link
Contributor

Choose a reason for hiding this comment

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

The checkpoint can keep the correctness i think.

.key(HoodieIndexConfig.BUCKET_INDEX_NUM_BUCKETS.key())
.intType()
.defaultValue(256) // default 256 buckets per partition
.withDescription("Hudi bucket number per partition. Only affected if using Hudi bucket index.");
Copy link
Contributor

Choose a reason for hiding this comment

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

If people change the BUCKET_INDEX_NUM_BUCKETS or the write function parallelism, does the hash index still work ?

final int bucketNum = BucketIdentifier.getBucketId(hoodieKey, indexKeyFields, hiveBucketNum);
final String partitionBucketId = BucketIdentifier.partitionBucketIdStr(hoodieKey.getPartitionPath(), bucketNum);

if (bucketToFileIDMap.containsKey(partitionBucketId)) {
Copy link
Contributor

Choose a reason for hiding this comment

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

Partition changes are not supported?

Copy link
Member Author

Choose a reason for hiding this comment

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

you can change the job parallelism, but you can't change the bucket index number at this point.

Copy link
Contributor

Choose a reason for hiding this comment

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

I guess @loukey-lj want to address that when the record switches to new partition, how we send a delete record to the old partition ?

@garyli1019
Copy link
Member Author

@danny0405 If people change the write parallelism, it will still work because we load the parallelism-bucketID mapping at the runtime, but the parallelism should be less than the bucket number to avoid empty task.
At this point, the bucket number could not be changed. @minihippo is working on to support changing this number.

@garyli1019
Copy link
Member Author

@hudi-bot run azure

1 similar comment
@garyli1019
Copy link
Member Author

@hudi-bot run azure

@yangxiao0320
Copy link
Contributor

yangxiao0320 commented Feb 25, 2022 via email

table.getMetaClient().getBasePath()));

// Iterate through all existing partitions to load existing fileID belongs to this task
List<String> partitions = table.getMetadata().getAllPartitionPaths();
Copy link
Contributor

Choose a reason for hiding this comment

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

It may have poor performance for application starting when the partition num is huge. Load it at runtime may be better, especially in the case most of the partitions in the table are frozen.

Copy link
Member Author

Choose a reason for hiding this comment

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

let's do the optimization part in a separate PR.

Copy link
Contributor

Choose a reason for hiding this comment

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

Thanks, can we fire an issue to address this improvement ?

Copy link
Member Author

Choose a reason for hiding this comment

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

Copy link
Contributor

Choose a reason for hiding this comment

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

fix it with #5093

@garyli1019
Copy link
Member Author

@hudi-bot run azure


public static DataStream<Object> hoodieStreamWrite(Configuration conf, int defaultParallelism, DataStream<HoodieRecord> dataStream) {
WriteOperatorFactory<HoodieRecord> operatorFactory = StreamWriteOperator.getFactory(conf);
return dataStream
Copy link
Contributor

Choose a reason for hiding this comment

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

Can we have a separate method for hash index and not modify these two methods ? The methods are already too complex i think.

Copy link
Member Author

Choose a reason for hiding this comment

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

man you forgot your previous comment lol. It used to be separated and you suggested to put them into hoodieStreamWrite. Let me know if I misunderstood.

Copy link
Contributor

Choose a reason for hiding this comment

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

Yeah, if we can make both the code in Pipelines and HoodieTableSink clean ~

@hudi-bot
Copy link
Collaborator

hudi-bot commented Mar 1, 2022

CI report:

Bot commands @hudi-bot supports the following commands:
  • @hudi-bot run azure re-run the last Azure build

Copy link
Contributor

@danny0405 danny0405 left a comment

Choose a reason for hiding this comment

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

+1, thanks for the contribution ~

@danny0405 danny0405 merged commit 10d866f into apache:master Mar 2, 2022
@minchowang
Copy link
Contributor

🆒 I will try to this the PR soon. thanks for the bigolds. 🙏

@wxplovecc
Copy link
Contributor

I have tryed and got an Exception:

Caused by: java.util.NoSuchElementException: No value present in Option
	at org.apache.hudi.common.util.Option.get(Option.java:88) ~[hudi-flink-bundle_2.11-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT]
	at org.apache.hudi.io.HoodieMergeHandle.<init>(HoodieMergeHandle.java:116) ~[hudi-flink-bundle_2.11-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT]
	at org.apache.hudi.io.FlinkMergeHandle.<init>(FlinkMergeHandle.java:70) ~[hudi-flink-bundle_2.11-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT]
	at org.apache.hudi.client.HoodieFlinkWriteClient.getOrCreateWriteHandle(HoodieFlinkWriteClient.java:485) ~[hudi-flink-bundle_2.11-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT]
	at org.apache.hudi.client.HoodieFlinkWriteClient.upsert(HoodieFlinkWriteClient.java:142) ~[hudi-flink-bundle_2.11-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT]

@garyli1019
maybe we should not set instantTime to U when the fileId is created before checkpoint

@garyli1019 garyli1019 deleted the rfc-35 branch March 3, 2022 13:22
@garyli1019
Copy link
Member Author

@wxplovecc hi, this issue looks like not related to this PR, would you submit an issue then we can take a look.

@danny0405
Copy link
Contributor

@wxplovecc hi, this issue looks like not related to this PR, would you submit an issue then we can take a look.

Seems a bug 🐛 of the pr

vingov pushed a commit to vingov/hudi that referenced this pull request Apr 3, 2022
…e#4679)

* Support bucket index in Flink writer
* Use record key as default index key
stayrascal pushed a commit to stayrascal/hudi that referenced this pull request Apr 12, 2022
…e#4679)

* Support bucket index in Flink writer
* Use record key as default index key
@yangxiao0320
Copy link
Contributor

yangxiao0320 commented Oct 11, 2022 via email

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Labels

rfc Request for comments

Projects

None yet

Development

Successfully merging this pull request may close these issues.

10 participants