Skip to content

Conversation

@SteNicholas
Copy link
Member

@SteNicholas SteNicholas commented Sep 24, 2020

What is the purpose of the pull request

When Insert operation is chosen, it could be no-op in trying to merge w/ existing data file. So, this patch introduces HoodieConcatHandle which will just append/concat new incoming data to existing data for a given data file w/o performing any merge(combineAndUpdate) as such. Its users responsibility to ensure there are no duplicate if the config is enabled with Insert operation.

Brief change log

  • Added HoodieConcatHandle to concat/append incoming records to existing records for a given data file w/o performing any merge for Insert operation.
  • Added a new config hoodie.merge.allow.duplicate.inserts to guard this new behavior.

Verify this pull request

  • Added tests to TestHoodieClientOnCopyOnWriteStorage(testInsertsWithHoodieConcatHandle and testInsertsPreppedWithHoodieConcatHandle) to verify new behavior

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.

@SteNicholas
Copy link
Member Author

@leesf @bvaradar Could you please review this pull request?

@leesf
Copy link
Contributor

leesf commented Sep 25, 2020

@SteNicholas would you please also add some tests to the new changes?

@leesf leesf self-assigned this Sep 26, 2020
@linshan-ma
Copy link
Contributor

According to this(https://github.com/apache/hudi/issues/2051) test。I can't get the results I want。When we set different value(hoodie.parquet.small.file.limit), the results are still different

@leesf
Copy link
Contributor

leesf commented Sep 29, 2020

According to this(https://github.com/apache/hudi/issues/2051) test。I can't get the results I want。When we set different value(hoodie.parquet.small.file.limit), the results are still different

@linshan-ma hi, you mean that after you apply this PR and insert records again, the results written before get updated, instead of inserting new records?

@linshan-ma
Copy link
Contributor

@leesf yes,it was updated.When we set hoodie.parquet.small.file.limit=0, it was inserted

@SteNicholas SteNicholas requested a review from leesf September 29, 2020 08:31
@linshan-ma
Copy link
Contributor

According to this(https://github.com/apache/hudi/issues/2051) test。I can't get the results I want。When we set different value(hoodie.parquet.small.file.limit), the results are still different

@linshan-ma Could you please provide this test again? I couldn't visit the test you mentioned.

@SteNicholas hi,this issue.#2051

@SteNicholas SteNicholas force-pushed the upsert-partitioner-insert branch 7 times, most recently from 3872213 to 3ce6e02 Compare October 2, 2020 14:53
@SteNicholas
Copy link
Member Author

According to this(https://github.com/apache/hudi/issues/2051) test。I can't get the results I want。When we set different value(hoodie.parquet.small.file.limit), the results are still different

@linshan-ma Could you please provide this test again? I couldn't visit the test you mentioned.

@SteNicholas hi,this issue.#2051

@linshan-ma You could use the latest change to verify the test case you provide.

Copy link
Contributor

@leesf leesf left a comment

Choose a reason for hiding this comment

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

@SteNicholas Thanks for your contribution, left some comments.

@SteNicholas SteNicholas force-pushed the upsert-partitioner-insert branch from 70a6777 to cd244bb Compare October 15, 2020 07:40
@codecov-io
Copy link

codecov-io commented Oct 15, 2020

Codecov Report

Merging #2111 into master will increase coverage by 1.14%.
The diff coverage is 100.00%.

Impacted file tree graph

@@             Coverage Diff              @@
##             master    #2111      +/-   ##
============================================
+ Coverage     52.47%   53.62%   +1.14%     
- Complexity     2748     2848     +100     
============================================
  Files           352      359       +7     
  Lines         16129    16553     +424     
  Branches       1752     1780      +28     
============================================
+ Hits           8464     8876     +412     
+ Misses         6937     6918      -19     
- Partials        728      759      +31     
Flag Coverage Δ Complexity Δ
#hudicli 38.37% <ø> (ø) 193.00 <ø> (ø)
#hudiclient 100.00% <ø> (ø) 0.00 <ø> (ø)
#hudicommon 54.73% <100.00%> (-0.01%) 1795.00 <1.00> (+2.00) ⬇️
#hudihadoopmr 33.05% <ø> (ø) 181.00 <ø> (ø)
#hudispark 65.42% <ø> (-1.76%) 304.00 <ø> (-7.00)
#huditimelineservice 62.29% <ø> (?) 50.00 <ø> (?)
#hudiutilities 70.07% <ø> (+10.51%) 325.00 <ø> (+55.00)

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

Impacted Files Coverage Δ Complexity Δ
...g/apache/hudi/common/model/WriteOperationType.java 53.84% <100.00%> (ø) 2.00 <1.00> (ø)
...n/java/org/apache/hudi/internal/DefaultSource.java 0.00% <0.00%> (-80.00%) 0.00% <0.00%> (-6.00%)
...pache/hudi/internal/HoodieWriterCommitMessage.java 83.33% <0.00%> (-16.67%) 2.00% <0.00%> (-1.00%)
.../hudi/internal/HoodieDataSourceInternalWriter.java 78.12% <0.00%> (-9.38%) 6.00% <0.00%> (-2.00%)
...n/scala/org/apache/hudi/HoodieSparkSqlWriter.scala 50.95% <0.00%> (-5.26%) 0.00% <0.00%> (ø%)
.../main/java/org/apache/hudi/common/util/Option.java 66.66% <0.00%> (-3.61%) 23.00% <0.00%> (+1.00%) ⬇️
...in/scala/org/apache/hudi/IncrementalRelation.scala 76.19% <0.00%> (-2.30%) 20.00% <0.00%> (+1.00%) ⬇️
...udi/utilities/sources/helpers/DFSPathSelector.java 82.05% <0.00%> (-2.16%) 12.00% <0.00%> (ø%)
...mmon/table/log/AbstractHoodieLogRecordScanner.java 79.68% <0.00%> (-1.92%) 29.00% <0.00%> (ø%)
...src/main/java/org/apache/hudi/DataSourceUtils.java 44.76% <0.00%> (-0.60%) 22.00% <0.00%> (+1.00%) ⬇️
... and 27 more

@SteNicholas
Copy link
Member Author

@SteNicholas Thanks for your contribution, left some comments.

Thanks for your comments. I have already followed up with your comments. Please feel free to review again.

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.

@nsivabalan nsivabalan changed the title [HUDI-1234] Insert new records regardless of small file when using insert operation [HUDI-1234] Insert new records to data files without merging with "Insert" operation. Jan 20, 2021
@nsivabalan nsivabalan changed the title [HUDI-1234] Insert new records to data files without merging with "Insert" operation. [HUDI-1234] Insert new records to data files without merging for "Insert" operation. Jan 20, 2021
Copy link
Contributor

Choose a reason for hiding this comment

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

@vinothchandar : does the config name looks ok? somehow I feel it does not exactly convey the actual intent, but talks about the effect. For instance,
something like "hoodie.concat.inserts.without.merge.on.insert" or "hoodie.concat.records.on.insert" etc.

Copy link
Member

Choose a reason for hiding this comment

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

first one is too long IMO. I think hoodie.merge.allow.duplicates.on.insert is a better name (I added the .on) given it explains to the user what the impact is. Whether we concat or prepend in implementation, thats not very useful for the end user

@nsivabalan nsivabalan force-pushed the upsert-partitioner-insert branch from e574133 to 23c6ce3 Compare January 20, 2021 15:17
@nsivabalan nsivabalan force-pushed the upsert-partitioner-insert branch from 23c6ce3 to 287a016 Compare January 20, 2021 15:21
@nsivabalan
Copy link
Contributor

Fixed the config name. Once CI succeeds, will land this PR.

@SteNicholas
Copy link
Member Author

Fixed the config name. Once CI succeeds, will land this PR.

Thanks for @nsivabalan updates. I'm sorry for the delay for this pull request because of busy work. Thanks again.

@vinothchandar
Copy link
Member

@nsivabalan I thought we were going to get this in to 0.7.0? checked back again, to see why this was missing

@nsivabalan
Copy link
Contributor

yeah, for some reason integration test was failing in CI and we had some flakiness or long queuing time w/ travis. So, I put this in my backlog and missed to get it in. Will check it out in a day or two.

@nsivabalan nsivabalan merged commit 2ee1c3f into apache:master Jan 27, 2021
prashantwason pushed a commit to prashantwason/incubator-hudi that referenced this pull request Aug 5, 2021
…ert" operation. (apache#2111)

* Added HoodieConcatHandle to skip merging for "insert" operation when the corresponding config is set

Co-authored-by: Sivabalan Narayanan <[email protected]>
prashantwason added a commit to prashantwason/incubator-hudi that referenced this pull request Aug 5, 2021
…OSS master

Summary:
[HUDI-1509]: Reverting LinkedHashSet changes to combine fields from oldSchema and newSchema in favor of using only new schema for record rewriting (apache#2424)
[MINOR] Bumping snapshot version to 0.7.0 (apache#2435)
[HUDI-1533] Make SerializableSchema work for large schemas and add ability to sortBy numeric values (apache#2453)
[HUDI-1529] Add block size to the FileStatus objects returned from metadata table to avoid too many file splits (apache#2451)
[HUDI-1532] Fixed suboptimal implementation of a magic sequence search  (apache#2440)
[HUDI-1535] Fix 0.7.0 snapshot (apache#2456)
[MINOR] Fixing setting defaults for index config (apache#2457)
[HUDI-1540] Fixing commons codec shading in spark bundle (apache#2460)
[HUDI 1308] Harden RFC-15 Implementation based on production testing (apache#2441)
[MINOR] Remove redundant judgments (apache#2466)
[MINOR] Fix dataSource cannot use hoodie.datasource.hive_sync.auto_create_database (apache#2444)
[MINOR] Disabling problematic tests temporarily to stabilize CI (apache#2468)
[MINOR] Make a separate travis CI job for hudi-utilities (apache#2469)
[HUDI-1512] Fix spark 2 unit tests failure with Spark 3 (apache#2412)
[HUDI-1511] InstantGenerateOperator support multiple parallelism (apache#2434)
[HUDI-1332] Introduce FlinkHoodieBloomIndex to hudi-flink-client (apache#2375)
[HUDI] Add bloom index for hudi-flink-client
[MINOR] Remove InstantGeneratorOperator parallelism limit in HoodieFlinkStreamer and update docs (apache#2471)
[MINOR] Improve code readability,remove the continue keyword (apache#2459)
[HOTFIX] Revert upgrade flink verison to 1.12.0 (apache#2473)
[HUDI-1453] Fix NPE using HoodieFlinkStreamer to etl data from kafka to hudi (apache#2474)
[MINOR] Use skipTests flag for skip.hudi-spark2.unit.tests property (apache#2477)
[HUDI-1476] Introduce unit test infra for java client (apache#2478)
[MINOR] Update doap with 0.7.0 release (apache#2491)
[MINOR]Fix NPE when using HoodieFlinkStreamer with multi parallelism (apache#2492)
[HUDI-1234] Insert new records to data files without merging for "Insert" operation.  (apache#2111)
[MINOR] Add Jira URL and Mailing List (apache#2404)
[HUDI-1522] Add a new pipeline for Flink writer (apache#2430)
[HUDI-1522] Add a new pipeline for Flink writer
[HUDI-623] Remove UpgradePayloadFromUberToApache (apache#2455)
[HUDI-1555] Remove isEmpty to improve clustering execution performance (apache#2502)
[HUDI-1266] Add unit test for validating replacecommit rollback (apache#2418)
[MINOR] Quickstart.generateUpdates method add check (apache#2505)
[HUDI-1519] Improve minKey/maxKey computation in HoodieHFileWriter (apache#2427)
[HUDI-1550] Honor ordering field for MOR Spark datasource reader (apache#2497)
[MINOR] Fix method comment typo (apache#2518)
[MINOR] Rename FileSystemViewHandler to RequestHandler and corrected the class comment (apache#2458)
[HUDI-1335] Introduce FlinkHoodieSimpleIndex to hudi-flink-client (apache#2271)
[HUDI-1523] Call mkdir(partition) only if not exists (apache#2501)
[HUDI-1538] Try to init class trying different signatures instead of checking its name (apache#2476)
[HUDI-1538] Try to init class trying different signatures instead of checking its name.
[HUDI-1547] CI intermittent failure: TestJsonStringToHoodieRecordMapF… (apache#2521)
[MINOR] Fixing the default value for source ordering field for payload config (apache#2516)
[HUDI-1420] HoodieTableMetaClient.getMarkerFolderPath works incorrectly on windows client with hdfs server for wrong file seperator (apache#2526)
[HUDI-1571] Adding commit_show_records_info to display record sizes for commit (apache#2514)
[HUDI-1589] Fix Rollback Metadata AVRO backwards incompatiblity (apache#2543)
[MINOR] Fix wrong logic for checking state condition (apache#2524)
[HUDI-1557] Make Flink write pipeline write task scalable (apache#2506)
[HUDI-1545] Add test cases for INSERT_OVERWRITE Operation (apache#2483)
[HUDI-1603] fix DefaultHoodieRecordPayload serialization failure (apache#2556)
[MINOR] Fix the wrong comment for HoodieJavaWriteClientExample (apache#2559)
[HUDI-1526] Translate the api partitionBy in spark datasource to hoodie.datasource.write.partitionpath.field (apache#2431)
[HUDI-1612] Fix write test flakiness in StreamWriteITCase (apache#2567)
[HUDI-1612] Fix write test flakiness in StreamWriteITCase
[MINOR] Default to empty list for unset datadog tags property (apache#2574)
[MINOR] Add clustering to feature list (apache#2568)
[HUDI-1598] Write as minor batches during one checkpoint interval for the new writer (apache#2553)
[HUDI-1109] Support Spark Structured Streaming read from Hudi table (apache#2485)
[HUDI-1621] Gets the parallelism from context when init StreamWriteOperatorCoordinator (apache#2579)
[HUDI-1381] Schedule compaction based on time elapsed (apache#2260)
[HUDI-1582] Throw an exception when syncHoodieTable() fails, with RuntimeException (apache#2536)
[HUDI-1539] Fix bug in HoodieCombineRealtimeRecordReader with reading empty iterators (apache#2583)
[HUDI-1315] Adding builder for HoodieTableMetaClient initialization (apache#2534)
[HUDI-1486] Remove inline inflight rollback in hoodie writer (apache#2359)
[HUDI-1586] [Common Core] [Flink Integration] Reduce the coupling of hadoop. (apache#2540)
[HUDI-1624] The state based index should bootstrap from existing base files (apache#2581)
[HUDI-1477] Support copyOnWriteTable in java client (apache#2382)
[MINOR] Ensure directory exists before listing all marker files. (apache#2594)
[MINOR] hive sync checks for table after creating db if auto create is true (apache#2591)
[HUDI-1620] Add azure pipelines configs (apache#2582)
[HUDI-1347] Fix Hbase index to make rollback synchronous (via config) (apache#2188)
[HUDI-1637] Avoid to rename for bucket update when there is only one flush action during a checkpoint (apache#2599)
[HUDI-1638] Some improvements to BucketAssignFunction (apache#2600)
[HUDI-1367] Make deltaStreamer transition from dfsSouce to kafkasouce (apache#2227)
[HUDI-1269] Make whether the failure of connect hive affects hudi ingest process configurable (apache#2443)
[HUDI-1611] Added a configuration to allow specific directories to be filtered out during Metadata Table bootstrap. (apache#2565)
[Hudi-1583]: Fix bug that Hudi will skip remaining log files if there is logFile with zero size in logFileList when merge on read. (apache#2584)
[HUDI-1632] Supports merge on read write mode for Flink writer (apache#2593)
[HUDI-1540] Fixing commons codec dependency in bundle jars (apache#2562)
[HUDI-1644] Do not delete older rollback instants as part of rollback. Archival can take care of removing old instants cleanly (apache#2610)
[HUDI-1634] Re-bootstrap metadata table when un-synced instants have been archived. (apache#2595)
[HUDI-1584] Modify maker file path, which should start with the target base path. (apache#2539)
[MINOR] Fix default value for hoodie.deltastreamer.source.kafka.auto.reset.offsets (apache#2617)
[HUDI-1553] Configuration and metrics for the TimelineService. (apache#2495)
[HUDI-1587] Add latency and freshness support (apache#2541)
[HUDI-1647] Supports snapshot read for Flink (apache#2613)
[HUDI-1646] Provide mechanism to read uncommitted data through InputFormat (apache#2611)
[HUDI-1655] Support custom date format and fix unsupported exception in DatePartitionPathSelector (apache#2621)
[HUDI-1636] Support Builder Pattern To Build Table Properties For HoodieTableConfig (apache#2596)
[HUDI-1660] Excluding compaction and clustering instants from inflight rollback (apache#2631)
[HUDI-1661] Exclude clustering commits from getExtraMetadataFromLatest API (apache#2632)
[MINOR] Fix import in StreamerUtil.java (apache#2638)
[HUDI-1618] Fixing NPE with Parquet src in multi table delta streamer (apache#2577)
[HUDI-1662] Fix hive date type conversion for mor table (apache#2634)
[HUDI-1673] Replace scala.Tule2 to Pair in FlinkHoodieBloomIndex (apache#2642)
[MINOR] HoodieClientTestHarness close resources in AfterAll phase (apache#2646)
[HUDI-1635] Improvements to Hudi Test Suite (apache#2628)
[HUDI-1651] Fix archival of requested replacecommit (apache#2622)
[HUDI-1663] Streaming read for Flink MOR table (apache#2640)
[HUDI-1678] Row level delete for Flink sink (apache#2659)
[HUDI-1664] Avro schema inference for Flink SQL table (apache#2658)
[HUDI-1681] Support object storage for Flink writer (apache#2662)
[HUDI-1685] keep updating current date for every batch (apache#2671)
[HUDI-1496] Fixing input stream detection of GCS FileSystem (apache#2500)
[HUDI-1684] Tweak hudi-flink-bundle module pom and reorganize the pacakges for hudi-flink module (apache#2669)
[HUDI-1692] Bounded source for stream writer (apache#2674)
[HUDI-1552] Improve performance of key lookups from base file in Metadata Table. (apache#2494)
[HUDI-1552] Improve performance of key lookups from base file in Metadata Table.
[HUDI-1695] Fixed the error messaging (apache#2679)
[HUDI 1615] Fixing null schema in bulk_insert row writer path  (apache#2653)
[HUDI-845] Added locking capability to allow multiple writers (apache#2374)
[HUDI-1701] Implement HoodieTableSource.explainSource for all kinds of pushing down (apache#2690)
[HUDI-1704] Use PRIMARY KEY syntax to define record keys for Flink Hudi table (apache#2694)
[HUDI-1688]hudi write should uncache rdd, when the write operation is finnished (apache#2673)
[MINOR] Remove unused var in AbstractHoodieWriteClient (apache#2693)
[HUDI-1653] Add support for composite keys in NonpartitionedKeyGenerator (apache#2627)
[HUDI-1705] Flush as per data bucket for mini-batch write (apache#2695)
[1568] Fixing spark3 bundles (apache#2625)
[HUDI-1650] Custom avro kafka deserializer. (apache#2619)
[HUDI-1667]: Fix a null value related bug for spark vectorized reader. (apache#2636)
[HUDI-1709] Improving config names and adding hive metastore uri config (apache#2699)
[MINOR][DOCUMENT] Update README doc for integ test (apache#2703)
[HUDI-1710] Read optimized query type for Flink batch reader (apache#2702)
[HUDI-1712] Rename & standardize config to match other configs (apache#2708)
[hotfix] Log the error message for creating table source first (apache#2711)
[HUDI-1495] Bump Flink version to 1.12.2 (apache#2718)
[HUDI-1728] Fix MethodNotFound for HiveMetastore Locks (apache#2731)
[HUDI-1478] Introduce HoodieBloomIndex to hudi-java-client (apache#2608)
[HUDI-1729] Asynchronous Hive sync and commits cleaning for Flink writer (apache#2732)
[HOTFIX] close spark session in functional test suite and disable spark3 test for spark2 (apache#2727)
[HOTFIX] Disable ITs for Spark3 and scala2.12 (apache#2733)
[HOTFIX] fix deploy staging jars script
[MINOR] Add Missing Apache License to test files (apache#2736)
[UBER] Fixed creation of HoodieMetadataClient which now uses a Builder pattern instead of a constructor.

Reviewers: balajee, O955 Project Hoodie Project Reviewer: Add blocking reviewers!, PHID-PROJ-pxfpotkfgkanblb3detq!

JIRA Issues: HUDI-593

Differential Revision: https://code.uberinternal.com/D5867129
prashantwason added a commit to prashantwason/incubator-hudi that referenced this pull request Aug 5, 2021
…OSS master

Summary:
[HUDI-1509]: Reverting LinkedHashSet changes to combine fields from oldSchema and newSchema in favor of using only new schema for record rewriting (apache#2424)
[MINOR] Bumping snapshot version to 0.7.0 (apache#2435)
[HUDI-1533] Make SerializableSchema work for large schemas and add ability to sortBy numeric values (apache#2453)
[HUDI-1529] Add block size to the FileStatus objects returned from metadata table to avoid too many file splits (apache#2451)
[HUDI-1532] Fixed suboptimal implementation of a magic sequence search  (apache#2440)
[HUDI-1535] Fix 0.7.0 snapshot (apache#2456)
[MINOR] Fixing setting defaults for index config (apache#2457)
[HUDI-1540] Fixing commons codec shading in spark bundle (apache#2460)
[HUDI 1308] Harden RFC-15 Implementation based on production testing (apache#2441)
[MINOR] Remove redundant judgments (apache#2466)
[MINOR] Fix dataSource cannot use hoodie.datasource.hive_sync.auto_create_database (apache#2444)
[MINOR] Disabling problematic tests temporarily to stabilize CI (apache#2468)
[MINOR] Make a separate travis CI job for hudi-utilities (apache#2469)
[HUDI-1512] Fix spark 2 unit tests failure with Spark 3 (apache#2412)
[HUDI-1511] InstantGenerateOperator support multiple parallelism (apache#2434)
[HUDI-1332] Introduce FlinkHoodieBloomIndex to hudi-flink-client (apache#2375)
[HUDI] Add bloom index for hudi-flink-client
[MINOR] Remove InstantGeneratorOperator parallelism limit in HoodieFlinkStreamer and update docs (apache#2471)
[MINOR] Improve code readability,remove the continue keyword (apache#2459)
[HOTFIX] Revert upgrade flink verison to 1.12.0 (apache#2473)
[HUDI-1453] Fix NPE using HoodieFlinkStreamer to etl data from kafka to hudi (apache#2474)
[MINOR] Use skipTests flag for skip.hudi-spark2.unit.tests property (apache#2477)
[HUDI-1476] Introduce unit test infra for java client (apache#2478)
[MINOR] Update doap with 0.7.0 release (apache#2491)
[MINOR]Fix NPE when using HoodieFlinkStreamer with multi parallelism (apache#2492)
[HUDI-1234] Insert new records to data files without merging for "Insert" operation.  (apache#2111)
[MINOR] Add Jira URL and Mailing List (apache#2404)
[HUDI-1522] Add a new pipeline for Flink writer (apache#2430)
[HUDI-1522] Add a new pipeline for Flink writer
[HUDI-623] Remove UpgradePayloadFromUberToApache (apache#2455)
[HUDI-1555] Remove isEmpty to improve clustering execution performance (apache#2502)
[HUDI-1266] Add unit test for validating replacecommit rollback (apache#2418)
[MINOR] Quickstart.generateUpdates method add check (apache#2505)
[HUDI-1519] Improve minKey/maxKey computation in HoodieHFileWriter (apache#2427)
[HUDI-1550] Honor ordering field for MOR Spark datasource reader (apache#2497)
[MINOR] Fix method comment typo (apache#2518)
[MINOR] Rename FileSystemViewHandler to RequestHandler and corrected the class comment (apache#2458)
[HUDI-1335] Introduce FlinkHoodieSimpleIndex to hudi-flink-client (apache#2271)
[HUDI-1523] Call mkdir(partition) only if not exists (apache#2501)
[HUDI-1538] Try to init class trying different signatures instead of checking its name (apache#2476)
[HUDI-1538] Try to init class trying different signatures instead of checking its name.
[HUDI-1547] CI intermittent failure: TestJsonStringToHoodieRecordMapF… (apache#2521)
[MINOR] Fixing the default value for source ordering field for payload config (apache#2516)
[HUDI-1420] HoodieTableMetaClient.getMarkerFolderPath works incorrectly on windows client with hdfs server for wrong file seperator (apache#2526)
[HUDI-1571] Adding commit_show_records_info to display record sizes for commit (apache#2514)
[HUDI-1589] Fix Rollback Metadata AVRO backwards incompatiblity (apache#2543)
[MINOR] Fix wrong logic for checking state condition (apache#2524)
[HUDI-1557] Make Flink write pipeline write task scalable (apache#2506)
[HUDI-1545] Add test cases for INSERT_OVERWRITE Operation (apache#2483)
[HUDI-1603] fix DefaultHoodieRecordPayload serialization failure (apache#2556)
[MINOR] Fix the wrong comment for HoodieJavaWriteClientExample (apache#2559)
[HUDI-1526] Translate the api partitionBy in spark datasource to hoodie.datasource.write.partitionpath.field (apache#2431)
[HUDI-1612] Fix write test flakiness in StreamWriteITCase (apache#2567)
[HUDI-1612] Fix write test flakiness in StreamWriteITCase
[MINOR] Default to empty list for unset datadog tags property (apache#2574)
[MINOR] Add clustering to feature list (apache#2568)
[HUDI-1598] Write as minor batches during one checkpoint interval for the new writer (apache#2553)
[HUDI-1109] Support Spark Structured Streaming read from Hudi table (apache#2485)
[HUDI-1621] Gets the parallelism from context when init StreamWriteOperatorCoordinator (apache#2579)
[HUDI-1381] Schedule compaction based on time elapsed (apache#2260)
[HUDI-1582] Throw an exception when syncHoodieTable() fails, with RuntimeException (apache#2536)
[HUDI-1539] Fix bug in HoodieCombineRealtimeRecordReader with reading empty iterators (apache#2583)
[HUDI-1315] Adding builder for HoodieTableMetaClient initialization (apache#2534)
[HUDI-1486] Remove inline inflight rollback in hoodie writer (apache#2359)
[HUDI-1586] [Common Core] [Flink Integration] Reduce the coupling of hadoop. (apache#2540)
[HUDI-1624] The state based index should bootstrap from existing base files (apache#2581)
[HUDI-1477] Support copyOnWriteTable in java client (apache#2382)
[MINOR] Ensure directory exists before listing all marker files. (apache#2594)
[MINOR] hive sync checks for table after creating db if auto create is true (apache#2591)
[HUDI-1620] Add azure pipelines configs (apache#2582)
[HUDI-1347] Fix Hbase index to make rollback synchronous (via config) (apache#2188)
[HUDI-1637] Avoid to rename for bucket update when there is only one flush action during a checkpoint (apache#2599)
[HUDI-1638] Some improvements to BucketAssignFunction (apache#2600)
[HUDI-1367] Make deltaStreamer transition from dfsSouce to kafkasouce (apache#2227)
[HUDI-1269] Make whether the failure of connect hive affects hudi ingest process configurable (apache#2443)
[HUDI-1611] Added a configuration to allow specific directories to be filtered out during Metadata Table bootstrap. (apache#2565)
[Hudi-1583]: Fix bug that Hudi will skip remaining log files if there is logFile with zero size in logFileList when merge on read. (apache#2584)
[HUDI-1632] Supports merge on read write mode for Flink writer (apache#2593)
[HUDI-1540] Fixing commons codec dependency in bundle jars (apache#2562)
[HUDI-1644] Do not delete older rollback instants as part of rollback. Archival can take care of removing old instants cleanly (apache#2610)
[HUDI-1634] Re-bootstrap metadata table when un-synced instants have been archived. (apache#2595)
[HUDI-1584] Modify maker file path, which should start with the target base path. (apache#2539)
[MINOR] Fix default value for hoodie.deltastreamer.source.kafka.auto.reset.offsets (apache#2617)
[HUDI-1553] Configuration and metrics for the TimelineService. (apache#2495)
[HUDI-1587] Add latency and freshness support (apache#2541)
[HUDI-1647] Supports snapshot read for Flink (apache#2613)
[HUDI-1646] Provide mechanism to read uncommitted data through InputFormat (apache#2611)
[HUDI-1655] Support custom date format and fix unsupported exception in DatePartitionPathSelector (apache#2621)
[HUDI-1636] Support Builder Pattern To Build Table Properties For HoodieTableConfig (apache#2596)
[HUDI-1660] Excluding compaction and clustering instants from inflight rollback (apache#2631)
[HUDI-1661] Exclude clustering commits from getExtraMetadataFromLatest API (apache#2632)
[MINOR] Fix import in StreamerUtil.java (apache#2638)
[HUDI-1618] Fixing NPE with Parquet src in multi table delta streamer (apache#2577)
[HUDI-1662] Fix hive date type conversion for mor table (apache#2634)
[HUDI-1673] Replace scala.Tule2 to Pair in FlinkHoodieBloomIndex (apache#2642)
[MINOR] HoodieClientTestHarness close resources in AfterAll phase (apache#2646)
[HUDI-1635] Improvements to Hudi Test Suite (apache#2628)
[HUDI-1651] Fix archival of requested replacecommit (apache#2622)
[HUDI-1663] Streaming read for Flink MOR table (apache#2640)
[HUDI-1678] Row level delete for Flink sink (apache#2659)
[HUDI-1664] Avro schema inference for Flink SQL table (apache#2658)
[HUDI-1681] Support object storage for Flink writer (apache#2662)
[HUDI-1685] keep updating current date for every batch (apache#2671)
[HUDI-1496] Fixing input stream detection of GCS FileSystem (apache#2500)
[HUDI-1684] Tweak hudi-flink-bundle module pom and reorganize the pacakges for hudi-flink module (apache#2669)
[HUDI-1692] Bounded source for stream writer (apache#2674)
[HUDI-1552] Improve performance of key lookups from base file in Metadata Table. (apache#2494)
[HUDI-1552] Improve performance of key lookups from base file in Metadata Table.
[HUDI-1695] Fixed the error messaging (apache#2679)
[HUDI 1615] Fixing null schema in bulk_insert row writer path  (apache#2653)
[HUDI-845] Added locking capability to allow multiple writers (apache#2374)
[HUDI-1701] Implement HoodieTableSource.explainSource for all kinds of pushing down (apache#2690)
[HUDI-1704] Use PRIMARY KEY syntax to define record keys for Flink Hudi table (apache#2694)
[HUDI-1688]hudi write should uncache rdd, when the write operation is finnished (apache#2673)
[MINOR] Remove unused var in AbstractHoodieWriteClient (apache#2693)
[HUDI-1653] Add support for composite keys in NonpartitionedKeyGenerator (apache#2627)
[HUDI-1705] Flush as per data bucket for mini-batch write (apache#2695)
[1568] Fixing spark3 bundles (apache#2625)
[HUDI-1650] Custom avro kafka deserializer. (apache#2619)
[HUDI-1667]: Fix a null value related bug for spark vectorized reader. (apache#2636)
[HUDI-1709] Improving config names and adding hive metastore uri config (apache#2699)
[MINOR][DOCUMENT] Update README doc for integ test (apache#2703)
[HUDI-1710] Read optimized query type for Flink batch reader (apache#2702)
[HUDI-1712] Rename & standardize config to match other configs (apache#2708)
[hotfix] Log the error message for creating table source first (apache#2711)
[HUDI-1495] Bump Flink version to 1.12.2 (apache#2718)
[HUDI-1728] Fix MethodNotFound for HiveMetastore Locks (apache#2731)
[HUDI-1478] Introduce HoodieBloomIndex to hudi-java-client (apache#2608)
[HUDI-1729] Asynchronous Hive sync and commits cleaning for Flink writer (apache#2732)
[HOTFIX] close spark session in functional test suite and disable spark3 test for spark2 (apache#2727)
[HOTFIX] Disable ITs for Spark3 and scala2.12 (apache#2733)
[HOTFIX] fix deploy staging jars script
[MINOR] Add Missing Apache License to test files (apache#2736)
[UBER] Fixed creation of HoodieMetadataClient which now uses a Builder pattern instead of a constructor.

Reviewers: balajee

Reviewed By: balajee

JIRA Issues: HUDI-593

Differential Revision: https://code.uberinternal.com/D5867129
@codecov-commenter
Copy link

codecov-commenter commented Sep 5, 2024

Codecov Report

All modified and coverable lines are covered by tests ✅

Project coverage is 50.44%. Comparing base (a38612b) to head (287a016).
Report is 4350 commits behind head on master.

Additional details and impacted files

Impacted file tree graph

@@             Coverage Diff              @@
##             master    #2111      +/-   ##
============================================
- Coverage     50.69%   50.44%   -0.25%     
- Complexity     3059     3062       +3     
============================================
  Files           419      419              
  Lines         18810    18897      +87     
  Branches       1924     1937      +13     
============================================
- Hits           9535     9533       -2     
- Misses         8498     8587      +89     
  Partials        777      777              
Flag Coverage Δ
hudicli 37.21% <ø> (-0.06%) ⬇️
hudiclient ∅ <ø> (∅)
hudicommon 51.47% <ø> (-0.46%) ⬇️
hudiflink 10.20% <ø> (ø)
hudihadoopmr 33.16% <ø> (+0.09%) ⬆️
hudisparkdatasource 65.85% <ø> (-0.05%) ⬇️
hudisync 48.61% <ø> (ø)
huditimelineservice 66.49% <ø> (-0.35%) ⬇️
hudiutilities 69.43% <ø> (-0.06%) ⬇️

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

see 4 files with indirect coverage changes

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

Labels

priority:blocker Production down; release blocker

Projects

None yet

Development

Successfully merging this pull request may close these issues.

8 participants