-
Notifications
You must be signed in to change notification settings - Fork 2.4k
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
[HUDI-1302] Add support for timestamp field in HiveSync #2129
Conversation
Can you test on Hive to check if Hive can read data with hudi avro timestamp type? |
Yes, additional changes are needed in Hive. For short term, we made a temporary patch to Hive to make it work (@s-sanjay plans to open a draft PR soon). |
e6b74e8
to
86965e6
Compare
@pratyakshsharma will you be able to review this week? |
@@ -68,6 +68,9 @@ | |||
@Parameter(names = {"--help", "-h"}, help = true) | |||
public Boolean help = false; | |||
|
|||
@Parameter(names = {"--support-timestamp"}, description = "If true, converts int64(timestamp_micros) to timestamp type") | |||
public Boolean supportTimestamp = false; |
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.
I think we need to add this option into DataSourceOptions, DataSourceUtils, and HoodieSparkSqlWriter
something like?
"hoodie.datasource.hive_sync.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.
@bschell thank you for pointing this out. We only use standalone mode, so I missed this DataSourceOptions. Fixed now. PTAL.
86965e6
to
e0e1ae8
Compare
Apologies for the delay. Will take a look today. |
cc @umehrot2 as well. |
@@ -167,7 +173,10 @@ private static String convertField(final Type parquetType) { | |||
.append(decimalMetadata.getScale()).append(")").toString(); | |||
} else if (originalType == OriginalType.DATE) { | |||
return field.append("DATE").toString(); | |||
} else if (supportTimestamp && originalType == OriginalType.TIMESTAMP_MICROS) { |
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.
Could you please explain what happens now if the originalType == OriginalType.TIMESTAMP_MICROS
?
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.
@n3nash without this change, we get 'bigint' for hive type (goes into parquetPrimitiveTypeName.convert -> convertINT64 method)
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.
@satishkotha Changes looks fine, left 1 comment based on which we could have some possibility of refactoring
@pratyakshsharma @n3nash Please take a look. |
@n3nash @pratyakshsharma @bschell can any of you review this? |
@satishkotha |
Changes look good to me. @n3nash Do you have any concerns here? |
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.
LGTM
What is the purpose of the pull request
Add support for timestamp field in HiveSync
Brief change log
Verify this pull request
This change added tests. Also verified with steps in docker demo. Testing in staging in progress.
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.