1
0
Commit Graph

198 Commits

Author SHA1 Message Date
Sivabalan Narayanan
61148c1c43 [HUDI-2176, 2178, 2179] Adding virtual key support to COW table (#3306) 2021-07-26 17:21:04 -04:00
jsbali
66207ed91a [HUDI-1848] Adding support for HMS for running DDL queries in hive-sy… (#2879)
* [HUDI-1848] Adding support for HMS for running DDL queries in hive-sync-tool

* [HUDI-1848] Fixing test cases

* [HUDI-1848] CR changes

* [HUDI-1848] Fix checkstyle violations

* [HUDI-1848] Fixed a bug when metastore api fails for complex schemas with multiple levels.

* [HUDI-1848] Adding the complex schema and resolving merge conflicts

* [HUDI-1848] Adding some more javadocs

* [HUDI-1848] Added javadocs for DDLExecutor impls

* [HUDI-1848] Fixed style issue
2021-07-23 09:03:15 -07:00
Samrat
a086d255c8 [HUDI-1860] Add INSERT_OVERWRITE and INSERT_OVERWRITE_TABLE support to DeltaStreamer (#3184) 2021-07-19 21:49:43 -04:00
liujinhui
af837d2f18 [HUDI-1447] DeltaStreamer kafka source supports consuming from specified timestamp (#2438) 2021-07-17 00:31:06 -04:00
Sagar Sumit
5804ad8e32 [HUDI-1483] Support async clustering for deltastreamer and Spark streaming (#3142)
- Integrate async clustering service with HoodieDeltaStreamer and HoodieStreamingSink
- Added methods in HoodieAsyncService to reuse code
2021-07-11 14:43:38 -04:00
Sebastian Bernauer
8f7ad8b178 [HUDI-2069] Refactored String constants (#3172) 2021-07-07 14:22:00 -04:00
Randal Boyle
60e0254e67 [HUDI-1996] Adding functionality to allow the providing of basic auth creds for confluent cloud schema registry (#3097)
* adding support for basic auth with confluent cloud schema registry
2021-07-05 23:40:23 -07:00
wenningd
d412fb2fe6 [HUDI-89] Add configOption & refactor all configs based on that (#2833)
Co-authored-by: Wenning Ding <wenningd@amazon.com>
2021-06-30 14:26:30 -07:00
Vinay Patil
94f0f40fec [HUDI-1944] Support Hudi to read from committed offset (#3175)
* [HUDI-1944] Support Hudi to read from committed offset

* [HUDI-1944] Adding group option to KafkaResetOffsetStrategies

* [HUDI-1944] Update Exception msg
2021-06-30 16:41:28 +08:00
Vinay Patil
039aeb6dce [HUDI-1910] Commit Offset to Kafka after successful Hudi commit (#3092) 2021-06-28 21:52:05 +08:00
Vinay Patil
ed1a5daa9a [HUDI-2060] Added tests for KafkaOffsetGen (#3136) 2021-06-25 12:37:47 -04:00
n3nash
23dbc09a0d [MINOR] Removing un-used files and references (#3150) 2021-06-24 22:17:40 -07:00
s-sanjay
0fb8556b0d Add ability to provide multi-region (global) data consistency across HMS in different regions (#2542)
[global-hive-sync-tool] Add a global hive sync tool to sync hudi table across clusters. Add a way to rollback the replicated time stamp if we fail to sync or if we partly sync

Co-authored-by: Jagmeet Bali <jsbali@uber.com>
2021-06-24 20:26:26 -07:00
Sebastian Bernauer
b32855545b [HUDI-2069] Fix KafkaAvroSchemaDeserializer to not rely on reflection (#3111)
[HUDI-2069] KafkaAvroSchemaDeserializer should get sourceSchema passed instead using Reflection
2021-06-24 09:08:21 -04:00
Vaibhav Sinha
43b9c1fa1c [HUDI-1826] Add ORC support in HoodieSnapshotExporter (#3130) 2021-06-23 17:04:25 +08:00
Sagar Sumit
429e9fb5fe [HUDI-1248] Increase timeout for deltaStreamerTestRunner in TestHoodieDeltaStreamer (#3110) 2021-06-20 21:42:12 -07:00
Sagar Sumit
1cbdb49816 [HUDI-251] Adds JDBC source support for DeltaStreamer (#2915)
As discussed in RFC-14, this change implements the first phase of JDBC incremental puller.
It consists following changes:

- JdbcSource: This class extends RowSource and implements
  fetchNextBatch(Option<String> lastCkptStr, long sourceLimit)

- SqlQueryBuilder: A simple utility class to build sql queries fluently.

- Implements two modes of fetching: full and incremental.
  Full is a complete scan of RDBMS table.
  Incremental is delta since last checkpoint.
  Incremental mode falls back to full fetch in case of any exception.
2021-06-19 10:12:11 -04:00
Wei
53396061cc [MINOR] Fix wrong package name (#3114) 2021-06-19 11:50:01 +08:00
Vinay Patil
769dd2d7c9 [HUDI-2004] Move CheckpointUtils test cases to independant class (#3072) 2021-06-14 17:14:59 +08:00
Vinoth Govindarajan
9e4114dd46 [HUDI-1790] Added SqlSource to fetch data from any partitions for backfill use case (#2896) 2021-06-10 18:03:07 -04:00
pengzhiwei
f760ec543e [HUDI-1659] Basic Implement Of Spark Sql Support For Hoodie (#2645)
Main functions:
Support create table for hoodie.
Support CTAS.
Support Insert for hoodie. Including dynamic partition and static partition insert.
Support MergeInto for hoodie.
Support DELETE
Support UPDATE
Both support spark2 & spark3 based on DataSourceV1.

Main changes:
Add sql parser for spark2.
Add HoodieAnalysis for sql resolve and logical plan rewrite.
Add commands implementation for CREATE TABLE、INSERT、MERGE INTO & CTAS.
In order to push down the update&insert logical to the HoodieRecordPayload for MergeInto, I make same change to the
HoodieWriteHandler and other related classes.
1、Add the inputSchema for parser the incoming record. This is because the inputSchema for MergeInto is different from writeSchema as there are some transforms in the update& insert expression.
2、Add WRITE_SCHEMA to HoodieWriteConfig to pass the write schema for merge into.
3、Pass properties to HoodieRecordPayload#getInsertValue to pass the insert expression and table schema.


Verify this pull request
Add TestCreateTable for test create hoodie tables and CTAS.
Add TestInsertTable for test insert hoodie tables.
Add TestMergeIntoTable for test merge hoodie tables.
Add TestUpdateTable for test update hoodie tables.
Add TestDeleteTable for test delete hoodie tables.
Add TestSqlStatement for test supported ddl/dml currently.
2021-06-07 23:24:32 -07:00
Vinoth Govindarajan
57611d10b5 [HUDI-1743] Added support for SqlFileBasedTransformer (#2747) 2021-06-07 21:48:27 -04:00
wangxianghu
974b476180 [HUDI-1940] Add SqlQueryBasedTransformer unit test (#3004) 2021-05-28 22:30:30 +08:00
Raymond Xu
afa6bc0b10 [HUDI-1723] Fix path selector listing files with the same mod date (#2845) 2021-05-25 10:19:10 -04:00
zhangminglei
fe3f5c2d56 [HUDI-1913] Using streams instead of loops for input/output (#2962) 2021-05-19 09:13:38 +08:00
TeRS-K
be9db2c4f5 [HUDI-1055] Remove hardcoded parquet in tests (#2740)
* Remove hardcoded parquet in tests
* Use DataFileUtils.getInstance
* Renaming DataFileUtils to BaseFileUtils

Co-authored-by: Vinoth Chandar <vinoth@apache.org>
2021-05-11 10:01:45 -07:00
Volodymyr Burenin
8a48d16e41 [HUDI-1707] Reduces log level for too verbose messages from info to debug level. (#2714)
* Reduces log level for too verbose messages from info to debug level.
* Sort config output.
* Code Review : Small restructuring + rebasing to master
 - Fixing flaky multi delta streamer test
 - Using isDebugEnabled() checks
 - Some changes to shorten log message without moving to DEBUG

Co-authored-by: volodymyr.burenin <volodymyr.burenin@cloudkitchens.com>
Co-authored-by: Vinoth Chandar <vinoth@apache.org>
2021-05-10 07:16:02 -07:00
Nick Young
f4e3b94971 [HUDI-1742] Improve table level config priority for HoodieMultiTableDeltaStreamer (#2744) 2021-04-26 22:05:06 +08:00
Sivabalan Narayanan
3e4fa170cf [HUDI-1835] Fixing kafka native config param for auto offset reset (#2864) 2021-04-25 12:16:09 -04:00
pengzhiwei
aacb8be521 [HUDI-1415] Read Hoodie Table As Spark DataSource Table (#2283) 2021-04-20 14:21:38 -07:00
Aditya Tiwari
ec2334ceac [HUDI-1716]: Resolving default values for schema from dataframe (#2765)
- Adding default values and setting null as first entry in UNION data types in avro schema. 

Co-authored-by: Aditya Tiwari <aditya.tiwari@flipkart.com>
2021-04-19 10:05:20 -04: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
n3nash
bec70413c0 [HUDI-1728] Fix MethodNotFound for HiveMetastore Locks (#2731) 2021-03-27 10:07:10 -07:00
n3nash
01a1d7997b [HUDI-1712] Rename & standardize config to match other configs (#2708) 2021-03-24 17:24:02 +08:00
n3nash
d7b18783bd [HUDI-1709] Improving config names and adding hive metastore uri config (#2699) 2021-03-22 01:22:06 -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
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
Ankush Kanungo
f5e31be086 [HUDI-1685] keep updating current date for every batch (#2671) 2021-03-12 15:53:01 -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
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
liujinhui
617cc24ad1 [HUDI-1367] Make deltaStreamer transition from dfsSouce to kafkasouce (#2227)
Co-authored-by: Sivabalan Narayanan <sivabala@uber.com>
2021-02-25 07:08:13 -05:00
Sivabalan Narayanan
c9fcf964b2 [HUDI-1315] Adding builder for HoodieTableMetaClient initialization (#2534) 2021-02-20 09:54:26 +08:00
lw0090
368c1a8f5c [HUDI-1399] support a independent clustering spark job to asynchronously clustering (#2379)
* [HUDI-1481]  add  structured streaming and delta streamer clustering unit test

* [HUDI-1399] support a independent clustering spark job to asynchronously clustering

* [HUDI-1399]  support a  independent clustering spark job to asynchronously clustering

* [HUDI-1498] Read clustering plan from requested file for inflight instant (#2389)

* [HUDI-1399]  support  a independent clustering spark job with schedule generate instant time

Co-authored-by: satishkotha <satishkotha@uber.com>
2021-01-09 17:30:16 -08:00
Udit Mehrotra
4e64226844 [HUDI-1450] Use metadata table for listing in HoodieROTablePathFilter (apache#2326)
[HUDI-1394] [RFC-15] Use metadata table (if present) to get all partition paths (apache#2351)
2021-01-04 07:59:47 -08:00
lw0090
9e6889a8ce [HUDI-1481] add structured streaming and delta streamer clustering unit test (#2360) 2020-12-27 20:27:09 -08:00
Bhavani Sudha Saktheeswaran
14d5d1100c [HUDI-1406] Add date partition based source input selector for Delta streamer (#2264)
- Adds ability to list only recent date based partitions from source data.
- Parallelizes listing for faster tailing of DFSSources
2020-12-17 03:59:30 -08:00
liujinhui
62b392b49c [HUDI-1343] Add standard schema postprocessor which would rewrite the schema using spark-avro conversion (#2192)
Co-authored-by: liujh <liujh@t3go.cn>
2020-12-03 19:28:34 -08:00
wangxianghu
a23230c8c2 [HUDI-1400] Replace Operation enum with WriteOperationType (#2259) 2020-11-19 13:40:04 +08:00