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

Handle Date type in HCatToRow #32695

Merged
merged 3 commits into from
Oct 9, 2024
Merged

Handle Date type in HCatToRow #32695

merged 3 commits into from
Oct 9, 2024

Conversation

deadb0d4
Copy link
Contributor

@deadb0d4 deadb0d4 commented Oct 8, 2024

This addresses #20685.

Implementation notes:

  1. Manually cast hive Date values to Instant's in HCatToRow.java per my understanding of damccorm's suggestions in the issue above.
  2. I test this currently in testReadHCatalogDateType by writing a test hcatalog with a date column, reading it then, and converting the HCatRecord collection to a Row collection.

I aimed to make the least invasive change, since the issue appears to be specific to hive.

Some other notes:

  • The issue original backtrace deals with java.sql.Date, which I wasn't able to reproduce fully. I could only write hcatalog with org.apache.hadoop.hive.common.type.Date, so I'm not really sure if we wanna account for java.sql.Date case as well.
  • On this note, 267f76f changed the code involved so that there's a direct cast to AbstractInstant in RowUtils.java. This doesn't change much, but jfyi.

Checklist to help us incorporate your contribution quickly and easily:

  • Mention the appropriate issue in your description
  • Update CHANGES.md with noteworthy changes (I don't think it's that noteworthy)
  • If this contribution is large... (it's not that big currently)

Build python source distribution and wheels
Python tests
Java tests
Go tests

See CI.md for more information about GitHub Actions CI or the workflows README to see a list of phrases to trigger workflows.

Some initial notes:
- The issue (apache#20685) deals with java.sql.Date, which I wasn't able to
  reproduce fully (I can currently write hcatalog hadoop.hive date)
- On this note, 267f76f changed the
  code involved so that there's a direct cast to AbstractInstant in
  RowUtils.java. This doesn't change much, but jfyi.
Copy link
Contributor

github-actions bot commented Oct 8, 2024

Assigning reviewers. If you would like to opt out of this review, comment assign to next reviewer:

R: @kennknowles for label java.
R: @johnjcasey for label io.

Available commands:

  • stop reviewer notifications - opt out of the automated review tooling
  • remind me after tests pass - tag the comment author after tests pass
  • waiting on author - shift the attention set back to the author (any comment or push by the author will return the attention set to the reviewers)

The PR bot will only process comments in the main thread (not review comments).

HCatToRowFn(Schema schema) {
this.schema = schema;
}

@ProcessElement
public void processElement(ProcessContext c) {
HCatRecord hCatRecord = c.element();
c.output(Row.withSchema(schema).addValues(hCatRecord.getAll()).build());
List<Object> recordValues =
hCatRecord.getAll().stream().map(this::castHDate).collect(Collectors.toList());
Copy link
Contributor

Choose a reason for hiding this comment

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

Shall we generalize the function, e.g. just naming as "castTypes", leaving space for future improvement if other type need a conversion

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Hey, thanks for the suggestion! I moved this whole logic to a separate new util castTypes. Please let me know if you simply meant s/castHDate/castTypes though!

@Abacn
Copy link
Contributor

Abacn commented Oct 9, 2024

how to format java code (fwiw, this wiki seems to be missing the command)

I simply use ./gradlew spotlessApply which should format all code (may take time). If you just changed HCataglogIO, could use ./gradlew :sdks:java:io:hcatalog:spotlessApply

That wiki is about configure git command to automatically format code before push. Personally I didn't do this config so I am not sure.

- s/castHDate/maybeCastHDate/ to be more concise
- move values manipulation to a separate util (hopefully, I understood
  the cr in the right way)
@deadb0d4
Copy link
Contributor Author

deadb0d4 commented Oct 9, 2024

I simply use ./gradlew spotlessApply

Nice, thanks so much! I found this command, but forgot to update the PR description.

Copy link
Contributor

@Abacn Abacn left a comment

Choose a reason for hiding this comment

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

thank you!

@Abacn Abacn merged commit 2ee6100 into apache:master Oct 9, 2024
19 checks passed
reeba212 pushed a commit to reeba212/beam that referenced this pull request Dec 4, 2024
* Handle Date type in HCatToRow

Some initial notes:
- The issue (apache#20685) deals with java.sql.Date, which I wasn't able to
  reproduce fully (I can currently write hcatalog hadoop.hive date)
- On this note, 267f76f changed the
  code involved so that there's a direct cast to AbstractInstant in
  RowUtils.java. This doesn't change much, but jfyi.

* Run: ./gradlew :sdks:java:io:hcatalog:spotlessApply

* review cr: castTypes util

- s/castHDate/maybeCastHDate/ to be more concise
- move values manipulation to a separate util (hopefully, I understood
  the cr in the right way)
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Projects
None yet
Development

Successfully merging this pull request may close these issues.

2 participants