-
Notifications
You must be signed in to change notification settings - Fork 2.5k
[HUDI-83] Fix Timestamp/Date type read by Hive3 #3391
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
Closed
Closed
Changes from all commits
Commits
Show all changes
15 commits
Select commit
Hold shift + click to select a range
ccd29b9
Rebase master 2022-12-04
cdmikechen 20ea53a
Fix by review
cdmikechen c52e649
Change to static final
cdmikechen 0fa6a88
Supplement modification description and adjust the class name to stat…
cdmikechen a482134
Fix review (change default support_timestamp to false, rename class n…
cdmikechen af53d0b
rebase master and rename FSUtils method
cdmikechen 57166d2
Change support version
cdmikechen 32c2be2
Append supportTimestamp
cdmikechen fc58129
Change version
cdmikechen 00f0a27
rename inputformat
cdmikechen 60bff79
add inputformat
cdmikechen 2e30eed
Optimise codes
cdmikechen 0fb7a3b
Fix timestamp
cdmikechen 2cd8e80
Rebase 2022.12.10 and fix test case error
cdmikechen b5e5133
Fix tab
cdmikechen 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
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
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
104 changes: 104 additions & 0 deletions
104
hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/avro/HoodieAvroParquetReader.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,104 @@ | ||
| /* | ||
| * Licensed to the Apache Software Foundation (ASF) under one | ||
| * or more contributor license agreements. See the NOTICE file | ||
| * distributed with this work for additional information | ||
| * regarding copyright ownership. The ASF licenses this file | ||
| * to you 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 org.apache.hudi.hadoop.avro; | ||
|
|
||
| import org.apache.avro.Schema; | ||
| import org.apache.avro.generic.GenericData; | ||
| import org.apache.avro.generic.GenericRecord; | ||
| import org.apache.hadoop.conf.Configuration; | ||
| import org.apache.hadoop.io.ArrayWritable; | ||
| import org.apache.hadoop.mapreduce.InputSplit; | ||
| import org.apache.hadoop.mapreduce.RecordReader; | ||
| import org.apache.hadoop.mapreduce.TaskAttemptContext; | ||
| import org.apache.hudi.hadoop.HoodieColumnProjectionUtils; | ||
| import org.apache.hudi.hadoop.utils.HoodieRealtimeRecordReaderUtils; | ||
| import org.apache.parquet.avro.AvroReadSupport; | ||
| import org.apache.parquet.avro.AvroSchemaConverter; | ||
| import org.apache.parquet.format.converter.ParquetMetadataConverter; | ||
| import org.apache.parquet.hadoop.ParquetFileReader; | ||
| import org.apache.parquet.hadoop.ParquetInputSplit; | ||
| import org.apache.parquet.hadoop.ParquetRecordReader; | ||
| import org.apache.parquet.hadoop.metadata.ParquetMetadata; | ||
| import org.apache.parquet.schema.MessageType; | ||
|
|
||
| import java.io.IOException; | ||
| import java.util.Arrays; | ||
| import java.util.List; | ||
| import java.util.stream.Collectors; | ||
|
|
||
| import static org.apache.parquet.hadoop.ParquetInputFormat.getFilter; | ||
|
|
||
| public class HoodieAvroParquetReader extends RecordReader<Void, ArrayWritable> { | ||
cdmikechen marked this conversation as resolved.
Show resolved
Hide resolved
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. @cdmikechen can you please help me understand why we need custom ParquetReader? |
||
|
|
||
| private final ParquetRecordReader<GenericData.Record> parquetRecordReader; | ||
|
|
||
| public HoodieAvroParquetReader(InputSplit inputSplit, Configuration conf) throws IOException { | ||
| AvroReadSupport avroReadSupport = new AvroReadSupport<>(); | ||
| // if exists read columns, we need to filter columns. | ||
| List<String> readColNames = Arrays.asList(HoodieColumnProjectionUtils.getReadColumnNames(conf)); | ||
| if (!readColNames.isEmpty()) { | ||
| // get base schema | ||
| ParquetMetadata fileFooter = | ||
codope marked this conversation as resolved.
Show resolved
Hide resolved
|
||
| ParquetFileReader.readFooter(conf, ((ParquetInputSplit) inputSplit).getPath(), ParquetMetadataConverter.NO_FILTER); | ||
| MessageType messageType = fileFooter.getFileMetaData().getSchema(); | ||
| Schema baseSchema = new AvroSchemaConverter(conf).convert(messageType); | ||
| // filter schema for reading | ||
| final Schema filterSchema = Schema.createRecord(baseSchema.getName(), | ||
| baseSchema.getDoc(), baseSchema.getNamespace(), baseSchema.isError(), | ||
| baseSchema.getFields().stream() | ||
| .filter(f -> readColNames.contains(f.name())) | ||
| .map(f -> new Schema.Field(f.name(), f.schema(), f.doc(), f.defaultVal())) | ||
| .collect(Collectors.toList())); | ||
| avroReadSupport.setAvroReadSchema(conf, filterSchema); | ||
| avroReadSupport.setRequestedProjection(conf, filterSchema); | ||
| } | ||
| parquetRecordReader = new ParquetRecordReader<>(avroReadSupport, getFilter(conf)); | ||
| } | ||
|
|
||
| @Override | ||
| public void initialize(InputSplit split, TaskAttemptContext context) throws IOException, InterruptedException { | ||
| parquetRecordReader.initialize(split, context); | ||
| } | ||
|
|
||
| @Override | ||
| public boolean nextKeyValue() throws IOException, InterruptedException { | ||
| return parquetRecordReader.nextKeyValue(); | ||
| } | ||
|
|
||
| @Override | ||
| public Void getCurrentKey() throws IOException, InterruptedException { | ||
| return parquetRecordReader.getCurrentKey(); | ||
| } | ||
|
|
||
| @Override | ||
| public ArrayWritable getCurrentValue() throws IOException, InterruptedException { | ||
| GenericRecord record = parquetRecordReader.getCurrentValue(); | ||
| return (ArrayWritable) HoodieRealtimeRecordReaderUtils.avroToArrayWritable(record, record.getSchema(), true); | ||
codope marked this conversation as resolved.
Show resolved
Hide resolved
|
||
| } | ||
|
|
||
| @Override | ||
| public float getProgress() throws IOException, InterruptedException { | ||
| return parquetRecordReader.getProgress(); | ||
| } | ||
|
|
||
| @Override | ||
| public void close() throws IOException { | ||
| parquetRecordReader.close(); | ||
| } | ||
| } | ||
44 changes: 44 additions & 0 deletions
44
...-mr/src/main/java/org/apache/hudi/hadoop/avro/HoodieTimestampAwareParquetInputFormat.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,44 @@ | ||
| /* | ||
| * Licensed to the Apache Software Foundation (ASF) under one | ||
| * or more contributor license agreements. See the NOTICE file | ||
| * distributed with this work for additional information | ||
| * regarding copyright ownership. The ASF licenses this file | ||
| * to you 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 org.apache.hudi.hadoop.avro; | ||
|
|
||
| import org.apache.hadoop.conf.Configuration; | ||
| import org.apache.hadoop.io.ArrayWritable; | ||
| import org.apache.hadoop.mapreduce.InputSplit; | ||
| import org.apache.hadoop.mapreduce.RecordReader; | ||
| import org.apache.hadoop.mapreduce.TaskAttemptContext; | ||
| import org.apache.parquet.hadoop.ParquetInputFormat; | ||
| import org.apache.parquet.hadoop.util.ContextUtil; | ||
|
|
||
| import java.io.IOException; | ||
|
|
||
| /** | ||
| * To resolve issue <a href="https://issues.apache.org/jira/browse/HUDI-83">Fix Timestamp/Date type read by Hive3</a>, | ||
| * we need to handle timestamp types separately based on the parquet-avro approach | ||
| */ | ||
| public class HoodieTimestampAwareParquetInputFormat extends ParquetInputFormat<ArrayWritable> { | ||
|
|
||
| @Override | ||
| public RecordReader<Void, ArrayWritable> createRecordReader( | ||
| InputSplit inputSplit, | ||
| TaskAttemptContext taskAttemptContext) throws IOException, InterruptedException { | ||
| Configuration conf = ContextUtil.getConfiguration(taskAttemptContext); | ||
| return new HoodieAvroParquetReader(inputSplit, conf); | ||
| } | ||
| } |
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
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.
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.
not quite sure about
types.isEmpty()being true then support timestamp?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.
@xushiyan
I was trying to think of a worst case scenario if I couldn't find the columns. I had encountered a similar problem with some of the test cases when running the azure test cases, so I've added this treatment here.