1
0
Commit Graph

1434 Commits

Author SHA1 Message Date
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
vinoyang
3cab928b50 [HUDI-1735] Add hive-exec dependency for hudi-examples (#2737) 2021-03-30 21:35:16 +08:00
Gary Li
050626ad6c [MINOR] Add Missing Apache License to test files (#2736) 2021-03-29 07:17:23 -07:00
garyli1019
e069b64e10 [HOTFIX] fix deploy staging jars script 2021-03-29 06:04:48 -07:00
Gary Li
4db970dc8a [HOTFIX] Disable ITs for Spark3 and scala2.12 (#2733) 2021-03-29 06:04:48 -07: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
Danny Chan
8b774fe331 [HUDI-1495] Bump Flink version to 1.12.2 (#2718) 2021-03-26 14:25:57 +08:00
garyli1019
6e803e08b1 Moving to 0.9.0-SNAPSHOT on master branch. 2021-03-24 21:37:14 +08:00
Danny Chan
29b79c99b0 [hotfix] Log the error message for creating table source first (#2711) 2021-03-24 18:25:37 +08:00
n3nash
01a1d7997b [HUDI-1712] Rename & standardize config to match other configs (#2708) 2021-03-24 17:24:02 +08:00
Danny Chan
03668dbaf1 [HUDI-1710] Read optimized query type for Flink batch reader (#2702)
Read optimized query returns the records from:

* COW table: the latest parquet files
* MOR table: parquet file records from the latest compaction committed
2021-03-23 18:41:30 -07:00
legendtkl
0e6909d3e2 [MINOR][DOCUMENT] Update README doc for integ test (#2703) 2021-03-23 20:21:56 +08:00
n3nash
d7b18783bd [HUDI-1709] Improving config names and adding hive metastore uri config (#2699) 2021-03-22 01:22:06 -07:00
Liulietong
ce3e8ec870 [HUDI-1667]: Fix a null value related bug for spark vectorized reader. (#2636) 2021-03-20 07:54:20 -07:00
Volodymyr Burenin
900de34e45 [HUDI-1650] Custom avro kafka deserializer. (#2619)
* Custom avro kafka deserializer

Co-authored-by: volodymyr.burenin <volodymyr.burenin@cloudkitchens.com>
Co-authored-by: Sivabalan Narayanan <sivabala@uber.com>
2021-03-20 00:51:08 -07:00
Sivabalan Narayanan
161d530f93 Fixing kafka auto.reset.offsets config param key (#2691) 2021-03-19 12:54:29 -07:00
Sivabalan Narayanan
55a489c769 [1568] Fixing spark3 bundles (#2625)
- [1568] Fixing spark3 bundles
2021-03-19 14:21:36 -04:00
Danny Chan
f74828fca1 [HUDI-1705] Flush as per data bucket for mini-batch write (#2695)
Detects the buffer size for each data bucket before flushing. So that we
avoid flushing data buckets with few records.
2021-03-19 16:30:54 +08: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
xiarixiaoyao
d429169ff7 [HUDI-1688]hudi write should uncache rdd, when the write operation is finnished (#2673) 2021-03-18 10:19:18 -07:00
Danny Chan
f1e0018f12 [HUDI-1704] Use PRIMARY KEY syntax to define record keys for Flink Hudi table (#2694)
The SQL PRIMARY KEY semantics is very same with Hoodie record key, using
PRIMARY KEY is more straight-forward way instead of a table option:
hoodie.datasource.write.recordkey.field.

After this change, both PRIMARY KEY and table option can define hoodie
record key, while the PRIMARY KEY has higher priority if both are
defined.

Note: a column with PRIMARY KEY constraint is forced to be non-nullable.
2021-03-18 20:21:52 +08:00
Danny Chan
968488fa3a [HUDI-1701] Implement HoodieTableSource.explainSource for all kinds of pushing down (#2690)
We should implement the interface HoodieTableSource.explainSource to
track the table source signature diff for all kinds of pushing down,
such as filter pushing or limit pushing.
2021-03-17 23:05:18 +08: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
Sivabalan Narayanan
b038623ed3 [HUDI 1615] Fixing null schema in bulk_insert row writer path (#2653)
* [HUDI-1615] Avoid passing in null schema from row writing/deltastreamer
* Fixing null schema in bulk insert row writer path
* Fixing tests

Co-authored-by: vc <vinoth@apache.org>
2021-03-16 09:44:11 -07:00
Vinoth Govindarajan
16864aee14 [HUDI-1695] Fixed the error messaging (#2679) 2021-03-16 11:30:26 +08: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
76bf2cc790 [HUDI-1692] Bounded source for stream writer (#2674)
Supports bounded source such as VALUES for stream mode writer.
2021-03-15 19:42:36 +08:00
Danny Chan
fc6c5f4285 [HUDI-1684] Tweak hudi-flink-bundle module pom and reorganize the pacakges for hudi-flink module (#2669)
* Add required dependencies for hudi-flink-bundle module
* Some packages reorganization of hudi-flink module
2021-03-15 16:02:05 +08:00
Sivabalan Narayanan
e93c6a5693 [HUDI-1496] Fixing input stream detection of GCS FileSystem (#2500)
* Adding SchemeAwareFSDataInputStream for abstract out special handling for GCSFileSystem
* Moving wrapping of fsDataInputStream to separate method in HoodieLogFileReader

Co-authored-by: Vinoth Chandar <vinoth@apache.org>
2021-03-14 00:57:57 -08:00
Ankush Kanungo
f5e31be086 [HUDI-1685] keep updating current date for every batch (#2671) 2021-03-12 15:53:01 -08: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
Danny Chan
e8e6708aea [HUDI-1664] Avro schema inference for Flink SQL table (#2658)
A Flink SQL table has DDL that defines the table schema, we can use that
to infer the Avro schema and there is no need to declare a Avro schema
explicitly anymore.

But we still keep the config option for explicit Avro schema in case
there is corner cases that the inferred schema is not correct
(especially for the nullability).
2021-03-11 19:45:48 +08:00
Danny Chan
12ff562d2b [HUDI-1678] Row level delete for Flink sink (#2659) 2021-03-11 19:44:06 +08:00
Danny Chan
2fdae6835c [HUDI-1663] Streaming read for Flink MOR table (#2640)
Supports two read modes:
* Read the full data set starting from the latest commit instant and
  subsequent incremental data set
* Read data set that starts from a specified commit instant
2021-03-10 22:44:06 +08:00
satishkotha
c4a66324cd [HUDI-1651] Fix archival of requested replacecommit (#2622) 2021-03-09 15:56:44 -08:00
Balajee Nagasubramaniam
d8af24d8a2 [HUDI-1635] Improvements to Hudi Test Suite (#2628) 2021-03-09 13:29:38 -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
xiarixiaoyao
02073235c3 [HUDI-1662] Fix hive date type conversion for mor table (#2634) 2021-03-08 12:16:13 +08:00
Sivabalan Narayanan
5cf2f2618b [HUDI-1618] Fixing NPE with Parquet src in multi table delta streamer (#2577) 2021-03-07 16:40:40 -05:00
Raymond Xu
9437e0ddef [MINOR] Fix import in StreamerUtil.java (#2638) 2021-03-07 12:37:15 -08:00
satishkotha
11ad4ed26b [HUDI-1661] Exclude clustering commits from getExtraMetadataFromLatest API (#2632) 2021-03-05 13:42:19 -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
f53bca404f [HUDI-1655] Support custom date format and fix unsupported exception in DatePartitionPathSelector (#2621)
- Add a config to allow parsing custom date format in `DatePartitionPathSelector`. Currently it assumes date partition string in the format of `yyyy-MM-dd`.
- Fix a bug where `UnsupportedOperationException` was thrown when sort `eligibleFiles` in-place. Changed to sort it and store in a new list.
2021-03-04 21:01:51 -08:00
satishkotha
7cc75e0be2 [HUDI-1646] Provide mechanism to read uncommitted data through InputFormat (#2611) 2021-03-04 17:43:31 -08:00