Skip to content

Conversation

@n3nash
Copy link
Contributor

@n3nash n3nash commented Dec 24, 2020

  1. Added LockProvider API for pluggable lock methologies
  2. Added Resolution Strategy API to allow for pluggable conflict resolution

This diff -> #2359 is a pre-requisite for landing this PR.

Tips

What is the purpose of the pull request

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

Brief change log

(for example:)

  • Modify AnnotationLocation checkstyle rule in checkstyle.xml

Verify this pull request

(Please pick either of the following options)

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

(or)

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

(or)

This change added tests and can be verified as follows:

(example:)

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

Committer checklist

  • Has a corresponding JIRA in PR title & commit

  • Commit message is descriptive of the change

  • CI is green

  • Necessary doc changes done or have another open PR

  • For large changes, please consider breaking it into sub-tasks under an umbrella JIRA.

@n3nash n3nash added the status:in-progress Work in progress label Dec 24, 2020
@codecov-io
Copy link

codecov-io commented Dec 24, 2020

Codecov Report

Merging #2374 (0e7fbb3) into master (2fdae68) will decrease coverage by 42.34%.
The diff coverage is 0.00%.

Impacted file tree graph

@@             Coverage Diff              @@
##             master   #2374       +/-   ##
============================================
- Coverage     51.87%   9.52%   -42.35%     
+ Complexity     3556      48     -3508     
============================================
  Files           465      53      -412     
  Lines         22165    1963    -20202     
  Branches       2357     235     -2122     
============================================
- Hits          11498     187    -11311     
+ Misses         9667    1763     -7904     
+ Partials       1000      13      -987     
Flag Coverage Δ Complexity Δ
hudicli ? ?
hudiclient ? ?
hudicommon ? ?
hudiflink ? ?
hudihadoopmr ? ?
hudisparkdatasource ? ?
hudisync ? ?
huditimelineservice ? ?
hudiutilities 9.52% <0.00%> (-59.96%) 0.00 <0.00> (ø)

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

Impacted Files Coverage Δ Complexity Δ
...apache/hudi/utilities/deltastreamer/DeltaSync.java 0.00% <0.00%> (-70.00%) 0.00 <0.00> (-52.00)
...va/org/apache/hudi/utilities/IdentitySplitter.java 0.00% <0.00%> (-100.00%) 0.00% <0.00%> (-2.00%)
...va/org/apache/hudi/utilities/schema/SchemaSet.java 0.00% <0.00%> (-100.00%) 0.00% <0.00%> (-3.00%)
...a/org/apache/hudi/utilities/sources/RowSource.java 0.00% <0.00%> (-100.00%) 0.00% <0.00%> (-4.00%)
.../org/apache/hudi/utilities/sources/AvroSource.java 0.00% <0.00%> (-100.00%) 0.00% <0.00%> (-1.00%)
.../org/apache/hudi/utilities/sources/JsonSource.java 0.00% <0.00%> (-100.00%) 0.00% <0.00%> (-1.00%)
...rg/apache/hudi/utilities/sources/CsvDFSSource.java 0.00% <0.00%> (-100.00%) 0.00% <0.00%> (-10.00%)
...g/apache/hudi/utilities/sources/JsonDFSSource.java 0.00% <0.00%> (-100.00%) 0.00% <0.00%> (-4.00%)
...apache/hudi/utilities/sources/JsonKafkaSource.java 0.00% <0.00%> (-100.00%) 0.00% <0.00%> (-6.00%)
...pache/hudi/utilities/sources/ParquetDFSSource.java 0.00% <0.00%> (-100.00%) 0.00% <0.00%> (-5.00%)
... and 437 more

@nsivabalan nsivabalan added the priority:blocker Production down; release blocker label Dec 26, 2020
@vinothchandar vinothchandar self-assigned this Dec 26, 2020
@n3nash n3nash force-pushed the concurrent_writer_lock branch from f2fc458 to 592e4bd Compare December 28, 2020 05:10
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.

some high level comments. High level, looks promising to me.

Copy link
Member

Choose a reason for hiding this comment

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

whats the reason to have this in hudi-common? locking is only used by writing correct.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

So we need to pass the same configs to HiveMetastoreBasedLockProvider which is in hudi-hive-sync as well as ZookeeperBasedLockProvider which is in hudi-client. This object helps to keep the config reading from the same class.

@n3nash n3nash force-pushed the concurrent_writer_lock branch from c2f054f to 98c3c7d Compare January 3, 2021 08:41
@n3nash
Copy link
Contributor Author

n3nash commented Jan 3, 2021

@vinothchandar Addressed most of the code comments replied to some which needs clarification/resolution. 2 high level items are missing 1. Adding a new test class for multi-writer 2. Testing with test-suite / real job. It's ready for final review in the meantime.

