Skip to content

Conversation

@danny0405
Copy link
Contributor

What is the purpose of the pull request

This is the #step 2 of RFC-24:
https://cwiki.apache.org/confluence/display/HUDI/RFC+-+24%3A+Hoodie+Flink+Writer+Proposal

This PR introduces a BucketAssigner that assigns bucket ID (partition
path & fileID) for each stream record.

There is no need to look up index and partition the records anymore in
the following pipeline for these records,
we actually decide the write target location before the write and each
record computes its location when the BucketAssigner receives it, thus,
the indexing is with streaming style.

Computing locations for a batch of records all at a time is resource
consuming so a pressure to the engine,
we should avoid that in streaming system.

Brief change log

  • Add BucketAssigner
  • Modify the behavior of HoodieFlinkWriteClient and BaseFlinkCommitActionExecutor

Verify this pull request

Added 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.

@codecov-io
Copy link

codecov-io commented Jan 28, 2021

Codecov Report

Merging #2506 (5129df4) into master (bc0325f) will increase coverage by 0.67%.
The diff coverage is 70.19%.

Impacted file tree graph

@@             Coverage Diff              @@
##             master    #2506      +/-   ##
============================================
+ Coverage     50.26%   50.94%   +0.67%     
- Complexity     3119     3167      +48     
============================================
  Files           430      433       +3     
  Lines         19565    19788     +223     
  Branches       2004     2031      +27     
============================================
+ Hits           9835    10081     +246     
+ Misses         8925     8887      -38     
- Partials        805      820      +15     
Flag Coverage Δ Complexity Δ
hudicli 37.21% <ø> (ø) 0.00 <ø> (ø)
hudiclient 100.00% <ø> (ø) 0.00 <ø> (ø)
hudicommon 51.38% <0.00%> (-0.12%) 0.00 <0.00> (ø)
hudiflink 43.21% <71.31%> (+10.17%) 0.00 <39.00> (ø)
hudihadoopmr 33.16% <ø> (ø) 0.00 <ø> (ø)
hudisparkdatasource 69.46% <ø> (+3.60%) 0.00 <ø> (ø)
hudisync 48.61% <ø> (ø) 0.00 <ø> (ø)
huditimelineservice 66.49% <ø> (ø) 0.00 <ø> (ø)
hudiutilities 69.46% <ø> (+0.03%) 0.00 <ø> (ø)

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

Impacted Files Coverage Δ Complexity Δ
...di/common/table/timeline/HoodieActiveTimeline.java 70.64% <0.00%> (-1.33%) 42.00 <0.00> (ø)
.../apache/hudi/operator/InstantGenerateOperator.java 0.00% <0.00%> (ø) 0.00 <0.00> (ø)
...pache/hudi/operator/KeyedWriteProcessFunction.java 0.00% <0.00%> (ø) 0.00 <0.00> (ø)
.../org/apache/hudi/operator/StreamWriteOperator.java 0.00% <0.00%> (ø) 0.00 <0.00> (ø)
...ache/hudi/operator/StreamWriteOperatorFactory.java 0.00% <0.00%> (ø) 0.00 <0.00> (ø)
.../org/apache/hudi/streamer/FlinkStreamerConfig.java 0.00% <ø> (ø) 0.00 <0.00> (ø)
.../org/apache/hudi/streamer/HoodieFlinkStreamer.java 0.00% <0.00%> (ø) 0.00 <0.00> (ø)
...rg/apache/hudi/streamer/HoodieFlinkStreamerV2.java 0.00% <0.00%> (ø) 0.00 <0.00> (ø)
...ache/hudi/operator/partitioner/BucketAssigner.java 80.00% <80.00%> (ø) 18.00 <18.00> (?)
...c/main/java/org/apache/hudi/util/StreamerUtil.java 43.68% <82.35%> (+11.13%) 13.00 <3.00> (+4.00)
... and 21 more

@yanghua
Copy link
Contributor

yanghua commented Jan 29, 2021

@danny0405 ping us, when you are ready to review.

@danny0405
Copy link
Contributor Author

@danny0405 ping us, when you are ready to review.

@yanghua @garyli1019 Yes, this PR is ready to review, thanks for the review if you have time.

Copy link
Contributor

@yanghua yanghua left a comment

Choose a reason for hiding this comment

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

@danny0405 Have a quick look, left some comments.

Copy link
Contributor

Choose a reason for hiding this comment

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

Can we use getter/setter pattern instead of using public fields?

Copy link
Contributor

Choose a reason for hiding this comment

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

ditto

Copy link
Contributor

Choose a reason for hiding this comment

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

@wangxianghu Do you agree this?

Copy link
Contributor

Choose a reason for hiding this comment

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

@danny0405 if we do not query the location, how to identify the operation per record is insert or update

Copy link
Contributor Author

Choose a reason for hiding this comment

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

See the document of BucketAssignerFunction.

Copy link
Member

Choose a reason for hiding this comment

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

nit: Can we add some comments about the indexing here and pointing to BucketAssignerFunction? This will be helpful for others to read the code. It's fine now, we can do this on your next PR.

Copy link
Contributor

Choose a reason for hiding this comment

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

String#isEmpty

Copy link
Contributor

Choose a reason for hiding this comment

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

IMO, FlinkOptions.BASE_PATH is more readable?

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 want to keep it sync with Spark data source, e.g. org.apache.hudi.DefaultSource line 62.

Copy link
Contributor

Choose a reason for hiding this comment

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

Extract it to be a constant.

Copy link
Contributor

Choose a reason for hiding this comment

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

More information to be provided? e.g. basePath...?

Copy link
Contributor

