Skip to content

Conversation

@cdmikechen
Copy link
Contributor

@cdmikechen cdmikechen commented Aug 3, 2021

Change Logs

This pull request let hive can read timestamp type column datas correctly.
The problem was initially related to JIRA HUDI-83 and related issues on issue #2544

  • Change HoodieParquetInputFormat to use a custom ParquetInputFormat named HudiAvroParquetInputFormat
  • In HudiAvroParquetInputFormat we use a custom RecordReader named HudiAvroParquetReader. In this class we use AvroReadSupport so that Hive can get parquet data with an avro GenericRecord.
  • Use org.apache.hudi.hadoop.utils.HoodieRealtimeRecordReaderUtils.avroToArrayWritable to transform GenericRecord to ArrayWriteable. At the same time, timestamp/date type processing for different situations of hive2 and hive3 is added to this method.
  • Set hoodie.datasource.hive_sync.support_timestamp default value from false to true
  • add a supportAvroRead value to be compatible with the adaptation of some old hudi versions for hive3 timestamp/date types

Impact

  • hudi-hadoop-mr
  • spark

Risk level

low

Documentation Update

The javadoc has been modified and the website document will be on other PR later.

Contributor's checklist

  • Read through contributor's guide
  • Change Logs and Impact were stated clearly
  • Adequate tests were added if applicable
  • CI passed

@cdmikechen cdmikechen force-pushed the HUDI-83 branch 2 times, most recently from 84b1840 to e19068f Compare August 8, 2021 14:53
@cdmikechen
Copy link
Contributor Author

@hudi-bot run azure

@cdmikechen cdmikechen marked this pull request as draft August 8, 2021 15:40
Copy link
Member

@vinothchandar vinothchandar left a comment

Choose a reason for hiding this comment

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

I have a concern around performance overhead and also wondering if we can just do it as a part of the existing inputformat with a flag, instead of switching over entirely to a new ipf? thougnts?

