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-19365][hive] Migrate Hive source to FLIP-27 source interface f… #13915

Merged
merged 4 commits into from Nov 5, 2020

Conversation

lirui-apache
Copy link
Contributor

…or batch

What is the purpose of the change

Migrate hive connector to FLIP-27 source for batch reading

Brief change log

  • Implement HiveSourceSplit as sub-class of FileSourceSplit. It remembers the partition which a split belongs to.
  • Implement HiveSourceSplitSerializer as SerDe for HiveSourceSplit.
  • Implement HiveSource as sub-class of AbstractFileSource, and works with RowData and HiveSourceSplit.
  • Implement HiveSourceFileEnumerator as an implementation of FileEnumerator, but it generates splits based on partitions rather than paths.
  • Implement HiveBulkFormatAdapter as an implementation of BulkFormat<RowData, HiveSourceSplit>. It delegates the reading to other BulkFormat instances based on the information we have from each partition to read.
  • Update PartitionValueConverter to take partition col name when converting the partition value. This makes it easier for hive to reuse ParquetColumnarRowInputFormat because the partition values have been pre-computed.

Verifying this change

Existing tests

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

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

Documentation

  • Does this pull request introduce a new feature? no
  • If yes, how is the feature documented? NA

@flinkbot
Copy link
Collaborator

flinkbot commented Nov 4, 2020

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 197131c (Wed Nov 04 03:50:36 UTC 2020)

Warnings:

  • 1 pom.xml files were touched: Check for build and licensing issues.
  • No documentation files were touched! Remember to keep the Flink docs up to date!

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

@flinkbot
Copy link
Collaborator

flinkbot commented Nov 4, 2020

CI report:

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

JobConf jobConf,
CatalogTable catalogTable,
List<HiveTablePartition> partitions,
int[] projectedFields,
Copy link
Contributor

Choose a reason for hiding this comment

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

Already has producedType, I think projectedFields can be removed

Copy link
Contributor Author

Choose a reason for hiding this comment

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

HiveMapredSplitReader still requires it. We can remove it when we refactor HiveMapredSplitReader

Copy link
Contributor

Choose a reason for hiding this comment

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

I think you can create projectedFields from producedType easily.

}

private void serialize(ObjectOutputStream outputStream, HiveSourceSplit split) throws IOException {
byte[] superBytes = FileSourceSplitSerializer.INSTANCE.serialize(new FileSourceSplit(
Copy link
Contributor

Choose a reason for hiding this comment

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

Why do you want to create a new FileSourceSplit? Why not just use HiveSourceSplit

Copy link
Contributor Author

Choose a reason for hiding this comment

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

I think it's better to reuse FileSourceSplitSerializer to serialize the super class, but FileSourceSplitSerializer doesn't accept sub-classes.
Or we can just call InstantiationUtil.serializeObject to serialize the whole object. WDYT?

Copy link
Contributor

Choose a reason for hiding this comment

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

but FileSourceSplitSerializer doesn't accept sub-classes.

Oh, that is.

Copy link
Contributor

Choose a reason for hiding this comment

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

I am OK to keep this as it is.

partitionKeys,
jobConfWrapper.conf().get(HiveConf.ConfVars.DEFAULTPARTITIONNAME.varname,
HiveConf.ConfVars.DEFAULTPARTITIONNAME.defaultStrVal),
(PartitionValueConverter) (colName, valStr, type) -> split.getHiveTablePartition().getPartitionSpec().get(colName),
Copy link
Contributor

Choose a reason for hiding this comment

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

This will fail when partition values can not be found in path.
I create #13919 to refactor this.

JobConf clonedConf = new JobConf(jobConfWrapper.conf());
addSchemaToConf(clonedConf);
HiveTableInputSplit oldSplit = new HiveTableInputSplit(-1, split.toMapRedSplit(), clonedConf, split.getHiveTablePartition());
hiveMapredSplitReader = new HiveMapredSplitReader(clonedConf, partitionKeys, fieldTypes, selectedFields, oldSplit, hiveShim);
Copy link
Contributor

Choose a reason for hiding this comment

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

If you don't want migrate HiveMapredSplitReader now, can you create JIRA for this?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

@lirui-apache
Copy link
Contributor Author

@JingsongLi Thanks for the review. I have rebased and addressed your comments.

new org.apache.flink.core.fs.Path[1],
new HiveSourceFileEnumerator.Provider(partitions, new JobConfWrapper(jobConf)),
DEFAULT_SPLIT_ASSIGNER,
createBulkFormat(new JobConf(jobConf), catalogTable, hiveVersion, producedDataType, useMapRedReader, limit),
Copy link
Contributor

Choose a reason for hiding this comment

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

I think you should use LimitableBulkFormat here.

parallelism = Math.min(parallelism, (int) limit / 1000);
int limit(Long limit) {
if (limit != null) {
parallelism = Math.min(parallelism, limit.intValue() / 1000);
Copy link
Contributor

Choose a reason for hiding this comment

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

Can you change limit.intValue() / 1000 to (int) (limit / 1000)

private final RowType producedDataType;
private final boolean useMapRedReader;
// We should limit the input read count of the splits, null represents no limit.
private final Long limit;
Copy link
Contributor

Choose a reason for hiding this comment

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

Remove limit things in this class

private final DataType[] fieldTypes;
private final String hiveVersion;
private final HiveShim hiveShim;
private final RowType producedDataType;
Copy link
Contributor

Choose a reason for hiding this comment

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

producedRowType

String hiveVersion,
boolean useMapRedReader,
boolean isStreamingSource,
RowType producedDataType) {
Copy link
Contributor

Choose a reason for hiding this comment

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

producedRowType

}

private boolean reachLimit() {
return limit != null && numRead >= limit;
Copy link
Contributor

Choose a reason for hiding this comment

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

Please remove limit things

Copy link
Contributor

@JingsongLi JingsongLi left a comment

Choose a reason for hiding this comment

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

Looks good to me, Thanks @lirui-apache for update.

@JingsongLi JingsongLi closed this Nov 5, 2020
@JingsongLi JingsongLi reopened this Nov 5, 2020
@JingsongLi JingsongLi merged commit 56207e8 into apache:master Nov 5, 2020
@lirui-apache lirui-apache deleted the FLINK-19365 branch November 5, 2020 08:20
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Projects
None yet
4 participants