@yanghua yanghua left a comment

Choose a reason for hiding this comment

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

@danny0405 Left some comments.

Copy link
Contributor

Choose a reason for hiding this comment

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

What's the difference between CreateHandleFactory and FlinkCreateHandleFactory ? It seems they are the same.

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 said the reason in the document, they are different in rolling over new files, FlinkCreateHandleFactory always uses the file handle name that we specified and never roll over with a number suffix.

Copy link
Contributor

Choose a reason for hiding this comment

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

“No need”, but the method name is still updateIndex.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Rename to setUpWriteMetadata.

Copy link
Contributor

Choose a reason for hiding this comment

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

Can we define a context object e.g. excutorContext to put the information? IMO, extract general information from data is not a good design.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

We can, but this needs refactoring to the executor interface, it's sad that i have no time for that now ~, we can fire an issue for a future work.

Copy link
Contributor

Choose a reason for hiding this comment

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

OK, so file a Jira issue to track this work?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Copy link
Contributor

Choose a reason for hiding this comment

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

Can we define another flag to host it or put it into another place?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

We can when we refactor the writer interface to take a specified bucket type, currently this is enough, it has the full functionality but does not bring in extra burden to the engine.

Copy link
Contributor

Choose a reason for hiding this comment

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

File an issue?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Copy link
Contributor

Choose a reason for hiding this comment

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

It seems handleUpsertPartition is enough?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Yes.

Copy link
Member

Choose a reason for hiding this comment

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

IIUC the tagLocation was moved to here?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Yes, you are right ~

Copy link
Contributor

@yanghua yanghua left a comment

Choose a reason for hiding this comment

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

@danny0405 Left some comments.

Copy link
Contributor

Choose a reason for hiding this comment

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

This method is called in many places in one flink job. Do we really call it many times?

If yes, from sematics, rename it to be initTableIfNotExist or something like initTableOnDemand?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Rename to initTableIfNotExists.

Copy link
Contributor

Choose a reason for hiding this comment

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

EXPECTED1?

Copy link
Contributor

Choose a reason for hiding this comment

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

If it's a UT, let us follow the naming rule: TestXXX

Copy link
Contributor

Choose a reason for hiding this comment

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

Align with the old style of Java's class doc in the project looks better.

Copy link
Contributor

Choose a reason for hiding this comment

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

All the same rows? Why not build them via a loop?

Copy link
Contributor

Choose a reason for hiding this comment

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

Does this config cause any problem? AFAIK, it copied from deltastreamer.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

The path does not exists and it throws when fetch the properties, use empty string as the default to skip using empty properties too.

Copy link
Contributor

Choose a reason for hiding this comment

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

Align the comment style.

Copy link
Contributor

Choose a reason for hiding this comment

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

BucketAssignFunction sounds better?

@yanghua
Copy link
Contributor

yanghua commented Feb 4, 2021

@garyli1019 @wangxianghu Any review input? Let us land this PR at the end time towards tomorrow night?

@yanghua
Copy link
Contributor

yanghua commented Feb 5, 2021

@danny0405 Can you check CI?

@danny0405
Copy link
Contributor Author

@danny0405 Can you check CI?

Yes, i checked the CI exception and it seems that the exception has no relationship with this PR.

This is the #step 2 of RFC-24:
https://cwiki.apache.org/confluence/display/HUDI/RFC+-+24%3A+Hoodie+Flink+Writer+Proposal

This PR introduce a BucketAssigner that assigns bucket ID (partition
path & fileID) for each stream record.

There is no need to look up index and partition the records anymore in
the following pipeline for these records,
we actually decide the write target location before the write and each
record computes its location when the BucketAssigner receives it, thus,
the indexing is with streaming style.

Computing locations for a batch of records all at a time is resource
consuming so a pressure to the engine,
we should avoid that in streaming system.
@danny0405
Copy link
Contributor Author

Force push to re-trigger the CI tests.

@yanghua
Copy link
Contributor

yanghua commented Feb 6, 2021

@garyli1019 Will land this PR towards tomorrow morning, any concerns please let us know!

Copy link
Contributor

@yanghua yanghua left a comment

Choose a reason for hiding this comment

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

LGTM, let us refactor some minor issues later.

Copy link
Member

@garyli1019 garyli1019 left a comment

Choose a reason for hiding this comment

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

@garyli1019 Will land this PR towards tomorrow morning, any concerns please let us know!

Sorry about the delay. Too busy on other stuff. Please feel free to land if this PR looks good to you. @danny0405 Thanks for your contribution, as always!

Copy link
Member

Choose a reason for hiding this comment

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

nit: Can we add some comments about the indexing here and pointing to BucketAssignerFunction? This will be helpful for others to read the code. It's fine now, we can do this on your next PR.

@yanghua yanghua merged commit 4c5b692 into apache:master Feb 6, 2021
prashantwason pushed a commit to prashantwason/incubator-hudi that referenced this pull request Aug 5, 2021
This is the #step 2 of RFC-24:
https://cwiki.apache.org/confluence/display/HUDI/RFC+-+24%3A+Hoodie+Flink+Writer+Proposal

This PR introduce a BucketAssigner that assigns bucket ID (partition
path & fileID) for each stream record.

There is no need to look up index and partition the records anymore in
the following pipeline for these records,
we actually decide the write target location before the write and each
record computes its location when the BucketAssigner receives it, thus,
the indexing is with streaming style.

Computing locations for a batch of records all at a time is resource
consuming so a pressure to the engine,
we should avoid that in streaming system.
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
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Labels

None yet

Projects

None yet

Development

Successfully merging this pull request may close these issues.

5 participants