[jira] [Comment Edited] (HUDI-538) Restructuring hudi client module for multi engine support

2020-01-20 Thread vinoyang (Jira)


[ 
https://issues.apache.org/jira/browse/HUDI-538?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17019814#comment-17019814
 ] 

vinoyang edited comment on HUDI-538 at 1/21/20 2:32 AM:


bq. Initially, the DeltaStreamer will not work over Flink, and that may be 
okay? 

[~vinoth] {{DeltaStreamer}} may not support Flink in the first phase. However, 
there should be some sources support Flink(let Flink connector with the 
upstream system) to read records then write to Hudi. Otherwise, where the 
records wait to be writen come from?


was (Author: yanghua):
bg. Initially, the DeltaStreamer will not work over Flink, and that may be 
okay? 

{{DeltaStreamer}} may not support Flink in the first phase. However, there 
should be some sources support Flink(let Flink connector with the upstream 
system) to read records then write to Hudi. Otherwise, where the records wait 
to be writen come from?

> Restructuring hudi client module for multi engine support
> -
>
> Key: HUDI-538
> URL: https://issues.apache.org/jira/browse/HUDI-538
> Project: Apache Hudi (incubating)
>  Issue Type: Wish
>  Components: Code Cleanup
>Reporter: vinoyang
>Priority: Major
>
> Hudi is currently tightly coupled with the Spark framework. It caused the 
> integration with other computing engine more difficult. We plan to decouple 
> it with Spark. This umbrella issue used to track this work.
> Some thoughts wrote here: 
> https://docs.google.com/document/d/1Q9w_4K6xzGbUrtTS0gAlzNYOmRXjzNUdbbe0q59PX9w/edit?usp=sharing
> The feature branch is {{restructure-hudi-client}}.



--
This message was sent by Atlassian Jira
(v8.3.4#803005)


[jira] [Comment Edited] (HUDI-538) Restructuring hudi client module for multi engine support

2020-01-19 Thread vinoyang (Jira)


[ 
https://issues.apache.org/jira/browse/HUDI-538?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17019265#comment-17019265
 ] 

vinoyang edited comment on HUDI-538 at 1/20/20 7:14 AM:


[~vinoth] OK, another thing we may need to consider. Based on our discussion, 
we agreed on put {{hudi-utilities}} aside. However, for both Flink and Spark, 
they follow {{source -> transform -> sink}} mode. Currently, the sources host 
in {{hudi-utilities}} package and they are not Spark-free. So, it seems we also 
need to consider it. WDYT?


was (Author: yanghua):
[~vinoth] OK, another thing we may need to consider. Based on our discussion, 
we agreed on put {{hudi-utilities}} aside. However, for both Flink and Spark, 
they observe {{source -> transform -> sink}} mode. Currently, the sources host 
in {{hudi-utilities}} package and they are not Spark-free. So, it seems we also 
need to consider it. WDYT?

> Restructuring hudi client module for multi engine support
> -
>
> Key: HUDI-538
> URL: https://issues.apache.org/jira/browse/HUDI-538
> Project: Apache Hudi (incubating)
>  Issue Type: Wish
>  Components: Code Cleanup
>Reporter: vinoyang
>Priority: Major
>
> Hudi is currently tightly coupled with the Spark framework. It caused the 
> integration with other computing engine more difficult. We plan to decouple 
> it with Spark. This umbrella issue used to track this work.
> Some thoughts wrote here: 
> https://docs.google.com/document/d/1Q9w_4K6xzGbUrtTS0gAlzNYOmRXjzNUdbbe0q59PX9w/edit?usp=sharing
> The feature branch is {{restructure-hudi-client}}.



--
This message was sent by Atlassian Jira
(v8.3.4#803005)


[jira] [Comment Edited] (HUDI-538) Restructuring hudi client module for multi engine support

2020-01-18 Thread Vinoth Chandar (Jira)


[ 
https://issues.apache.org/jira/browse/HUDI-538?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17018664#comment-17018664
 ] 

Vinoth Chandar edited comment on HUDI-538 at 1/18/20 6:05 PM:
--

+1 [~yanghua] , I added a second task for moving classes around based on your 
changes..

Core issue we need a solution for IMO is the following .. (if we solve this, 
rest is more or less easy)  I will illustrate using Spark (since my 
understanding of Flink is somewhat limited atm) ..

 

So,  even for Spark I would like the writing to be done via _RDD_ or 
_DataFrame_ routes and the current code converts the dataframe into RDDs to 
perform writes. This has some performance side-effects (suprisingly, :P) 

 

1) If you take a single class like _HoodieWriteClient_, then it currently does 
something like `hoodieRecordRDD.map().sort()` internally.. if we want to 
support Flink DataStream or Spark DataFrame as the object, then we need to 
somehow define an abstraction like `HoodieExecutionContext`  which will have 
a common set of map(T) -> T, sortBy(T) -> T, filter(), repartition() methods? 
There will be subclasses like _HoodieSparkRDDExecutionContext,_ 
_HoodieSparkDataFrameExecutionContext_, 
_HoodieFlinkDataStreamExecutionContext_ which will implement them 
in engine specific ways and hand back the transformed T object? 

 

2) Right now, we work with _HoodieRecord_, as the record level abstraction.. 
i.e we eagerly parse the input into a HoodieKey (String recordKey, String 
partitionPath) and HoodieRecordPayload. The key is needed during indexing, and 
the payload is needed to precombine duplicates within a batch (may be spark 
specific)/combine incoming record with whats stored in the table during 
writing.. We need a way to do these lazily by pushing the key extraction 
function into the entire writing path. 

 

