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

[HUDI-1969] Support reading logs for MOR Hive rt table #3033

Merged
merged 1 commit into from
Jul 14, 2021

Conversation

danny0405
Copy link
Contributor

Tips

What is the purpose of the pull request

(For example: This pull request adds quick-start document.)

Brief change log

(for example:)

  • Modify AnnotationLocation checkstyle rule in checkstyle.xml

Verify this pull request

(Please pick either of the following options)

This pull request is a trivial rework / code cleanup without any test coverage.

(or)

This pull request is already covered by existing tests, such as (please describe tests).

(or)

This change added tests and can be verified as follows:

(example:)

  • Added integration tests for end-to-end.
  • Added HoodieClientWriteTest to verify the change.
  • Manually verified the change by running a job locally.

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.

@codecov-commenter
Copy link

codecov-commenter commented Jun 4, 2021

Codecov Report

Merging #3033 (c80dca3) into master (8c0dbaa) will increase coverage by 0.02%.
The diff coverage is 65.85%.

Impacted file tree graph

@@             Coverage Diff              @@
##             master    #3033      +/-   ##
============================================
+ Coverage     47.62%   47.64%   +0.02%     
- Complexity     5502     5505       +3     
============================================
  Files           930      930              
  Lines         41274    41279       +5     
  Branches       4137     4136       -1     
============================================
+ Hits          19657    19668      +11     
+ Misses        19868    19862       -6     
  Partials       1749     1749              
Flag Coverage Δ
hudicli 39.97% <ø> (ø)
hudiclient 34.58% <ø> (ø)
hudicommon 48.57% <ø> (ø)
hudiflink 59.58% <ø> (ø)
hudihadoopmr 51.76% <65.85%> (+0.47%) ⬆️
hudisparkdatasource 67.32% <ø> (ø)
hudisync 54.48% <ø> (ø)
huditimelineservice 64.07% <ø> (ø)
hudiutilities 58.57% <ø> (ø)

Flags with carried forward coverage won't be shown. Click here to find out more.

Impacted Files Coverage Δ
...hadoop/realtime/RealtimeCompactedRecordReader.java 76.81% <65.85%> (+11.18%) ⬆️
...apache/hudi/utilities/deltastreamer/DeltaSync.java 71.14% <0.00%> (ø)

Continue to review full report at Codecov.

Legend - Click here to learn more
Δ = absolute <relative> (impact), ø = not affected, ? = missing data
Powered by Codecov. Last update 8c0dbaa...c80dca3. Read the comment docs.

@vinothchandar vinothchandar added this to Ready for Review in PR Tracker Board Jun 7, 2021
@danny0405
Copy link
Contributor Author

Hi @vinothchandar , can you take a look for this, i found that there was a TODO here, and HUDI has global index now and we can remove the TODO and support reading logs.

@danny0405
Copy link
Contributor Author

@n3nash Can you take a look, this is important to our flink hive users, thanks in advance.

@hudi-bot
Copy link

hudi-bot commented Jun 17, 2021

CI report:

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

@vinothchandar vinothchandar self-assigned this Jun 19, 2021
@vinothchandar
Copy link
Member

@danny0405 on it. Going to spend the next week, fully in reviews. So we will move this along

@danny0405
Copy link
Contributor Author

@danny0405 on it. Going to spend the next week, fully in reviews. So we will move this along

The pure logs file group was never supported before, should we support that in this PR ?

@vinothchandar
Copy link
Member

@danny0405 I see less reasons for this outside of HDFS, which can actually take appends. For cloud storage, wrtiing a base file is probably better and now we also have things like clustering which can then control file sizing. So we need not rely only on compaction for e.g

@danny0405
Copy link
Contributor Author

@danny0405 I see less reasons for this outside of HDFS, which can actually take appends. For cloud storage, wrtiing a base file is probably better and now we also have things like clustering which can then control file sizing. So we need not rely only on compaction for e.g

Thanks @vinothchandar , i know that maybe the cloud storage is in major for North America users, but in China, most of the people still use the HDFS yet, and they still use Hive as the query engine most often, from the perspective of functional integrity, i think we should support it instead of throwing a confusing exception or ignoring the logs silently.

@vinothchandar
Copy link
Member

ok makes sense.

@vinothchandar
Copy link
Member

Marked this a release blocker, so we can get it into the release.

@@ -155,7 +155,7 @@ public void testReader(boolean partitioned) throws Exception {
} else {
writer =
InputFormatTestUtil.writeDataBlockToLogFile(partitionDir, fs, schema, "fileid0", baseInstant,
instantTime, 100, 0, logVersion);
instantTime, 120, 0, logVersion);
Copy link
Member

Choose a reason for hiding this comment

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

can you explain why this was needed? This test was already adding inserts into logs? and now with this change we can actually read this.?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

I added 20 more records because the first 100 log records was expected to merge with the initial 100 parquet records, so after merge, we should read out 120 records actually.

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.

Just one comment. The logic seems correct. Can we add tests that explicitly send inserts to logs and see if its all returned correctly

