1
0
Commit Graph

399 Commits

Author SHA1 Message Date
Danny Chan
ab4a7b0b4a [HUDI-1788] Insert overwrite (table) for Flink writer (#2808)
Supports `INSERT OVERWRITE` and `INSERT OVERWRITE TABLE` for Flink
writer.
2021-04-14 10:23:37 +08:00
wangxianghu
040756d8c0 [HUDI-1785] Move OperationConverter to hudi-client-common for code reuse (#2798) 2021-04-12 16:22:33 +08:00
hj2016
1da16dfd2e [HUDI-1784] Added print detailed stack log when hbase connection error (#2799) 2021-04-12 13:46:06 +08:00
hongdd
ecdbd2517f [HUDI-699] Fix CompactionCommand and add unit test for CompactionCommand (#2325) 2021-04-08 15:35:33 +08:00
Simon
18459d4045 [MINOR] Some unit test code optimize (#2782)
* Optimized code

* Optimized code
2021-04-08 13:35:03 +08:00
Danny Chan
9c369c607d [HUDI-1757] Assigns the buckets by record key for Flink writer (#2757)
Currently we assign the buckets by record partition path which could
cause hotspot if the partition field is datetime type. Changes to assign
buckets by grouping the record whth their key first, the assignment is
valid if only there is no conflict(two task write to the same bucket).

This patch also changes the coordinator execution to be asynchronous.
2021-04-06 19:06:41 +08:00
Roc Marshal
94a5e72f16 [HUDI-1737][hudi-client] Code Cleanup: Extract common method in HoodieCreateHandle & FlinkCreateHandle (#2745) 2021-04-02 11:39:05 +08:00
pengzhiwei
684622c7c9 [HUDI-1591] Implement Spark's FileIndex for Hudi to support queries via Hudi DataSource using non-globbed table path and partition pruning (#2651) 2021-04-01 11:12:28 -07:00
Danny Chan
9804662bc8 [HUDI-1738] Emit deletes for flink MOR table streaming read (#2742)
Current we did a soft delete for DELETE row data when writes into hoodie
table. For streaming read of MOR table, the Flink reader detects the
delete records and still emit them if the record key semantics are still
kept.

This is useful and actually a must for streaming ETL pipeline
incremental computation.
2021-04-01 15:25:31 +08:00
vinoyang
fe16d0de7c [MINOR] Delete useless UpsertPartitioner for flink integration (#2746) 2021-03-31 16:36:42 +08:00
Sebastian Bernauer
aa0da72c59 Preparation for Avro update (#2650) 2021-03-30 21:50:17 -07:00
leo-Iamok
8bc65b9318 [HUDI-1731] Rename UpsertPartitioner in hudi-java-client (#2734)
Co-authored-by: lei.zhu <lei.zhu@envisioncn.com>
2021-03-31 11:06:04 +08:00
Gary Li
452f5e2d66 [HOTFIX] close spark session in functional test suite and disable spark3 test for spark2 (#2727) 2021-03-29 06:04:48 -07:00
Danny Chan
d415d45416 [HUDI-1729] Asynchronous Hive sync and commits cleaning for Flink writer (#2732) 2021-03-29 10:47:29 +08:00
Shen Hong
ecbd389a3f [HUDI-1478] Introduce HoodieBloomIndex to hudi-java-client (#2608) 2021-03-28 20:28:40 +08:00
n3nash
bec70413c0 [HUDI-1728] Fix MethodNotFound for HiveMetastore Locks (#2731) 2021-03-27 10:07:10 -07:00
garyli1019
6e803e08b1 Moving to 0.9.0-SNAPSHOT on master branch. 2021-03-24 21:37:14 +08:00
n3nash
d7b18783bd [HUDI-1709] Improving config names and adding hive metastore uri config (#2699) 2021-03-22 01:22:06 -07:00
Jintao Guan
1277c62398 [HUDI-1653] Add support for composite keys in NonpartitionedKeyGenerator (#2627)
* [HUDI-1653] Add support for composite keys in NonpartitionedKeyGenerator

* update NonpartitionedKeyGenerator to support composite record keys

* update NonpartitionedKeyGenerator
2021-03-18 15:33:31 -07:00
wangxianghu
e602e5dfb9 [MINOR] Remove unused var in AbstractHoodieWriteClient (#2693) 2021-03-18 14:56:02 -07:00
n3nash
74241947c1 [HUDI-845] Added locking capability to allow multiple writers (#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
2021-03-16 16:43:53 -07:00
Prashant Wason
3b36cb805d [HUDI-1552] Improve performance of key lookups from base file in Metadata Table. (#2494)
* [HUDI-1552] Improve performance of key lookups from base file in Metadata Table.

1. Cache the KeyScanner across lookups so that the HFile index does not have to be read for each lookup.
2. Enable block caching in KeyScanner.
3. Move the lock to a limited scope of the code to reduce lock contention.
4. Removed reuse configuration

* Properly close the readers, when metadata table is accessed from executors

 - Passing a reuse boolean into HoodieBackedTableMetadata
 - Preserve the fast return behavior when reusing and opening from multiple threads (no contention)
 - Handle concurrent close() and open readers, for reuse=false, by always synchronizing

Co-authored-by: Vinoth Chandar <vinoth@apache.org>
2021-03-15 13:42:57 -07:00
Danny Chan
20786ab8a2 [HUDI-1681] Support object storage for Flink writer (#2662)
In order to support object storage, we need these changes:

* Use the Hadoop filesystem so that we can find the plugin filesystem
* Do not fetch file size until the file handle is closed
* Do not close the opened filesystem because we want to use the
  filesystem cache
2021-03-12 16:39:24 +08:00
satishkotha
c4a66324cd [HUDI-1651] Fix archival of requested replacecommit (#2622) 2021-03-09 15:56:44 -08:00
Raymond Xu
d3a451611c [MINOR] HoodieClientTestHarness close resources in AfterAll phase (#2646)
Parameterized test case like `org.apache.hudi.table.upgrade.TestUpgradeDowngrade#testUpgrade` incurs flakiness when org.apache.hadoop.fs.FileSystem#closeAll is invoked at BeforeEach; it should be invoked in AfterAll instead.
2021-03-08 17:36:03 +08:00
Shen Hong
8b9dea4ad9 [HUDI-1673] Replace scala.Tule2 to Pair in FlinkHoodieBloomIndex (#2642) 2021-03-08 14:30:34 +08:00
n3nash
f2159c4573 [HUDI-1660] Excluding compaction and clustering instants from inflight rollback (#2631) 2021-03-05 11:18:09 -08:00
pengzhiwei
bc883db5de [HUDI-1636] Support Builder Pattern To Build Table Properties For HoodieTableConfig (#2596) 2021-03-05 14:10:27 +08:00
Raymond Xu
899ae70fdb [HUDI-1587] Add latency and freshness support (#2541)
Save min and max of event time in each commit and compute the latency and freshness metrics.
2021-03-03 20:13:12 -08:00
Prashant Wason
f11a6c7b2d [HUDI-1553] Configuration and metrics for the TimelineService. (#2495) 2021-03-02 21:58:41 -08:00
Prashant Wason
73fa308ff0 [HUDI-1634] Re-bootstrap metadata table when un-synced instants have been archived. (#2595) 2021-03-01 20:31:55 -08:00
satishkotha
7a6b071647 [HUDI-1644] Do not delete older rollback instants as part of rollback. Archival can take care of removing old instants cleanly (#2610) 2021-03-01 09:40:00 -08:00
Danny Chan
7a11de1276 [HUDI-1632] Supports merge on read write mode for Flink writer (#2593)
Also supports async compaction with pluggable strategies.
2021-03-01 12:29:41 +08:00
Prashant Wason
022df0d1b1 [HUDI-1611] Added a configuration to allow specific directories to be filtered out during Metadata Table bootstrap. (#2565) 2021-02-25 16:52:28 -08:00
Danny Chan
97864a48c1 [HUDI-1637] Avoid to rename for bucket update when there is only one flush action during a checkpoint (#2599)
Some of the object storages do not have strong read-after-write
consistency, we should promote to remove the rename operations in the
future.
2021-02-25 10:21:27 +08:00
hj2016
77ba561a6b [HUDI-1347] Fix Hbase index to make rollback synchronous (via config) (#2188)
Co-authored-by: huangjing <huangjing@clinbrain.com>
Co-authored-by: Sivabalan Narayanan <sivabala@uber.com>
2021-02-23 20:56:58 -05:00
Prashant Wason
d2f360f5dd [MINOR] Ensure directory exists before listing all marker files. (#2594) 2021-02-23 08:05:59 -08:00
Shen Hong
2efd0760ac [HUDI-1477] Support copyOnWriteTable in java client (#2382) 2021-02-23 20:50:55 +08:00
Danny Chan
3ceb1b4c83 [HUDI-1624] The state based index should bootstrap from existing base files (#2581) 2021-02-23 13:37:44 +08:00
n3nash
ffcfb58bac [HUDI-1486] Remove inline inflight rollback in hoodie writer (#2359)
1. Refactor rollback and move cleaning failed commits logic into cleaner
2. Introduce hoodie heartbeat to ascertain failed commits
3. Fix test cases
2021-02-19 20:12:22 -08:00
Sivabalan Narayanan
c9fcf964b2 [HUDI-1315] Adding builder for HoodieTableMetaClient initialization (#2534) 2021-02-20 09:54:26 +08:00
Karl_Wang
9431aabfab [HUDI-1381] Schedule compaction based on time elapsed (#2260)
- introduce configs to control how compaction is triggered
- Compaction can be triggered using time, number of delta commits and/or combinations
- Default behaviour remains the same.
2021-02-17 07:44:53 -08:00
pengzhiwei
37972071ff [HUDI-1109] Support Spark Structured Streaming read from Hudi table (#2485) 2021-02-17 03:36:29 -08:00
Danny Chan
5d2491d10c [HUDI-1598] Write as minor batches during one checkpoint interval for the new writer (#2553) 2021-02-17 15:24:50 +08:00
Raymond Xu
527175ab0b [MINOR] Default to empty list for unset datadog tags property (#2574) 2021-02-13 15:52:03 +08:00
Sivabalan Narayanan
d5f202821b Adding fixes to test suite framework. Adding clustering node and validate async operations node. (#2400) 2021-02-12 09:29:21 -08:00
Danny Chan
4c5b6923cc [HUDI-1557] Make Flink write pipeline write task scalable (#2506)
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.
2021-02-06 22:03:52 +08:00
Sivabalan Narayanan
4a5683d54a [MINOR] Fixing the default value for source ordering field for payload config (#2516) 2021-02-04 08:43:03 -05:00
Sivabalan Narayanan
eb91e5ba70 [HUDI-1523] Call mkdir(partition) only if not exists (#2501) 2021-02-03 09:02:37 -05:00
wangxianghu
d74d8e2084 [HUDI-1335] Introduce FlinkHoodieSimpleIndex to hudi-flink-client (#2271) 2021-02-03 08:59:49 +08:00