I think we should deeply think about these issues.. have concrete approaches 
before we embark more deeply.. We will hit these issues.. 

 

 

 


was (Author: vc):
+1 [~yanghua] , I added a second task for moving classes around based on your 
changes..

Core issue we need a solution for IMO is the following .. (if we solve this, 
rest is more or less easy)  I will illustrate using Spark (since my 
understanding of Flink is somewhat limited atm) ..

 

So,  even for Spark I would like the writing to be done via _RDD_ or 
_DataFrame_ routes and the current code converts the dataframe into RDDs to 
perform writes. This has some performance side-effects (suprisingly, :P) 

 

1) If you take a single class like _HoodieWriteClient_, then it currently does 
something like `hoodieRecordRDD.map().sort()` internally.. if we want to 
support Flink DataStream or Spark DataFrame as the object, then we need to 
somehow define an abstraction like `HoodieExecutionContext`  which will have 
a common set of map(T) -> T, sortBy(T) -> T, filter(), repartition() methods? 
There will be subclasses like _HoodieSparkRDDExecutionContext,_ 
_HoodieSparkDataFrameExecutionContext_, 
_HoodieFlinkDataStreamExecutionContext_ which will implement them 
in engine specific ways and hand back the transformed T object? 

 

2) Right now, we work with _HoodieRecord_, as the record level abstraction.. 
i.e we eagerly parse the input into a HoodieKey (String recordKey, String 
partitionPath) and HoodieRecordPayload. The key is needed during indexing, and 
the payload is needed to precombine duplicates within a batch (may be spark 
specific)/combine incoming record with whats stored in the table during 
writing.. We need a way to do these lazily by pushing the key extraction 
function into the entire writing path. 

 

I think we should deeply think about these issues.. have concrete approaches 
before we embark more deeply.. We will hit these issues.. 

 

 

 

 

 

 

 

 

 

> Restructuring hudi client module for multi engine support
> -
>
> Key: HUDI-538
> URL: https://issues.apache.org/jira/browse/HUDI-538
> Project: Apache Hudi (incubating)
>  Issue Type: Wish
>  Components: Code Cleanup
>Reporter: vinoyang
>Priority: Major
>
> Hudi is currently tightly coupled with the Spark framework. It caused the 
> integration with other computing engine more difficult. We plan to decouple 
> it with Spark. This umbrella issue used to track this work.
> Some thoughts wrote here: 
> https://docs.google.com/document/d/1Q9w_4K6xzGbUrtTS0gAlzNYOmRXjzNUdbbe0q59PX9w/edit?usp=sharing
> The feature branch is {{restructure-hudi-client}}.



--
This message was sent by Atlassian Jira
(v8.3.4#803005)