@vinothchandar vinothchandar moved this from Ready for Review to Nearing Landing in PR Tracker Board Jul 8, 2021
@danny0405
Copy link
Contributor Author

danny0405 commented Jul 8, 2021

Just one comment. The logic seems correct. Can we add tests that explicitly send inserts to logs and see if its all returned correctly

Thanks @vinothchandar , before this patch , we assume that each file group has a base parquet fie PLUS logs with only UPDATE records, this is true for BloomFilter index, but with global index like flink, the logs are not only UPDATEs but can also be INSERTs. So the new logic is to read the parquet first then the logs.

The modified test actually tests the case, with first 100 log records to update the old one then another 20 insert records for pure INSERTs.

While this patch does not solve the case to read pure logs file group, because after debugging, i found that this input format found the file groups based on the visible parquet first, the logs are hidden file and be ignored.

I have add one line comment to the test case.

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.

LGTM

@vinothchandar vinothchandar merged commit ac75bda into apache:master Jul 14, 2021
PR Tracker Board automation moved this from Nearing Landing to Done Jul 14, 2021
Samrat002 pushed a commit to Samrat002/hudi that referenced this pull request Jul 15, 2021
change the insret overwrte return type

[HUDI-1860] Test wrapper for insert_overwrite and insert_overwrite_table

