Versions Compared

Key

  • This line was added.
  • This line was removed.
  • Formatting was changed.
Comment: update title form

Table of Contents

Proposers

  • @<proposer1 JIRA username>
  • @<proposer2 JIRA username>
  • ...

Approvers

  • @<approver1 JIRA username> : [APPROVED/REQUESTED_INFO/REJECTED]
  • @<approver2 JIRA username> : [APPROVED/REQUESTED_INFO/REJECTED]
  • ...

Status

Current state: ["Under Discussion"]

Discussion thread: here

JIRA: here

Released: <Hudi Version>

Abstract

Currently, in terms of computing engines, Hudi has mainly integrated deeply with Spark. Apache Flink is a popular streaming processing engine. Integrating Hudi with Flink is a valuable work. This will enable Hudi to embrace more computing engines, and the pluggable design will also make its architecture more flexible and open.

Background

The current design of Hudi is highly dependent on Spark in four modules that depend on Spark, as shown below:

If we expect Hudi to be decoupled from the computing engine, then we have to do some refactoring to improve the current situation. At a very high level, there are roughly two options:

  1. Keep the existing implementation and re-implement all Spark-related capabilities based on Flink (this means we may add four more Flink-related modules);
  2. Refactor the current design so that the parts related to Spark are cohesive into a specific module;


We need to rethink the functional design related to Spark so that it can better follow the pluggable design.

Implementation

The implementation contains two sides.

There are two key modules that need to redesign, they are: hudi-client and hudi-utlilites.

About hudi-client, We can split hudi-client module into two new modules: hudi-writer-common and hudi-spark. hudi-writer-common will have the HoodieIndex, HoodieTable abstract classes along with IOhandle classes, metrics, exceptions. Index implementations themselves now can move to hudi-spark. HoodieWriteClient and the table classes can also put into hudi-spark module. After this refactoring, we can introduce a new hudi-flink module to package flink specific implementation of the index.

About hudi-utlilites, we use some specific Spark data sources there. So we can either split the core deltastreamer logic as a hudi-deltastreamer-core or hudi-utilities-core and have the Sources themselves live in a separate module as hudi-utilities-spark, hudi-utilities-flink:

After step 1, we have decoupled Hudi and spark. Now, we need to implement some functions just like Spark did, e.g. Index.

The implementation of the index feature is one of the parts in the Flink Job DAG. Flink Stateful API can provide the ability of state management. We can store the index via Flink stateful API. From a low-level abstraction, in unbounded streaming, window is a mechanism that split the unbounded stream into bounded stream. We can use the window in Flink to mapping the micro-batch(RDD) in Spark.

Rollout/Adoption Plan

None

Test Plan

TBD.