Skip to content
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-25010][Connectors/Hive] Speed up hive's createMRSplits by multi thread #17988

Conversation

Myracle
Copy link
Contributor

@Myracle Myracle commented Dec 2, 2021

What is the purpose of the change

If there are many hive partitions, creating splits will take much time, for example, up to ten minutes. This mr speeds up the process by multi threads for different partitions.

Brief change log

  • Add class MRSplitsGetter to create splits by multi thread.
  • Add class HiveTablePartitionSplits to wrap hive partition and the corresponding inputSplits.
  • Transfer flinkConf to init MRSplitsGetter's thread num.
  • Adapt the related calls.

Verifying this change

This change is already covered by existing tests, such as HiveTableSourceITCase.

Does this pull request potentially affect one of the following parts:

  • Dependencies (does it add or upgrade a dependency): (no)
  • The public API, i.e., is any changed class annotated with @Public(Evolving): (no)
  • The serializers: (no)
  • The runtime per-record code paths (performance sensitive): (no)
  • Anything that affects deployment or recovery: JobManager (and its components), Checkpointing, Kubernetes/Yarn, ZooKeeper: (no)
  • The S3 file system connector: (no)

Documentation

  • Does this pull request introduce a new feature? (yes)
  • If yes, how is the feature documented? (not documented)

@flinkbot
Copy link
Collaborator

flinkbot commented Dec 2, 2021

CI report:

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

@flinkbot
Copy link
Collaborator

flinkbot commented Dec 2, 2021

Thanks a lot for your contribution to the Apache Flink project. I'm the @flinkbot. I help the community
to review your pull request. We will use this comment to track the progress of the review.

Automated Checks

Last check on commit 28b8d68 (Thu Dec 02 07:34:19 UTC 2021)

Warnings:

  • No documentation files were touched! Remember to keep the Flink docs up to date!
  • This pull request references an unassigned Jira ticket. According to the code contribution guide, tickets need to be assigned before starting with the implementation work.

Mention the bot in a comment to re-run the automated checks.

Review Progress

  • ❓ 1. The [description] looks good.
  • ❓ 2. There is [consensus] that the contribution should go into to Flink.
  • ❓ 3. Needs [attention] from.
  • ❓ 4. The change fits into the overall [architecture].
  • ❓ 5. Overall code [quality] is good.

Please see the Pull Request Review Guide for a full explanation of the review process.


The Bot is tracking the review progress through labels. Labels are applied according to the order of the review items. For consensus, approval by a Flink committer of PMC member is required Bot commands
The @flinkbot bot supports the following commands:

  • @flinkbot approve description to approve one or more aspects (aspects: description, consensus, architecture and quality)
  • @flinkbot approve all to approve all aspects
  • @flinkbot approve-until architecture to approve everything until architecture
  • @flinkbot attention @username1 [@username2 ..] to require somebody's attention
  • @flinkbot disapprove architecture to remove an approval you gave earlier

Copy link
Contributor

@luoyuxia luoyuxia left a comment

Choose a reason for hiding this comment

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

@Myracle Thanks for contribution. Looks general good to me. Just left some comments.
Also, would you like to share how long it takes to split how many hive partitions with single thread and muti-thread?

@Myracle
Copy link
Contributor Author

Myracle commented Dec 7, 2021

We have 5000 partitions for testing. When the thread is 1, it takes about 341 seconds. When the thread is 3, it takes 122 seconds. More threads means less time to split.

@Myracle Myracle force-pushed the FLINK-25010-Speed-up-hive-createMRSplits-by-multi-thread branch 4 times, most recently from 66b45d8 to ed14110 Compare December 10, 2021 09:00
@Myracle Myracle force-pushed the FLINK-25010-Speed-up-hive-createMRSplits-by-multi-thread branch from ed14110 to 140a48b Compare December 13, 2021 02:28
@Myracle Myracle force-pushed the FLINK-25010-Speed-up-hive-createMRSplits-by-multi-thread branch from 140a48b to f015c1d Compare December 13, 2021 11:05
@luoyuxia
Copy link
Contributor

lgtm. @wuchong @JingsongLi @lirui-apache Do you have time to review / merge?

@wuchong
Copy link
Member

wuchong commented Dec 15, 2021

Thanks @luoyuxia for the reviewing. I will have a look and merge it.

Copy link
Member

@wuchong wuchong left a comment

Choose a reason for hiding this comment

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

LGTM.

@wuchong wuchong force-pushed the FLINK-25010-Speed-up-hive-createMRSplits-by-multi-thread branch from f015c1d to e7be851 Compare December 21, 2021 04:09
@wuchong
Copy link
Member

wuchong commented Dec 21, 2021

@flinkbot run azure

1 similar comment
@wuchong
Copy link
Member

wuchong commented Dec 22, 2021

@flinkbot run azure

@@ -51,4 +51,11 @@
.withDescription(
"If it is false, using flink native writer to write parquet and orc files; "
+ "If it is true, using hadoop mapred record writer to write parquet and orc files.");

public static final ConfigOption<Integer> TABLE_EXEC_HIVE_PARTITION_SPLIT_THREAD_NUM =
key("table.exec.hive.partition-split.thread.num")
Copy link
Member

Choose a reason for hiding this comment

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

Sorry @Myracle, I have one more question. Is this configuration just like Hive's hive.load.dynamic.partitions.thread? I think maybe we can call the configuration table.exec.hive.load-partition-splits.thread-num to be closer to the actual behavior, what do you think?

Besides, could you also add this configuration to the Hive documentation ?

Copy link
Contributor Author

@Myracle Myracle Dec 25, 2021

Choose a reason for hiding this comment

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

@wuchong Good suggestion! I have modified it. Can you review it again? Thank you.

@Myracle Myracle force-pushed the FLINK-25010-Speed-up-hive-createMRSplits-by-multi-thread branch from e7be851 to 19a283c Compare December 25, 2021 03:08
@@ -163,6 +163,12 @@ following parameters in `TableConfig` (note that these parameters affect all sou
<td>Integer</td>
<td>Sets max infer parallelism for source operator.</td>
</tr>
<tr>
<td><h5>table.exec.hive.load-partition-splits.thread-num</h5></td>
Copy link
Member

Choose a reason for hiding this comment

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

This config option is not belong to Source Parallelism Inference, maybe we need to have a separate section, e.g. Load Partiton Splits

Copy link
Contributor Author

Choose a reason for hiding this comment

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

@wuchong Sorry for the wrong place. I have modified it. Please review it again. Thanks.

@Myracle Myracle force-pushed the FLINK-25010-Speed-up-hive-createMRSplits-by-multi-thread branch from 19a283c to ccc3e62 Compare January 6, 2022 03:14
@wuchong
Copy link
Member

wuchong commented Jan 6, 2022

Thanks for the updating @Myracle

@wuchong wuchong merged commit 2824c90 into apache:master Jan 6, 2022
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Projects
None yet
5 participants