[HUDI-2084] Resend the uncommitted write metadata when start up (apache#3168)

Co-authored-by: 喻兆靖 <yuzhaojing@bilibili.com>

[HUDI-2081] Move schema util tests out from TestHiveSyncTool (apache#3166)

[HUDI-2094] Supports hive style partitioning for flink writer (apache#3178)

[HUDI-2097] Fix Flink unable to read commit metadata error (apache#3180)

[HUDI-2085] Support specify compaction paralleism and compaction target io for flink batch compaction (apache#3169)

[HUDI-2092] Fix NPE caused by FlinkStreamerConfig#writePartitionUrlEncode null value (apache#3176)

[HUDI-2006] Adding more yaml templates to test suite (apache#3073)

[HUDI-2103] Add rebalance before index bootstrap (apache#3185)

Co-authored-by: 喻兆靖 <yuzhaojing@bilibili.com>

[HUDI-1944] Support Hudi to read from committed offset (apache#3175)

* [HUDI-1944] Support Hudi to read from committed offset

* [HUDI-1944] Adding group option to KafkaResetOffsetStrategies

* [HUDI-1944] Update Exception msg

[HUDI-2052] Support load logFile in BootstrapFunction (apache#3134)

Co-authored-by: 喻兆靖 <yuzhaojing@bilibili.com>

[HUDI-89] Add configOption & refactor all configs based on that (apache#2833)

Co-authored-by: Wenning Ding <wenningd@amazon.com>

[MINOR] Update .asf.yaml to codify notification settings, turn on jira comments, gh discussions (apache#3164)

- Turn on comment for jira, so we can track PR activity better
- Create a notification settings that match https://gitbox.apache.org/schemes.cgi?hudi
- Try and turn on "discussions" on Github, to experiment

[MINOR] Fix broken build due to FlinkOptions (apache#3198)

[HUDI-2088] Missing Partition Fields And PreCombineField In Hoodie Properties For Table Written By Flink (apache#3171)

[MINOR] Add Documentation to KEYGENERATOR_TYPE_PROP (apache#3196)

[HUDI-2105] Compaction Failed For MergeInto MOR Table (apache#3190)

[HUDI-2051] Enable Hive Sync When Spark Enable Hive Meta  For Spark Sql (apache#3126)

[HUDI-2112] Support reading pure logs file group for flink batch reader after compaction (apache#3202)

[HUDI-2114] Spark Query MOR Table Written By Flink Return Incorrect Timestamp Value (apache#3208)

[HUDI-2121] Add operator uid for flink stateful operators (apache#3212)

[HUDI-2123]  Exception When Merge With Null-Value Field (apache#3214)

[HUDI-2124] A Grafana dashboard for HUDI. (apache#3216)

[HUDI-2057]  CTAS Generate An External Table When Create Managed Table (apache#3146)

[HUDI-1930] Bootstrap support configure KeyGenerator by type (apache#3170)

* [HUDI-1930] Bootstrap support configure KeyGenerator by type

[HUDI-2116] Support batch synchronization of partition datas to  hive metastore to avoid oom problem (apache#3209)

[HUDI-2126] The coordinator send events to write function when there are no data for the checkpoint (apache#3219)

[HUDI-2127] Initialize the maxMemorySizeInBytes in log scanner (apache#3220)

[HUDI-2058]support incremental query for insert_overwrite_table/insert_overwrite operation on cow table (apache#3139)

[HUDI-2129] StreamerUtil.medianInstantTime should return a valid date time string (apache#3221)

[HUDI-2131] Exception Throw Out When MergeInto With Decimal Type Field (apache#3224)

[HUDI-2122] Improvement in packaging insert into smallfiles (apache#3213)

[HUDI-2132] Make coordinator events as POJO for efficient serialization (apache#3223)

[HUDI-2106] Fix flink batch compaction bug while user don't set compaction tasks (apache#3192)

[HUDI-2133] Support hive1 metadata sync for flink writer (apache#3225)

[HUDI-2089]fix the bug that metatable cannot support non_partition table (apache#3182)

[HUDI-2028] Implement RockDbBasedMap as an alternate to DiskBasedMap in ExternalSpillableMap (apache#3194)

Co-authored-by: Rajesh Mahindra <rmahindra@Rajeshs-MacBook-Pro.local>

[HUDI-2135] Add compaction schedule option for flink (apache#3226)

[HUDI-2055] Added deltastreamer metric for time of lastSync (apache#3129)

[HUDI-2046] Loaded too many classes like sun/reflect/GeneratedSerializationConstructorAccessor in JVM metaspace (apache#3121)

Loaded too many classes when use kryo of spark to hudi

Co-authored-by: weiwei.duan <weiwei.duan@linkflowtech.com>

[HUDI-1996] Adding functionality to allow the providing of basic auth creds for confluent cloud schema registry (apache#3097)

* adding support for basic auth with confluent cloud schema registry

[HUDI-2093] Fix empty avro schema path caused by duplicate parameters (apache#3177)

* [HUDI-2093] Fix empty avro schema path caused by duplicate parameters

* rename shcmea option key

* fix doc

* rename var name

[HUDI-2113] Fix integration testing failure caused by sql results out of order (apache#3204)

[HUDI-2016] Fixed bootstrap of Metadata Table when some actions are in progress. (apache#3083)

Metadata Table cannot be bootstrapped when any action is in progress. This is detected by the presence of inflight or requested instants. The bootstrapping is initiated in preWrite and postWrite of each commit. So bootstrapping will be retried again until it succeeds.
Also added metrics for when the bootstrapping fails or a table is re-bootstrapped. This will help detect tables which are not getting bootstrapped.

[HUDI-2140] Fixed the unit test TestHoodieBackedMetadata.testOnlyValidPartitionsAdded. (apache#3234)

[HUDI-2115] FileSlices in the filegroup is not descending by timestamp (apache#3206)

[HUDI-1104] Adding support for UserDefinedPartitioners and SortModes to BulkInsert with Rows (apache#3149)

[HUDI-2069] Refactored String constants (apache#3172)

[HUDI-1105] Adding dedup support for Bulk Insert w/ Rows (apache#2206)

[HUDI-2134]Add generics to avoif forced conversion in BaseSparkCommitActionExecutor#partition (apache#3232)

[HUDI-2009] Fixing extra commit metadata in row writer path (apache#3075)

[HUDI-2099]hive lock which state is WATING should be released, otherwise this hive lock will be locked forever (apache#3186)

[MINOR] Fix build broken from apache#3186 (apache#3245)

[HUDI-2136] Fix conflict when flink-sql-connector-hive and hudi-flink-bundle are both in flink lib (apache#3227)

[HUDI-2087] Support Append only in Flink stream (apache#3174)

Co-authored-by: 喻兆靖 <yuzhaojing@bilibili.com>

UnitTest for deltaSync

Removing cosmetic changes and reuse function for insert_overwrite_table

unit test

intial unit test for the insert_overwrite and insert_over_write_table

Adding failed test code for insert_overwrite

Revert "[HUDI-2087] Support Append only in Flink stream (apache#3174)" (apache#3251)

This reverts commit 3715267.

[HUDI-2147] Remove unused class AvroConvertor in hudi-flink (apache#3243)

[MINOR] Fix some wrong assert reasons (apache#3248)

[HUDI-2087] Support Append only in Flink stream (apache#3252)

Co-authored-by: 喻兆靖 <yuzhaojing@bilibili.com>

[HUDI-2143] Tweak the default compaction target IO to 500GB when flink async compaction is off (apache#3238)

[HUDI-2142] Support setting bucket assign parallelism for flink write task (apache#3239)

[HUDI-1483] Support async clustering for deltastreamer and Spark streaming (apache#3142)

- Integrate async clustering service with HoodieDeltaStreamer and HoodieStreamingSink
- Added methods in HoodieAsyncService to reuse code

[HUDI-2107] Support Read Log Only MOR Table For Spark (apache#3193)

[HUDI-2144]Bug-Fix:Offline clustering(HoodieClusteringJob) will cause insert action losing data (apache#3240)

* fixed

* add testUpsertPartitionerWithSmallFileHandlingAndClusteringPlan ut

* fix CheckStyle

Co-authored-by: yuezhang <yuezhang@freewheel.tv>

[MINOR] Fix EXTERNAL_RECORD_AND_SCHEMA_TRANSFORMATION config (apache#3250)

[HUDI-2168] Fix for AccessControlException for anonymous user (apache#3264)

[HUDI-2045] Support Read Hoodie As DataSource Table For Flink And DeltaStreamer

test with insert-overwrite and insert-overwrite-table

removing hardcoded action to pass the unit test

[HUDI-1969] Support reading logs for MOR Hive rt table (apache#3033)

[HUDI-2171] Add parallelism conf for bootstrap operator

using delta-commit for insert_overwrite
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Projects
Development

Successfully merging this pull request may close these issues.

None yet

4 participants