@n3nash n3nash removed the status:in-progress Work in progress label Jan 3, 2021
@n3nash n3nash changed the title [WIP] [HUDI-845] Added locking capability to allow multiple writers [HUDI-845] Added locking capability to allow multiple writers Jan 4, 2021
@n3nash n3nash force-pushed the concurrent_writer_lock branch 4 times, most recently from 21792c6 to 7f975c1 Compare January 8, 2021 08:08
@vinothchandar vinothchandar removed the priority:blocker Production down; release blocker label Jan 9, 2021
@n3nash n3nash force-pushed the concurrent_writer_lock branch 3 times, most recently from ccc5f9a to 15157b3 Compare January 19, 2021 05:28
@n3nash
Copy link
Contributor Author

n3nash commented Jan 19, 2021

@vinothchandar This PR is ready, concurrency control added for writers and table services, documentation to follow after this PR is merged.

@n3nash n3nash force-pushed the concurrent_writer_lock branch 5 times, most recently from 7ababea to 8b8d594 Compare January 22, 2021 05:49
@vinothchandar
Copy link
Member

@n3nash Beginning the final review of this. Should we still land #2359 first?

@vinothchandar
Copy link
Member

@n3nash can you please rebase and repush

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.

Can we move the scheduling of compaction and clustering into the writeClient itself, protected in the critical section.

I can review the lock implementations here more line-by-line. but this is the main gap that I saw

Copy link
Member

Choose a reason for hiding this comment

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

should archival be here too?

Copy link
Contributor Author

@n3nash n3nash Feb 3, 2021

Choose a reason for hiding this comment

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

It should be, but for each of these table services, we now follow the steps of schedule -> inflight -> complete. Archival doesn't do that right now and we never run archival async (unlike clean), so I haven't added it here. I have refactored the clean actions to do this. Filed a ticket for async archival -> https://issues.apache.org/jira/browse/HUDI-1576

Copy link
Member

Choose a reason for hiding this comment

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

we should check somewhere that user cannot turn on both async and inline? , if we are adding an explicit config.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Right now, this config is harmless, it just assumes the role of autoClean.

Copy link
Member

Choose a reason for hiding this comment

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

this does not read easily. rename inlineCleaningEnabled or shouldCleanInline() or something like tht? (same wherever applicable)

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 kept the same naming convention as before for isInlineCompaction etc. I refactored all of them to inline<..>Enabled

Copy link
Member

Choose a reason for hiding this comment

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

rename to initIfNeeded consistently

Copy link
Contributor Author

Choose a reason for hiding this comment

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

refactored to bootstrapMetadata

Copy link
Member

Choose a reason for hiding this comment

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

we can also rename this method if needed

Copy link
Contributor Author

Choose a reason for hiding this comment

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

refactored to bootstrapMetadata

@n3nash n3nash force-pushed the concurrent_writer_lock branch 2 times, most recently from e93d599 to 340efd2 Compare February 4, 2021 07:16
@n3nash
Copy link
Contributor Author

n3nash commented Feb 4, 2021

@vinothchandar Addressed your comments and replied to other ones, PTAL. I've also refactored the code to make the flow simpler.

@n3nash n3nash force-pushed the concurrent_writer_lock branch 2 times, most recently from 47403db to 99b341e Compare March 12, 2021 20:29
@n3nash
Copy link
Contributor Author

n3nash commented Mar 12, 2021

@vinothchandar Addressed your feedback comments and build is ready. Some more things to note:

  1. Added explicit dependency on curator to control the versions, earlier brought in through transitive dependency. NOTE that the latest stable version for curator is 4.X but we have to use curator version 2.X since 4.X is not backwards compatible with Zookeeper 2.x and 3.X while curator 2.X is backwards and forward compatible. The latest stable version of curator brings in many new features which are not applicable to us, so it should be OK to use 2.X
  2. Added Spark Datasource based nodes to HoodieTestSuite and added a test case to run through optimistic locking enabled with Spark datasource. NOTE that we can only test using ZookeeperBasedLockProvider for tests and cannot use HiveMetastoreBasedLockProvider. This is because embedded hive metastore only allows one Hive client. When we use spark, spark's HiveExternalCatalog already creates an internal hive client, when hudi tries to create a new hive client through the HiveMetastoreBasedLockProvider, it throws underlying derby related exceptions. Some workarounds are to not use derby and are mentioned here -> https://issues.apache.org/jira/browse/HIVE-21302 but this requires a change to the underlying choice of DB for hive metastore itself. Hence, for now, we can only use ZK locks for in-memory unit tests. Ran it against production setup of HiveMetastore to validate this is only a unit test problem.
  3. Added more tests and introduced TestSimpleConcurrentFileWritesConflictResolutionStrategy to ensure the right candidate streams are always used.
  4. Added another section to quickstart on how to enable optimistic locks. This PR -> [HUDI-1679] Adding example for using optimistic locks with zk #2660 has the details. Validated this by turning on INFO logs on docker to confirm that locking works.

Some warnings ( I will add these to the docs when I send the PR for documentation)

  1. DeltaStreamer cannot work with multi-writer. Currently we copy checkpoints from the previous commit to the next - this breaks when we perform multi-writing.
  2. Configs for retries and timeouts for locking can vary because of many factors : response times of file system APIs, number of commits to check conflict resolution against etc. We need to document that folks have to choose sane timeouts and retries for their use-case if defaults don't work.
  3. Incremental pull using TimelineAPI's will not work for clients. There will be a follow up PR for this in progress.

