1
0
Commit Graph

289 Commits

Author SHA1 Message Date
董可伦
bf5a52e51b [HUDI-2320] Add support ByteArrayDeserializer in AvroKafkaSource (#3502) 2021-08-30 10:01:15 +08:00
Udit Mehrotra
c350d05dd3 Restore 0.8.0 config keys with deprecated annotation (#3506)
Co-authored-by: Sagar Sumit <sagarsumit09@gmail.com>
Co-authored-by: Vinoth Chandar <vinoth@apache.org>
2021-08-19 13:36:40 -07:00
Udit Mehrotra
3e301196bf Moving to 0.10.0-SNAPSHOT on master branch. 2021-08-14 18:51:09 -07:00
Y Ethan Guo
23dca6c237 [HUDI-2268] Add upgrade and downgrade to and from 0.9.0 (#3470)
- Added upgrade and downgrade step to and from 0.9.0. Upgrade adds few table properties. Downgrade recreates timeline server based marker files if any.
2021-08-14 20:20:23 -04:00
Sagar Sumit
5cc96e85c1 [HUDI-1897] Deltastreamer source for AWS S3 (#3433)
- Added two sources for two stage pipeline. a. S3EventsSource that fetches events from SQS and ingests to a meta hoodie table. b. S3EventsHoodieIncrSource reads S3 events from this meta hoodie table, fetches actual objects from S3 and ingests to sink hoodie table. 
- Added selectors to assist in S3EventsSource. 

Co-authored-by: Satish M <84978833+satishmittal1111@users.noreply.github.com>
Co-authored-by: Vinoth Chandar <vinoth@apache.org>
2021-08-14 08:25:10 -04:00
Sagar Sumit
0544d70d8f [MINOR] Deprecate older configs (#3464)
Rename and deprecate props in HoodieWriteConfig

Rename and deprecate older props
2021-08-12 20:31:04 -07:00
Sivabalan Narayanan
b651336454 [HUDI-2294] Adding virtual keys support to deltastreamer (#3450) 2021-08-12 08:02:39 -04:00
liujinhui
c0fc9cdaf3 MINOR (#3459)
MOVE hoodie Deltrstreamer to hudi-utilties
2021-08-12 18:19:05 +08:00
vinoyang
dc3cbb28e7 [MINOR] Correct TestKafkaSource class and comment (#3451) 2021-08-12 09:11:00 +08:00
Y Ethan Guo
4783176554 [HUDI-1138] Add timeline-server-based marker file strategy for improving marker-related latency (#3233)
- Can be enabled for cloud stores like S3. Not supported for hdfs yet, due to partial write failures.
2021-08-11 11:48:13 -04:00
Sivabalan Narayanan
1196736185 [HUDI-1129] Improving schema evolution support in hudi (#2927)
* Adding support to ingest records with old schema after table's schema is evolved

* Rebasing against latest master

- Trimming test file to be < 800 lines
- Renaming config names

* Addressing feedback

Co-authored-by: Vinoth Chandar <vinoth@apache.org>
2021-08-10 09:15:37 -07:00
wenningd
91bb0d1318 [HUDI-2255] Refactor Datasource options (#3373)
Co-authored-by: Wenning Ding <wenningd@amazon.com>
2021-08-03 17:50:30 -07:00
rmahindra123
245e1fd17d [HUDI-2272] Pass base file format to sync clients (#3397)
Co-authored-by: Rajesh Mahindra <rmahindra@Rajeshs-MacBook-Pro.local>
2021-08-03 14:46:02 -07:00
zhangyue19921010
dde57b293c [HUDI-2164] Let users build cluster plan and execute this plan at once using HoodieClusteringJob for async clustering (#3259)
* add --mode schedule/execute/scheduleandexecute

* fix checkstyle

* add UT testHoodieAsyncClusteringJobWithScheduleAndExecute

* log changed

* try to make ut success

* try to fix ut

* modify ut

* review changed

* code review

* code review

* code review

* code review

Co-authored-by: yuezhang <yuezhang@freewheel.tv>
2021-08-02 08:07:59 +08:00
Sivabalan Narayanan
7bdae69053 [HUDI-2253] Refactoring few tests to reduce runningtime. DeltaStreamer and MultiDeltaStreamer tests. Bulk insert row writer tests (#3371)
Co-authored-by: Sivabalan Narayanan <nsb@Sivabalans-MBP.attlocal.net>
2021-07-29 22:22:26 -07:00
davehagman
8105cf588e [HUDI-2230] Make codahale times transient to avoid serializable exceptions (#3345) 2021-07-28 14:45:09 +08:00
Sivabalan Narayanan
61148c1c43 [HUDI-2176, 2178, 2179] Adding virtual key support to COW table (#3306) 2021-07-26 17:21:04 -04:00
rmahindra123
a14b19fdd5 [HUDI-1241] Automate the generation of configs webpage as configs are added to Hudi repo (#3302) 2021-07-23 21:33:34 -07: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
pengzhiwei
2c910ee3af [HUDI-2212] Missing PrimaryKey In Hoodie Properties For CTAS Table (#3332) 2021-07-23 15:21:57 +08: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
liujinhui
3b264e80d9 [HUDI-1633] Make callback return HoodieWriteStat (#2445)
* CALLBACK add partitionPath

* callback can send hoodieWriteStat

* add ApiMaturityLevel
2021-07-16 12:37:07 +08:00
vinoth chandar
75040ee9e5 [HUDI-2149] Ensure and Audit docs for every configuration class in the codebase (#3272)
- Added docs when missing
 - Rewrote, reworded as needed
 - Made couple more classes extend HoodieConfig
2021-07-14 10:56:08 -07: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
Sebastian Bernauer
05d6e18190 [HUDI-2055] Added deltastreamer metric for time of lastSync (#3129) 2021-07-05 23:34:46 -07:00
pengzhiwei
b34d53fa9c [HUDI-2088] Missing Partition Fields And PreCombineField In Hoodie Properties For Table Written By Flink (#3171) 2021-07-01 17:25:18 +08: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
zhangyue19921010
e99a6b031b [HUDI-2073] Fix the bug of hoodieClusteringJob never quit (#3157)
Co-authored-by: yuezhang <yuezhang@freewheel.tv>
2021-06-26 22:03:41 -07: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
Wei
d519c74626 [HUDI-2008] Avoid the raw type usage in some classes under hudi-utilities module (#3076) 2021-06-16 22:37:29 +08:00
Vinay Patil
769dd2d7c9 [HUDI-2004] Move CheckpointUtils test cases to independant class (#3072) 2021-06-14 17:14:59 +08:00
Wei
ba728d822f [HUDI-2002] Modify HiveIncrementalPuller log level to ERROR (#3070)
Co-authored-by: wei.zhang2 <wei.zhang2@dmall.com>
2021-06-12 10:21:43 -07: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
Wei
a8b10e9067 [MINOR] Remove boxing (#3062) 2021-06-10 13:03:32 +08:00
wangxianghu
7261f08507 [HUDI-1929] Support configure KeyGenerator by type (#2993) 2021-06-08 09:26:10 -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