@Override
public ArrayWritable getCurrentValue() throws IOException, InterruptedException {
GenericRecord record = parquetRecordReader.getCurrentValue();
return (ArrayWritable) HoodieRealtimeRecordReaderUtils.avroToArrayWritable(record, record.getSchema());
Copy link
Member

Choose a reason for hiding this comment

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

this extra avro conversion will cost performance? Wondering if we can avoid this.

Copy link
Contributor Author

@cdmikechen cdmikechen Dec 29, 2021

Choose a reason for hiding this comment

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

@vinothchandar
I have been running the fork for several months. At present, it does not cause too many additional problems. This may be related to the small amount of data processed by my hive and the insufficient impact of memory.

At present, the parsing of avro data by hudi spark in org.apache.hudi.AvroConversionHelper or hive itself in org.apache.hadoop.hive.serde2.avro.AvroGenericRecordWritable both wrapper around an Avro GenericRecord. Normally, I think data processing should not cause serious overhead .

Meanwhile, in part of the instantiation of TimestampWritableV2, I reconstructed some code to enhance and fix some of the original errors and problems.

@ghost
Copy link

ghost commented Sep 2, 2021

Hey,
Not sure if I'm "allowed" to chime in here, but I don't know where else to post this?

I was trying to build this locally and it compiled fine.

However, when using it on my Spark 2.4.4 environment, I started hitting NoSuchMethodError for getUseVectorizedInputFileFormat(). After investigation, this is because Spark 2.4.4 is built with hive-exec:1.2.1.spark2, and the method getUseVectorizedInputFileFormat() does not exist within Utilities in hive-exec:1.2.1.spark2.
I fixed this issue by manually implementing this method within HoodieParquetInputFormat.java and I was finally able to test this out in a proper spark 2.4.4 environment.

I'm not sure if manually implementing this method is the right way to go or not, but I thought I'd share my thoughts here hoping that it would help you...?

Let me know!

@vinothchandar vinothchandar added the area:schema Schema evolution and data types label Sep 7, 2021
@cdmikechen cdmikechen marked this pull request as ready for review December 29, 2021 04:06
@cdmikechen
Copy link
Contributor Author

I have a concern around performance overhead and also wondering if we can just do it as a part of the existing inputformat with a flag, instead of switching over entirely to a new ipf? thougnts?

For compatibility com.twitter:parquet-hadoop-bundle which used for ParquetInputFormat in Spark2 (It only contains a parameterless constructor, while in hive2 and hive3, a constructor containing ParquetInputFormat is added)

https://github.com/apache/hive/blob/8e7f23f34b2ce7328c9d571a13c336f0c8cdecb6/ql/src/java/org/apache/hadoop/hive/ql/io/parquet/MapredParquetInputFormat.java#L48-L55

  public MapredParquetInputFormat() {
    this(new ParquetInputFormat<ArrayWritable>(DataWritableReadSupport.class));
  }

  protected MapredParquetInputFormat(final ParquetInputFormat<ArrayWritable> inputFormat) {
    this.realInput = inputFormat;
    vectorizedSelf = new VectorizedParquetInputFormat();
  }

Otherwise, we can actually consider refactoring directly into

  public HoodieParquetInputFormat() {
    super(new HudiAvroParquetInputFormat());
  }

@lucasmo
Copy link

lucasmo commented Jan 20, 2022

I was directed to add a comment here from hudi slack.

Our team us experimenting with MOR tables. Our write ecosystem is AWS Glue and our query ecosystem for this use case is AWS Athena. Our writes are working fine. However, when querying with Athena, we get the following error:

GENERIC_INTERNAL_ERROR: org.apache.hadoop.io.LongWritable cannot be cast to org.apache.hadoop.hive.serde2.io.TimestampWritable

This error occurs if we choose to select any timestamp field. Selecting only non-timestamp fields works correctly. We searched and found no working resolution.

Our table looks like this:

CREATE EXTERNAL TABLE `foo_table_mor`(
  `foo_id` bigint, 
  `foo_timestamp` timestamp, 
  `_hoodie_commit_time` string, 
  `_hoodie_commit_seqno` string, 
  `_hoodie_record_key` string, 
  `_hoodie_partition_path` string, 
  `_hoodie_file_name` string)
ROW FORMAT SERDE 
  'org.apache.hadoop.hive.ql.io.parquet.serde.ParquetHiveSerDe' 
STORED AS INPUTFORMAT 
  'org.apache.hudi.hadoop.realtime.HoodieParquetRealtimeInputFormat' 
OUTPUTFORMAT 
  'org.apache.hadoop.hive.ql.io.parquet.MapredParquetOutputFormat'
LOCATION
  's3://foo-bucket/foo-prefix/'

@cdmikechen
Copy link
Contributor Author

@lucasmo
You can try this pr, but it looks like there are some conflicts after I push this commit. I will resolve the conflicts later.

@XuQianJin-Stars
Copy link
Contributor

hi @cdmikechen can rebase this pr?

@cdmikechen cdmikechen force-pushed the HUDI-83 branch 2 times, most recently from 9361bbe to f9b524a Compare May 4, 2022 04:14
@cdmikechen cdmikechen changed the title [HUDI-83] Fix Timestamp type read by Hive [HUDI-83] Fix Timestamp/Date type read by Hive3 May 10, 2022
@cdmikechen
Copy link
Contributor Author

cdmikechen commented May 10, 2022

@XuQianJin-Stars
I have a question about whether a hive3 pipeline task should be added to deal with some compatibility problems between hive2 and hive3 (and future hive4 support) ?

@cdmikechen cdmikechen force-pushed the HUDI-83 branch 5 times, most recently from e41458b to 689164e Compare May 13, 2022 12:30
@cdmikechen
Copy link
Contributor Author

@hudi-bot run azure

@cdmikechen
Copy link
Contributor Author

@XuQianJin-Stars
Hi~ You can review the codes when you have time, please.
I found that there were some unknown errors in the recent CI. After several rebase / merge, the current CI has passed.

.defaultValue("false")
.defaultValue("true")
.withDocumentation("‘INT64’ with original type TIMESTAMP_MICROS is converted to hive ‘timestamp’ type. "
+ "Disabled by default for backward compatibility.");
Copy link
Contributor

Choose a reason for hiding this comment

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

withDocumentation‘s content is need change also?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

@XuQianJin-Stars
Can this shows whether we can clearly explain the PR?

'INT64' with original type TIMESTAMP_MICROS is converted to hive 'timestamp' type. From 0.12.0, 'timestamp' type will be supported and also can be disabled by this variable. Previous versions keep being disabled by default.

Copy link
Contributor

Choose a reason for hiding this comment

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

@XuQianJin-Stars Can this shows whether we can clearly explain the PR?

'INT64' with original type TIMESTAMP_MICROS is converted to hive 'timestamp' type. From 0.12.0, 'timestamp' type will be supported and also can be disabled by this variable. Previous versions keep being disabled by default.

In deprecatedAfter method write version 0.12.0 and change withDocumentation‘s content?

Copy link
Contributor Author

@cdmikechen cdmikechen May 15, 2022

Choose a reason for hiding this comment

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

@XuQianJin-Stars Can this shows whether we can clearly explain the PR?
'INT64' with original type TIMESTAMP_MICROS is converted to hive 'timestamp' type. From 0.12.0, 'timestamp' type will be supported and also can be disabled by this variable. Previous versions keep being disabled by default.

In deprecatedAfter method write version 0.12.0 and change withDocumentation‘s content?

@XuQianJin-Stars
Is this right?

  public static final ConfigProperty<String> HIVE_SUPPORT_TIMESTAMP_TYPE = ConfigProperty
      .key("hoodie.datasource.hive_sync.support_timestamp")
      .defaultValue("true")
      .deprecatedAfter("0.12.0")
      .withDocumentation("'INT64' with original type TIMESTAMP_MICROS is converted to hive 'timestamp' type. "
          + "From 0.12.0, 'timestamp' type will be supported and also can be disabled by this variable. "
          + "Previous versions keep being disabled by default.");

If there's no problem, I'll change all the other descriptions.

@hudi-bot
Copy link
Collaborator

CI report:

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

@danny0405
Copy link
Contributor

Seems the PR is good to land, can you resolve the conflicts.


import static org.apache.parquet.hadoop.ParquetInputFormat.getFilter;

public class HoodieAvroParquetReader extends RecordReader<Void, ArrayWritable> {
Copy link
Contributor

Choose a reason for hiding this comment

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

@cdmikechen can you please help me understand why we need custom ParquetReader?

@bschell
Copy link
Contributor

bschell commented Feb 18, 2023

Hi, I am testing this fix backported to emr-6.9.0 with the reproduction steps linked by @lucasmo above. But I think something is not working right.

CREATE EXTERNAL TABLE `hudi_test`(
    `_hoodie_commit_time` string COMMENT '', 
    `_hoodie_commit_seqno` string COMMENT '', 
    `_hoodie_record_key` string COMMENT '', 
    `_hoodie_partition_path` string COMMENT '', 
    `_hoodie_file_name` string COMMENT '', 
    `id` string COMMENT '', 
    `tstamp` timestamp COMMENT '') 
ROW FORMAT SERDE 
    'org.apache.hadoop.hive.ql.io.parquet.serde.ParquetHiveSerDe' 
STORED AS INPUTFORMAT 
    'org.apache.hudi.hadoop.HoodieParquetInputFormat' 
OUTPUTFORMAT 
    'org.apache.hadoop.hive.ql.io.parquet.MapredParquetOutputFormat'
LOCATION 's3://{s3 path}'

I am still getting the same error

java.io.IOException: org.apache.hadoop.hive.ql.metadata.HiveException: java.lang.ClassCastException: org.apache.hadoop.io.LongWritable cannot be cast to org.apache.hadoop.hive.serde2.io.TimestampWritableV2

@veenaypatil
Copy link
Contributor

We are still seeing this issue with version 0.12.2 for timestamp column, is there a workaround today ? setting hoodie.datasource.hive_sync.support_timestamp=true does not work as well.
cc @codope @vinothchandar

@danny0405
Copy link
Contributor

Should be fixed in #7173.

@cdmikechen cdmikechen closed this Jun 6, 2023
@cdmikechen
Copy link
Contributor Author

@danny0405 @xicm
I'm sorry that I didn't make it to the end, but I'm glad to see that the problem was finally fixed.
Thanks all your efforts and the assistance of others~

@xicm
Copy link
Contributor

xicm commented Jun 6, 2023

Thanks @cdmikechen , you did most of the work.

@splate
Copy link

splate commented Jun 28, 2023

For the rest of us, can you clarify which jars are impacted (like hudi-spark-bundle* ?) and follow up with when this fix is released in an official version? Sorry if this is a dumb question, but I have this issue and this thread does not tell me how to solve my problem.

@danny0405
Copy link
Contributor

It should be the hudi-hadoop-mr jar, which is used by Hive.

@splate
Copy link

splate commented Jun 29, 2023

Would this bug also exist in the spark hudi libraries used in AWS glue? My issue is I am trying to use Spark SQL to query a hudi table and put it into a spark dataframe. I am getting a casting exception ("java.lang.ClassCastException: org.apache.hadoop.io.LongWritable cannot be cast to org.apache.hadoop.hive.serde2.io.TimestampWritable"). Would this be related to this?

@danny0405
Copy link
Contributor

Should be, but it is more related with how the timestamp type is synced I think: #8867

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

Labels

area:schema Schema evolution and data types priority:high Significant impact; potential bugs

Projects

Archived in project

Development

Successfully merging this pull request may close these issues.