@vinothchandar
Copy link
Member

DeltaStreamer cannot work with multi-writer. Currently we copy checkpoints from the previous commit to the next - this breaks when we perform multi-writing.

But we should be able to allow backfills using spark sql, which don't really mess with checkpoints? We cannot ignore this scenario. Can you look into how we can provide best practices

@n3nash n3nash force-pushed the concurrent_writer_lock branch 2 times, most recently from 1886bbd to 01577fd Compare March 14, 2021 09:21
@n3nash
Copy link
Contributor Author

n3nash commented Mar 14, 2021

@vinothchandar It's possible to allow backfills using spark-sql but there are some corner cases. Consider the following:

  1. Ingestion job running with commit c4 (checkpoint = c3)
  2. Ingestion job finishes with commit c4 (checkpoint = c3)
  3. Someone runs a spark-sql job to backfill some data in an older partition, commit c5. Since this spark-sql job (unlike deltastreamer) does not handle checkpoint copying from prev metadata to next, it would be the client's job to do this.
  4. If they fail to do this, deltastreamer next ingestion c6 will read no checkpoint from c5.

I've made the following changes:

  1. To make this manageable, I've added the following config : hoodie.write.meta.key.prefixes. One can set this config to ensure that during the critical section, if this config is set, it will copy over all the metadata for the keys that match with the prefix set in this config from the latest metadata to the current commit.
  2. Made changes and added these multi-writer tests to HoodieDeltaStreamer as well. Technically, one can do the backfill using HoodieDeltaStreamer or Spark-SQL. For HoodieDeltaStreamer they would have to set some custom checkpoint or mark it to null to ensure that the job just picks the data from the backfill location, for Spark-SQL it would not matter.

Yes, I am going to add documents on best practices / things to watch out in the other PR I opened for documentation. I will do that after resolving any further comments and landing this PR in the next couple of days.

NOTE: The test may be failing because of some thread.sleep related code that I'm trying to remove. Will update tomorrow.

@n3nash n3nash force-pushed the concurrent_writer_lock branch 2 times, most recently from 4e029d3 to 9278889 Compare March 15, 2021 04:03
1. Added LockProvider API for pluggable lock methodologies
2. Added Resolution Strategy API to allow for pluggable conflict resolution
3. Added TableService client API to schedule table services
4. Added Transaction Manager for wrapping actions within transactions
@n3nash n3nash force-pushed the concurrent_writer_lock branch 6 times, most recently from 2a305dc to f7685b4 Compare March 15, 2021 07:31
@n3nash
Copy link
Contributor Author

n3nash commented Mar 15, 2021

@vinothchandar Build succeeds locally and should pass on jenkins (will check tomorrow morning), ready for review.

Copy link
Member

@vinothchandar vinothchandar left a comment

Choose a reason for hiding this comment

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

Just went over the feedback from my last review.

  1. have you tested the spark bundle end to end? I think so, based on #2660 . Just making sure.
  2. Can we also add the curator deps to flink and utilities bundles?

Pending these, once CI is happy. We can land and do the follow ups.

@n3nash n3nash force-pushed the concurrent_writer_lock branch from f7685b4 to bcc0f5b Compare March 16, 2021 07:15
@n3nash n3nash force-pushed the concurrent_writer_lock branch from bcc0f5b to 0e7fbb3 Compare March 16, 2021 17:30
@n3nash
Copy link
Contributor Author

n3nash commented Mar 16, 2021

@vinothchandar I've added curator to utilities bundle. In this PR, we don't support locking for flink and java clients so I have not added to the flink bundle. If you have another reason to add curator to flink bundles, let me know.

@vinothchandar
Copy link
Member

cc @yanghua @danny0405 @leesf to take note for flink/java support follow up.

@n3nash
Copy link
Contributor Author

n3nash commented Mar 16, 2021

Here is the issue for the follow up -> https://issues.apache.org/jira/browse/HUDI-1698. I'll work with you guys offline @danny0405 @yanghua @leesf

@n3nash n3nash merged commit 7424194 into apache:master Mar 16, 2021
@danny0405
Copy link
Contributor

Here is the issue for the follow up -> https://issues.apache.org/jira/browse/HUDI-1698. I'll work with you guys offline @danny0405 @yanghua @leesf

Thanks for the notion, would take up the following work.

prashantwason pushed a commit to prashantwason/incubator-hudi that referenced this pull request Aug 5, 2021
…#2374)

* [HUDI-845] Added locking capability to allow multiple writers
1. Added LockProvider API for pluggable lock methodologies
2. Added Resolution Strategy API to allow for pluggable conflict resolution
3. Added TableService client API to schedule table services
4. Added Transaction Manager for wrapping actions within transactions
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

priority:blocker Production down; release blocker priority:high Significant impact; potential bugs

Projects

None yet

Development

Successfully merging this pull request may close these issues.

6 participants