-
Notifications
You must be signed in to change notification settings - Fork 2.5k
[HUDI-1486] Remove inflight rollback in hoodie writer #2359
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-1486] Remove inflight rollback in hoodie writer #2359
Conversation
879ba2c to
b8c9cb6
Compare
9071832 to
f858d5d
Compare
Codecov Report
@@ Coverage Diff @@
## master #2359 +/- ##
============================================
+ Coverage 50.87% 51.15% +0.28%
- Complexity 3150 3216 +66
============================================
Files 430 438 +8
Lines 19614 20043 +429
Branches 2042 2065 +23
============================================
+ Hits 9978 10253 +275
- Misses 8806 8946 +140
- Partials 830 844 +14
Flags with carried forward coverage won't be shown. Click here to find out more.
|
f858d5d to
e45bf61
Compare
e45bf61 to
35a82c7
Compare
vinothchandar
left a comment
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.
Will review the cleaning/archival changes more closely again later today
hudi-cli/src/main/java/org/apache/hudi/cli/commands/FileSystemViewCommand.java
Outdated
Show resolved
Hide resolved
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/AbstractHoodieClient.java
Outdated
Show resolved
Hide resolved
...lient/hudi-client-common/src/main/java/org/apache/hudi/client/AbstractHoodieWriteClient.java
Outdated
Show resolved
Hide resolved
...lient/hudi-client-common/src/main/java/org/apache/hudi/client/AbstractHoodieWriteClient.java
Outdated
Show resolved
Hide resolved
...lient/hudi-client-common/src/main/java/org/apache/hudi/client/AbstractHoodieWriteClient.java
Outdated
Show resolved
Hide resolved
hudi-common/src/main/java/org/apache/hudi/common/table/HoodieTableMetaClient.java
Outdated
Show resolved
Hide resolved
hudi-common/src/main/java/org/apache/hudi/common/table/log/AbstractHoodieLogRecordScanner.java
Outdated
Show resolved
Hide resolved
hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieDefaultTimeline.java
Outdated
Show resolved
Hide resolved
hudi-common/src/test/java/org/apache/hudi/common/testutils/FileCreateUtils.java
Outdated
Show resolved
Hide resolved
hudi-hadoop-mr/src/test/java/org/apache/hudi/hadoop/TestHoodieROTablePathFilter.java
Outdated
Show resolved
Hide resolved
|
also, the writer should check if heartbeat has expired before commiting. |
vinothchandar
left a comment
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.
In general, I have a concern around compactions. it's possible for a compaction (i.e .commit action) to be attempted and then reattempted after a long time. In this scenario, the heartbeats will expire and we will clean it as a failed compaction?
...-client/hudi-client-common/src/main/java/org/apache/hudi/table/HoodieTimelineArchiveLog.java
Outdated
Show resolved
Hide resolved
...-client/hudi-client-common/src/main/java/org/apache/hudi/table/HoodieTimelineArchiveLog.java
Outdated
Show resolved
Hide resolved
...t-common/src/main/java/org/apache/hudi/table/action/rollback/BaseRollbackActionExecutor.java
Outdated
Show resolved
Hide resolved
hudi-common/src/test/java/org/apache/hudi/common/testutils/FileCreateUtils.java
Outdated
Show resolved
Hide resolved
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.
I was expecting some changes in this file, to clean the failed writers for LAZY? Can you point me to the files that do this.
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.
@vinothchandar No change in the action executor, to keep this change simple, just added that here -> https://github.com/apache/hudi/blob/35a82c7f68c8bf0b381e47dc3f12374bec7081eb/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/AbstractHoodieWriteClient.java#L609
|
@n3nash I have left my first round of comments. Main call out: #2359 (comment) |
|
@vinothchandar Addressed most your comments, replied to the others. At a high level, LAZY rollback is supported only for commits/deltacommits since there is contention when creating multiple writers. Since compactions/clustering don't have contention, they continue to do inline rollback which does not cause any issues -> reason is because they re-use the same instant time so changing them requires deeper changes which is not needed for multi-writer. |
2ec3746 to
4718c03
Compare
|
@vinothchandar ready for review |
4718c03 to
557bbd3
Compare
vinothchandar
left a comment
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.
Overall looks good to me.
Two major things.
- We need to introduce a new config
hoodie.table.services.enablewhich should betrueby default. and we guard all of the following below duringpostCommitorpostWriteusing that flag. i.e only do it if the flag is true.
- scheduling of cleaning, compaction, clustering
- inline cleaning, compaction, clustering
- archival
the expectation is that users will have to turn the flag off for the other writers.
- Can we make a pass at all points where compaction (and clustering) is triggered (cli, the HoodieCompactor tool, async compaction runners in data source,deltastreamer) and ensure the policy is set to
EAGER.
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/AbstractHoodieClient.java
Outdated
Show resolved
Hide resolved
...lient/hudi-client-common/src/main/java/org/apache/hudi/client/AbstractHoodieWriteClient.java
Outdated
Show resolved
Hide resolved
...lient/hudi-client-common/src/main/java/org/apache/hudi/client/AbstractHoodieWriteClient.java
Outdated
Show resolved
Hide resolved
...lient/hudi-client-common/src/main/java/org/apache/hudi/client/AbstractHoodieWriteClient.java
Outdated
Show resolved
Hide resolved
...lient/hudi-client-common/src/main/java/org/apache/hudi/client/AbstractHoodieWriteClient.java
Outdated
Show resolved
Hide resolved
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.
withFailedWritesCleaningPolicy(). can we make this consistent everywhere.
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.
Done, is there an easy way to generate a with... for these variables ? I'm manually typing them hence end up introducing errors.
...-client/hudi-client-common/src/main/java/org/apache/hudi/table/HoodieTimelineArchiveLog.java
Outdated
Show resolved
Hide resolved
...-client-common/src/test/java/org/apache/hudi/client/heartbeat/TestHoodieHeartbeatClient.java
Outdated
Show resolved
Hide resolved
hudi-common/src/main/java/org/apache/hudi/common/model/HoodieFailedWritesCleaningPolicy.java
Outdated
Show resolved
Hide resolved
6413586 to
868fbab
Compare
|
@vinothchandar Addressed comments, ready for review. |
868fbab to
b702eb1
Compare
...hudi-client-common/src/main/java/org/apache/hudi/client/heartbeat/HoodieHeartbeatClient.java
Outdated
Show resolved
Hide resolved
|
@n3nash rebase, repush and get CI to be happy? |
vinothchandar
left a comment
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.
Few comments. High level, LGTM.
can we also add a test with metadatatable enabled and actually do some production runs?
...lient/hudi-client-common/src/main/java/org/apache/hudi/client/AbstractHoodieWriteClient.java
Outdated
Show resolved
Hide resolved
...lient/hudi-client-common/src/main/java/org/apache/hudi/client/AbstractHoodieWriteClient.java
Outdated
Show resolved
Hide resolved
...lient/hudi-client-common/src/main/java/org/apache/hudi/client/AbstractHoodieWriteClient.java
Outdated
Show resolved
Hide resolved
...lient/hudi-client-common/src/main/java/org/apache/hudi/client/AbstractHoodieWriteClient.java
Outdated
Show resolved
Hide resolved
...lient/hudi-client-common/src/main/java/org/apache/hudi/client/AbstractHoodieWriteClient.java
Outdated
Show resolved
Hide resolved
...-client-common/src/test/java/org/apache/hudi/client/heartbeat/TestHoodieHeartbeatClient.java
Outdated
Show resolved
Hide resolved
...-client-common/src/test/java/org/apache/hudi/client/heartbeat/TestHoodieHeartbeatClient.java
Outdated
Show resolved
Hide resolved
hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/SparkRDDWriteClient.java
Outdated
Show resolved
Hide resolved
hudi-common/src/main/java/org/apache/hudi/common/util/HeartbeatUtils.java
Outdated
Show resolved
Hide resolved
hudi-common/src/main/java/org/apache/hudi/common/util/HeartbeatUtils.java
Outdated
Show resolved
Hide resolved
b702eb1 to
fddf048
Compare
@vinothchandar I have made 2 changes for multi-writer.
Not following your ask around committed writes and skipping, can you elaborate ? Resolved other comments, left 1 open item. |
4a0cc16 to
a100eaf
Compare
37ad1ec to
fd7e2e5
Compare
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/AbstractHoodieClient.java
Outdated
Show resolved
Hide resolved
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/AbstractHoodieClient.java
Outdated
Show resolved
Hide resolved
...lient/hudi-client-common/src/main/java/org/apache/hudi/client/AbstractHoodieWriteClient.java
Outdated
Show resolved
Hide resolved
...lient/hudi-client-common/src/main/java/org/apache/hudi/client/AbstractHoodieWriteClient.java
Outdated
Show resolved
Hide resolved
...lient/hudi-client-common/src/main/java/org/apache/hudi/client/AbstractHoodieWriteClient.java
Outdated
Show resolved
Hide resolved
...hudi-client-common/src/main/java/org/apache/hudi/client/heartbeat/HoodieHeartbeatClient.java
Outdated
Show resolved
Hide resolved
...hudi-client-common/src/main/java/org/apache/hudi/client/heartbeat/HoodieHeartbeatClient.java
Outdated
Show resolved
Hide resolved
...-spark-client/src/main/java/org/apache/hudi/table/action/clean/SparkCleanActionExecutor.java
Outdated
Show resolved
Hide resolved
hudi-common/src/main/java/org/apache/hudi/common/util/CleanerUtils.java
Outdated
Show resolved
Hide resolved
hudi-common/src/main/java/org/apache/hudi/common/util/CleanerUtils.java
Outdated
Show resolved
Hide resolved
ab25ff9 to
ff7149d
Compare
|
@nsivabalan Addressed your comments PTAL. @vinothchandar This PR is ready. Following is the summary of outstanding items / previous questions.
|
|
IIUC, we are delegating the clean up of failed writes to cleaner. Did you folks discuss the possibility of introducing another async table service called FailedJobsCleaner or something. Bcoz, we can't force customer to make regular cleaning aggressive as they might have their own reasons to retain for longer period of time. But what if they wish to clean up failed writes sooner but not inline ;) Also, from my understanding, this new "FailedJobsCleaner" does not need to coordinate w/ anything or take locks. just need to delete unreachable files. So, overhead may not be much as compared to other jobs. no updates to metadata as well right? or do we need to? I understand we keep adding more weight to async operations (cleaning, compaction, clustering, etc). anyways, wanted to hear your thoughts. |
1. Refactor rollback and move cleaning failed commits logic into cleaner 2. Introduce hoodie heartbeat to ascertain failed commits 3. Fix test cases
ff7149d to
93572f2
Compare
vinothchandar
left a comment
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. @n3nash I am going to land this after CI passed. I already rebased this.
Left some minor comments and suggestions for tests. We can add this to the subsequent PR.
...lient/hudi-client-common/src/main/java/org/apache/hudi/client/AbstractHoodieWriteClient.java
Outdated
Show resolved
Hide resolved
...lient/hudi-client-common/src/main/java/org/apache/hudi/client/AbstractHoodieWriteClient.java
Outdated
Show resolved
Hide resolved
| /** | ||
| * Helper class to delete heartbeat for completed or failed instants with expired heartbeats. | ||
| */ | ||
| public class HeartbeatUtils { |
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.
to our discussion today, can you add a TestHeartbeatUtils in a subsequent PR?
| ValidationUtils.checkArgument(heartbeatClient != null); | ||
| try { | ||
| if (config.getFailedWritesCleanPolicy().isLazy() && heartbeatClient.isHeartbeatExpired(instantTime)) { | ||
| throw new HoodieException("Heartbeat for instant " + instantTime + " has expired, last heartbeat " |
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.
this deserves its own Exception class.
| + heartbeatClient.getLastHeartbeatTime(table.getMetaClient().getFs(), config.getBasePath(), instantTime)); | ||
| } | ||
| } catch (IOException io) { | ||
| throw new HoodieException("Unable to read heartbeat", io); |
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.
Pleas throw HoodieIOException whenever there is an IOException being rethrown
| LOG.info("Received request to start heartbeat for instant time " + instantTime); | ||
| Heartbeat heartbeat = instantToHeartbeatMap.get(instantTime); | ||
| ValidationUtils.checkArgument(heartbeat == null || !heartbeat.isHeartbeatStopped(), "Cannot restart a stopped heartbeat for " + instantTime); | ||
| if (heartbeat != null && heartbeat.isHeartbeatStarted()) { |
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.
invert the condition? and avoid the return ;
| return; | ||
| } else { | ||
| Heartbeat newHeartbeat = new Heartbeat(); | ||
| newHeartbeat.setHeartbeatStarted(true); |
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.
shuld the Heartbeat() constructor do this?
| HoodieTimelineArchiveLog archiveLog = new HoodieTimelineArchiveLog(cfg, table); | ||
|
|
||
| HoodieTimeline timeline = metaClient.getActiveTimeline().getCommitsAndCompactionTimeline(); | ||
| HoodieTimeline timeline = metaClient.getActiveTimeline().getWriteTimeline(); |
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 add a test case for archiving with inflight instants and lazy cleaning?
| public static final String AUXILIARYFOLDER_NAME = METAFOLDER_NAME + Path.SEPARATOR + ".aux"; | ||
| public static final String BOOTSTRAP_INDEX_ROOT_FOLDER_PATH = AUXILIARYFOLDER_NAME + Path.SEPARATOR + ".bootstrap"; | ||
|
|
||
| public static final String TEMPFOLDER_NAME = METAFOLDER_NAME + File.separator + ".temp"; |
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.
why this change?
| // hit a block with instant time greater than should be processed, stop processing further | ||
| break; | ||
| } | ||
| if (r.getBlockType() != CORRUPT_BLOCK && r.getBlockType() != COMMAND_BLOCK) { |
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.
test case for this?
…2359) 1. Refactor rollback and move cleaning failed commits logic into cleaner 2. Introduce hoodie heartbeat to ascertain failed commits 3. Fix test cases
…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
…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
Tips
What is the purpose of the pull request
(For example: This pull request adds quick-start document.)
Brief change log
(for example:)
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:)
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.