-
Notifications
You must be signed in to change notification settings - Fork 2.5k
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-1722]hive beeline/spark-sql query specified field on mor table occur NPE #2722
Conversation
test step: val df = spark.range(0, 100000).toDF("keyid") // create hoodie table hive_14b merge(df, 4, "default", "hive_14b", DataSourceWriteOptions.MOR_TABLE_TYPE_OPT_VAL, op = "bulk_insert") notice: bulk_insert will produce 4 files in hoodie table step2: val df = spark.range(99999, 100002).toDF("keyid") // upsert table merge(df, 4, "default", "hive_14b", DataSourceWriteOptions.MOR_TABLE_TYPE_OPT_VAL, op = "upsert") now : we have four base files and one log file in hoodie table step3: spark-sql/beeline: select count(col3) from hive_14b_rt; then the query failed. after patch: merge function: |
@garyli1019 could you pls help me review this pr, thanks |
@garyli1019 : replace "sev:critical" label with "sev:high" if applicable. Fix the corresponding jira as well if you had to change it. |
@xiarixiaoyao Thanks for your contribution. Looks like you are able to reproduce this problem in the unit test. Is that possible to add the unit test to this pr as well? |
thanks @garyli1019 . ok, i will try to add unit test |
6070c23
to
0a896f1
Compare
@garyli1019 unit test has added, pls review again, thanks |
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.
@xiarixiaoyao thanks for adding the test! @lw309637554 would you please review this PR as well?
hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/utils/HoodieRealtimeInputFormatUtils.java
Show resolved
Hide resolved
@xiarixiaoyao hello , will review it this weekend |
InputSplit[] splits = combineHiveInputFormat.getSplits(jobConf, 1); | ||
// Since the SPLIT_SIZE is 3, we should create only 1 split with all 3 file groups | ||
assertEquals(1, splits.length); | ||
RecordReader<NullWritable, ArrayWritable> recordReader = |
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.
hello , just see one recordreader?
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.
yes, we only create one combine recorder, but this recorder hold three RealtimeCompactedRecordReaders。
the creating order of those RealtimeCompactedRecordReaders lead this npe problem.
for test example:
combine recorder holds three RealtimeCompactedRecordReaders, we call them creader1, creader2, creader3
creader1: only has base file
creader2: only has base file
creader3: has base file and log file.
if creader3 is create firstly, hoodie additional projection columns will be added to jobConf and in this case the query will be ok
however if creader1 or creader2 is create firstly, no hoodie additional projection columns will be added to jobConf, the query will failed
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.
got it, thanks
...adoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/HoodieParquetRealtimeInputFormat.java
Outdated
Show resolved
Hide resolved
@xiarixiaoyao thanks for your contribution. Add the unit test is very necessary. Also the resolution left some comments. |
@lw309637554 thanks for you review. i have answered your questions, pls check them, thanks。 Another question: TestHoodieCombineHiveInputFormat.testHoodieRealtimeCombineHoodieInputFormat is disabled by default, i have checked that test function, and find there exists some problems. could i fix those problem and enable TestHoodieCombineHiveInputFormat.testHoodieRealtimeCombineHoodieInputFormat default |
@nsivabalan same over to you to get this ready for review. |
try it . |
@vinothchandar : I see that author is actively responding/working on the PR. Will leave it to the author to address feedback. If we don't see any activity for sometime, I can chime in. |
@lw309637554 @nsivabalan thanks for your review. i will try testHoodieRealtimeCombineHoodieInputFormat in another pr, since |
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
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.
@lw309637554 can you please let us know if you are okay with this change. This LGTM
synchronized (jobConf) { | ||
LOG.info( | ||
"Before adding Hoodie columns, Projections :" + jobConf.get(ColumnProjectionUtils.READ_COLUMN_NAMES_CONF_STR) | ||
+ ", Ids :" + jobConf.get(ColumnProjectionUtils.READ_COLUMN_IDS_CONF_STR)); | ||
if (jobConf.get(HoodieInputFormatUtils.HOODIE_READ_COLUMNS_PROP) == null) { | ||
if (jobConf.get(HoodieInputFormatUtils.HOODIE_READ_COLUMNS_PROP) == null | ||
|| (!realtimeSplit.getDeltaLogPaths().isEmpty() && !HoodieRealtimeInputFormatUtils.requiredProjectionFieldsExistInConf(jobConf))) { |
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.
can we pull this check into a small util method, that we can call in both places?
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.
yes, fixed
@xiarixiaoyao Could you please rebase this PR . I tried doing this myself, seems tricky |
@vinothchandar i will rebase this pr, thanks |
Codecov Report
@@ Coverage Diff @@
## master #2722 +/- ##
=========================================
Coverage 54.78% 54.78%
- Complexity 3808 3812 +4
=========================================
Files 481 481
Lines 23320 23326 +6
Branches 2488 2492 +4
=========================================
+ Hits 12775 12779 +4
- Misses 9390 9392 +2
Partials 1155 1155
Flags with carried forward coverage won't be shown. Click here to find out more.
|
@vinothchandar , i have rebased this pr pls check them, thanks |
@vinothchandar @xiarixiaoyao LGTM. |
Tips
What is the purpose of the pull request
Fix the bug introduced by HUDI-892 HUDI-892 introduce this problem。
this pr skip adding projection columns if there are no log files in the hoodieRealtimeSplit。 but this pr donnot consider that multiple getRecordReaders share same jobConf。
Consider the following questions:
we have four getRecordReaders:
now reader1 run first, HoodieInputFormatUtils.HOODIE_READ_COLUMNS_PROP in jobConf will be set to be true, and no hoodie additional projection columns will be added to jobConf (see HoodieParquetRealtimeInputFormat.addProjectionToJobConf)
reader2 run later, since HoodieInputFormatUtils.HOODIE_READ_COLUMNS_PROP in jobConf is set to be true, no hoodie additional projection columns will be added to jobConf. (see HoodieParquetRealtimeInputFormat.addProjectionToJobConf)
which lead to the result that _hoodie_record_key would be missing and merge step would throw exceptions
Obviously, this is an occasional problem。 if reader2 run first, hoodie additional projection columns will be added to jobConf and in this case the query will be ok
Brief change log
(for example:)
Verify this pull request
This pull request is already covered by existing tests
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.