Re: [DISCUSS] Refactor hudi-client module for better support of multiple engines

2021-10-06 Thread Vinoth Chandar
Hi Gary,

We can pass the constructed timeline and filesystem view into the IOHandle.
I think it makes sense for how Flink does things.

Thanks
Vinoth

On Fri, Sep 24, 2021 at 2:04 AM Gary Li  wrote:

> Hi Vinoth,
>
> Currently, each executor of Flink has a timeline server I believe. Do you
> think we can avoid passing the timeline and filesystem view into the
> IOHandle? I mean one IOHandle is handling the IO of one filegroup, and it
> doesn't need to know the timeline and filesystem view of the table, if we
> can define what the IOHandle is supposed to do during the initialization.
> Based on the log I see, constructing the filesystem view of a partition
> with 500 filegroups is taking 200ms. If the AppendHandle is only flushing a
> few records to disk, the actual flush could be faster than filesystem view
> construction.
>
> On Fri, Sep 24, 2021 at 12:03 PM Vinoth Chandar  wrote:
>
> > Thanks for the explanation. I get the streaming aspect better now. Esp in
> > Flink land. Timeline server and remote file system view are what the
> > defaults are. Assuming its a RPC call that takes 10-100 ms to the
> timeline
> > server, not sure how much room there is for optimization for loading of
> the
> > file system view itself. For timeline, it can be redundant (again in
> spark
> > model, it gets passed from driver to executor). Wondering if we can serve
> > timeline also view the server (duh).
> >
> > In flink, currently we run a timeline server per executor? Wondering if
> > that helps or hurts. In spark we run one in the driver alone.
> >
> > If we want to pass in a constructed table filesystem view and timeline
> into
> > IOHandle we can. I am fine with it, but trying to understand what exactly
> > we are solving by that.
> >
> >
> > On Thu, Sep 23, 2021 at 7:05 PM Gary Li  wrote:
> >
> > > Hi Vinoth,
> > >
> > > IMO the IOHandle should be as lightweight as possible, especially when
> we
> > > want to do streaming and near-real-time update(possibly real-time in
> the
> > > future?). Constructing the timeline and filesystem view inside the
> handle
> > > is time-consuming. In some cases, some handles only write a few records
> > in
> > > each commit, when we try to commit very aggressively. The timeline
> server
> > > and remote filesystem view are helpful, but I feel like there is still
> > some
> > > room for improvement.
> > >
> > > Best,
> > > Gary
> > >
> > > On Fri, Sep 24, 2021 at 3:04 AM Vinoth Chandar 
> > wrote:
> > >
> > > > Hi Gary,
> > > >
> > > > So in effect you want to pull all the timeline filtering out of the
> > > handles
> > > > and pass a plan i.e what file slice to work on - to the handle?
> > > > That does sound cleaner. but we need to introduce this additional
> > layer.
> > > > The timeline and filesystem view do live within the table, I believe
> > > today.
> > > >
> > > > Thanks
> > > > Vinoth
> > > >
> > > > On Wed, Sep 22, 2021 at 6:35 PM Gary Li  wrote:
> > > >
> > > > > Hi Vinoth,
> > > > >
> > > > > Thanks for your response. For HoodieIOHandle, IMO we could define
> the
> > > > scope
> > > > > of the Handle during the initialization, so we don't need to care
> > about
> > > > the
> > > > > timeline and table view when actually writing the data. Is that
> > > > possible? A
> > > > > HoodieTable could have many Handles writing data at the same time
> and
> > > it
> > > > > will look cleaner if we can keep the timeline and file system view
> > > inside
> > > > > the table itself.
> > > > >
> > > > > Best,
> > > > > Gary
> > > > >
> > > > > On Sat, Sep 18, 2021 at 12:06 AM Vinoth Chandar  >
> > > > wrote:
> > > > >
> > > > > > Hi Gary,
> > > > > >
> > > > > > Thanks for the detailed response. Let me add my take on it.
> > > > > >
> > > > > > >>HoodieFlinkMergeOnReadTable.upsert(List) to use
> the
> > > > > > AppendHandle.write(HoodieRecord) directly,
> > > > > >
> > > > > > I have the same issue on JavaClient, for the Kafka Connect
> > > > > implementation.
> > > > > > I have an idea of how we can implement this. Will raise a PR and
> > get
> > > > your
> > > > > > thoughts.
> > > > > > We can then see if this can be leveraged across Flink and Java
> > > clients.
> > > > > >
> > > > > > On the IOHandle not having the Table inside, I think the File
> > > > > > reader/writer  abstraction exists already and having the Table in
> > the
> > > > io
> > > > > > layers helps us perform I/O
> > > > > > while maintaining consistency with the timeline.
> > > > > >
> > > > > > +1 on the next two points.
> > > > > >
> > > > > > I think these layers have well defined roles, and probably why we
> > are
> > > > > able
> > > > > > to get this far :) . May be we need to pull I/O up into
> > hudi-common ?
> > > > > >
> > > > > > For this project, we can trim the scope to code reuse and moving
> > all
> > > > the
> > > > > > different engine specific implementations up into
> > hudi-client-common.
> > > > > >
> > > > > > What do you think?
> > > > > >
> > > > > > Thanks
> > > > > > Vinoth
> > > > > >

Re: [DISCUSS] Refactor hudi-client module for better support of multiple engines

2021-09-24 Thread Gary Li
Hi Vinoth,

Currently, each executor of Flink has a timeline server I believe. Do you
think we can avoid passing the timeline and filesystem view into the
IOHandle? I mean one IOHandle is handling the IO of one filegroup, and it
doesn't need to know the timeline and filesystem view of the table, if we
can define what the IOHandle is supposed to do during the initialization.
Based on the log I see, constructing the filesystem view of a partition
with 500 filegroups is taking 200ms. If the AppendHandle is only flushing a
few records to disk, the actual flush could be faster than filesystem view
construction.

On Fri, Sep 24, 2021 at 12:03 PM Vinoth Chandar  wrote:

> Thanks for the explanation. I get the streaming aspect better now. Esp in
> Flink land. Timeline server and remote file system view are what the
> defaults are. Assuming its a RPC call that takes 10-100 ms to the timeline
> server, not sure how much room there is for optimization for loading of the
> file system view itself. For timeline, it can be redundant (again in spark
> model, it gets passed from driver to executor). Wondering if we can serve
> timeline also view the server (duh).
>
> In flink, currently we run a timeline server per executor? Wondering if
> that helps or hurts. In spark we run one in the driver alone.
>
> If we want to pass in a constructed table filesystem view and timeline into
> IOHandle we can. I am fine with it, but trying to understand what exactly
> we are solving by that.
>
>
> On Thu, Sep 23, 2021 at 7:05 PM Gary Li  wrote:
>
> > Hi Vinoth,
> >
> > IMO the IOHandle should be as lightweight as possible, especially when we
> > want to do streaming and near-real-time update(possibly real-time in the
> > future?). Constructing the timeline and filesystem view inside the handle
> > is time-consuming. In some cases, some handles only write a few records
> in
> > each commit, when we try to commit very aggressively. The timeline server
> > and remote filesystem view are helpful, but I feel like there is still
> some
> > room for improvement.
> >
> > Best,
> > Gary
> >
> > On Fri, Sep 24, 2021 at 3:04 AM Vinoth Chandar 
> wrote:
> >
> > > Hi Gary,
> > >
> > > So in effect you want to pull all the timeline filtering out of the
> > handles
> > > and pass a plan i.e what file slice to work on - to the handle?
> > > That does sound cleaner. but we need to introduce this additional
> layer.
> > > The timeline and filesystem view do live within the table, I believe
> > today.
> > >
> > > Thanks
> > > Vinoth
> > >
> > > On Wed, Sep 22, 2021 at 6:35 PM Gary Li  wrote:
> > >
> > > > Hi Vinoth,
> > > >
> > > > Thanks for your response. For HoodieIOHandle, IMO we could define the
> > > scope
> > > > of the Handle during the initialization, so we don't need to care
> about
> > > the
> > > > timeline and table view when actually writing the data. Is that
> > > possible? A
> > > > HoodieTable could have many Handles writing data at the same time and
> > it
> > > > will look cleaner if we can keep the timeline and file system view
> > inside
> > > > the table itself.
> > > >
> > > > Best,
> > > > Gary
> > > >
> > > > On Sat, Sep 18, 2021 at 12:06 AM Vinoth Chandar 
> > > wrote:
> > > >
> > > > > Hi Gary,
> > > > >
> > > > > Thanks for the detailed response. Let me add my take on it.
> > > > >
> > > > > >>HoodieFlinkMergeOnReadTable.upsert(List) to use the
> > > > > AppendHandle.write(HoodieRecord) directly,
> > > > >
> > > > > I have the same issue on JavaClient, for the Kafka Connect
> > > > implementation.
> > > > > I have an idea of how we can implement this. Will raise a PR and
> get
> > > your
> > > > > thoughts.
> > > > > We can then see if this can be leveraged across Flink and Java
> > clients.
> > > > >
> > > > > On the IOHandle not having the Table inside, I think the File
> > > > > reader/writer  abstraction exists already and having the Table in
> the
> > > io
> > > > > layers helps us perform I/O
> > > > > while maintaining consistency with the timeline.
> > > > >
> > > > > +1 on the next two points.
> > > > >
> > > > > I think these layers have well defined roles, and probably why we
> are
> > > > able
> > > > > to get this far :) . May be we need to pull I/O up into
> hudi-common ?
> > > > >
> > > > > For this project, we can trim the scope to code reuse and moving
> all
> > > the
> > > > > different engine specific implementations up into
> hudi-client-common.
> > > > >
> > > > > What do you think?
> > > > >
> > > > > Thanks
> > > > > Vinoth
> > > > >
> > > > >
> > > > > On Thu, Sep 16, 2021 at 6:55 AM Gary Li  wrote:
> > > > >
> > > > > > Huge +1. Recently I am working on making the Flink writer in a
> > > > streaming
> > > > > > fashion and found the List interface is limiting
> the
> > > > > > streaming power of Flink. By switching from
> > > > > > HoodieFlinkMergeOnReadTable.upsert(List) to use the
> > > > > > AppendHandle.write(HoodieRecord) directly, the throughput was
> > almost
> > > > > > doubled and the checkpo

Re: [DISCUSS] Refactor hudi-client module for better support of multiple engines

2021-09-23 Thread Vinoth Chandar
Thanks for the explanation. I get the streaming aspect better now. Esp in
Flink land. Timeline server and remote file system view are what the
defaults are. Assuming its a RPC call that takes 10-100 ms to the timeline
server, not sure how much room there is for optimization for loading of the
file system view itself. For timeline, it can be redundant (again in spark
model, it gets passed from driver to executor). Wondering if we can serve
timeline also view the server (duh).

In flink, currently we run a timeline server per executor? Wondering if
that helps or hurts. In spark we run one in the driver alone.

If we want to pass in a constructed table filesystem view and timeline into
IOHandle we can. I am fine with it, but trying to understand what exactly
we are solving by that.


On Thu, Sep 23, 2021 at 7:05 PM Gary Li  wrote:

> Hi Vinoth,
>
> IMO the IOHandle should be as lightweight as possible, especially when we
> want to do streaming and near-real-time update(possibly real-time in the
> future?). Constructing the timeline and filesystem view inside the handle
> is time-consuming. In some cases, some handles only write a few records in
> each commit, when we try to commit very aggressively. The timeline server
> and remote filesystem view are helpful, but I feel like there is still some
> room for improvement.
>
> Best,
> Gary
>
> On Fri, Sep 24, 2021 at 3:04 AM Vinoth Chandar  wrote:
>
> > Hi Gary,
> >
> > So in effect you want to pull all the timeline filtering out of the
> handles
> > and pass a plan i.e what file slice to work on - to the handle?
> > That does sound cleaner. but we need to introduce this additional layer.
> > The timeline and filesystem view do live within the table, I believe
> today.
> >
> > Thanks
> > Vinoth
> >
> > On Wed, Sep 22, 2021 at 6:35 PM Gary Li  wrote:
> >
> > > Hi Vinoth,
> > >
> > > Thanks for your response. For HoodieIOHandle, IMO we could define the
> > scope
> > > of the Handle during the initialization, so we don't need to care about
> > the
> > > timeline and table view when actually writing the data. Is that
> > possible? A
> > > HoodieTable could have many Handles writing data at the same time and
> it
> > > will look cleaner if we can keep the timeline and file system view
> inside
> > > the table itself.
> > >
> > > Best,
> > > Gary
> > >
> > > On Sat, Sep 18, 2021 at 12:06 AM Vinoth Chandar 
> > wrote:
> > >
> > > > Hi Gary,
> > > >
> > > > Thanks for the detailed response. Let me add my take on it.
> > > >
> > > > >>HoodieFlinkMergeOnReadTable.upsert(List) to use the
> > > > AppendHandle.write(HoodieRecord) directly,
> > > >
> > > > I have the same issue on JavaClient, for the Kafka Connect
> > > implementation.
> > > > I have an idea of how we can implement this. Will raise a PR and get
> > your
> > > > thoughts.
> > > > We can then see if this can be leveraged across Flink and Java
> clients.
> > > >
> > > > On the IOHandle not having the Table inside, I think the File
> > > > reader/writer  abstraction exists already and having the Table in the
> > io
> > > > layers helps us perform I/O
> > > > while maintaining consistency with the timeline.
> > > >
> > > > +1 on the next two points.
> > > >
> > > > I think these layers have well defined roles, and probably why we are
> > > able
> > > > to get this far :) . May be we need to pull I/O up into hudi-common ?
> > > >
> > > > For this project, we can trim the scope to code reuse and moving all
> > the
> > > > different engine specific implementations up into hudi-client-common.
> > > >
> > > > What do you think?
> > > >
> > > > Thanks
> > > > Vinoth
> > > >
> > > >
> > > > On Thu, Sep 16, 2021 at 6:55 AM Gary Li  wrote:
> > > >
> > > > > Huge +1. Recently I am working on making the Flink writer in a
> > > streaming
> > > > > fashion and found the List interface is limiting the
> > > > > streaming power of Flink. By switching from
> > > > > HoodieFlinkMergeOnReadTable.upsert(List) to use the
> > > > > AppendHandle.write(HoodieRecord) directly, the throughput was
> almost
> > > > > doubled and the checkpoint time of the writer was reduced from
> > minutes
> > > to
> > > > > seconds. But I found it really difficult to fit this change into
> the
> > > > > current client interface.
> > > > >
> > > > > My 2 cents:
> > > > >
> > > > >- The HoodieIOHandle should only handle the IO, and not having
> > > > >HoodieTable inside.
> > > > >- We need a more streaming-friendly Handle. For Flink, we can
> > > > definitely
> > > > >change all the batch mode List to processing
> > > > HoodieRecord
> > > > > one
> > > > >by one, just like the AppendHandle.write(HoodieRecord) and
> > > > >AppendHandle.close(). This will spread the computing cost and
> > > > >flattening the curve.
> > > > >- We can use the Handle to precisely control the JVM to avoid
> OOM
> > > and
> > > > >optimize the memory footprint. Then we don't need to implement
> > > another
> > > > >memory control mechanism in

Re: [DISCUSS] Refactor hudi-client module for better support of multiple engines

2021-09-23 Thread Gary Li
Hi Vinoth,

IMO the IOHandle should be as lightweight as possible, especially when we
want to do streaming and near-real-time update(possibly real-time in the
future?). Constructing the timeline and filesystem view inside the handle
is time-consuming. In some cases, some handles only write a few records in
each commit, when we try to commit very aggressively. The timeline server
and remote filesystem view are helpful, but I feel like there is still some
room for improvement.

Best,
Gary

On Fri, Sep 24, 2021 at 3:04 AM Vinoth Chandar  wrote:

> Hi Gary,
>
> So in effect you want to pull all the timeline filtering out of the handles
> and pass a plan i.e what file slice to work on - to the handle?
> That does sound cleaner. but we need to introduce this additional layer.
> The timeline and filesystem view do live within the table, I believe today.
>
> Thanks
> Vinoth
>
> On Wed, Sep 22, 2021 at 6:35 PM Gary Li  wrote:
>
> > Hi Vinoth,
> >
> > Thanks for your response. For HoodieIOHandle, IMO we could define the
> scope
> > of the Handle during the initialization, so we don't need to care about
> the
> > timeline and table view when actually writing the data. Is that
> possible? A
> > HoodieTable could have many Handles writing data at the same time and it
> > will look cleaner if we can keep the timeline and file system view inside
> > the table itself.
> >
> > Best,
> > Gary
> >
> > On Sat, Sep 18, 2021 at 12:06 AM Vinoth Chandar 
> wrote:
> >
> > > Hi Gary,
> > >
> > > Thanks for the detailed response. Let me add my take on it.
> > >
> > > >>HoodieFlinkMergeOnReadTable.upsert(List) to use the
> > > AppendHandle.write(HoodieRecord) directly,
> > >
> > > I have the same issue on JavaClient, for the Kafka Connect
> > implementation.
> > > I have an idea of how we can implement this. Will raise a PR and get
> your
> > > thoughts.
> > > We can then see if this can be leveraged across Flink and Java clients.
> > >
> > > On the IOHandle not having the Table inside, I think the File
> > > reader/writer  abstraction exists already and having the Table in the
> io
> > > layers helps us perform I/O
> > > while maintaining consistency with the timeline.
> > >
> > > +1 on the next two points.
> > >
> > > I think these layers have well defined roles, and probably why we are
> > able
> > > to get this far :) . May be we need to pull I/O up into hudi-common ?
> > >
> > > For this project, we can trim the scope to code reuse and moving all
> the
> > > different engine specific implementations up into hudi-client-common.
> > >
> > > What do you think?
> > >
> > > Thanks
> > > Vinoth
> > >
> > >
> > > On Thu, Sep 16, 2021 at 6:55 AM Gary Li  wrote:
> > >
> > > > Huge +1. Recently I am working on making the Flink writer in a
> > streaming
> > > > fashion and found the List interface is limiting the
> > > > streaming power of Flink. By switching from
> > > > HoodieFlinkMergeOnReadTable.upsert(List) to use the
> > > > AppendHandle.write(HoodieRecord) directly, the throughput was almost
> > > > doubled and the checkpoint time of the writer was reduced from
> minutes
> > to
> > > > seconds. But I found it really difficult to fit this change into the
> > > > current client interface.
> > > >
> > > > My 2 cents:
> > > >
> > > >- The HoodieIOHandle should only handle the IO, and not having
> > > >HoodieTable inside.
> > > >- We need a more streaming-friendly Handle. For Flink, we can
> > > definitely
> > > >change all the batch mode List to processing
> > > HoodieRecord
> > > > one
> > > >by one, just like the AppendHandle.write(HoodieRecord) and
> > > >AppendHandle.close(). This will spread the computing cost and
> > > >flattening the curve.
> > > >- We can use the Handle to precisely control the JVM to avoid OOM
> > and
> > > >optimize the memory footprint. Then we don't need to implement
> > another
> > > >memory control mechanism in the compute engine itself.
> > > >- HoodieClient, HoodieTable, HoodieIOHandle, HoodieTimeline,
> > > >HoodieFileSystemView e.t.c should have a well-defined role and
> > > > well-defined
> > > >layer. We should know when to use what, it should be used by the
> > > driver
> > > > in
> > > >a single thread or used by the worker in a distributed way.
> > > >
> > > > This is a big project and could benefit Hudi in long term. Happy to
> > > discuss
> > > > more in the design doc or PRs.
> > > >
> > > > Best,
> > > > Gary
> > > >
> > > > On Thu, Sep 16, 2021 at 3:21 AM Raymond Xu <
> > xu.shiyan.raym...@gmail.com>
> > > > wrote:
> > > >
> > > > > +1 that's a great improvement.
> > > > >
> > > > > On Wed, Sep 15, 2021 at 10:40 AM Sivabalan 
> > wrote:
> > > > >
> > > > > > ++1. definitely help's Hudi scale and makes it more maintainable.
> > > > Thanks
> > > > > > for driving this effort. Mostly devs show interest in major
> > features
> > > > and
> > > > > > don't like to spend time in such foundational work. But as the
> > > project
> > > > > > scale

Re: [DISCUSS] Refactor hudi-client module for better support of multiple engines

2021-09-23 Thread Vinoth Chandar
Hi Gary,

So in effect you want to pull all the timeline filtering out of the handles
and pass a plan i.e what file slice to work on - to the handle?
That does sound cleaner. but we need to introduce this additional layer.
The timeline and filesystem view do live within the table, I believe today.

Thanks
Vinoth

On Wed, Sep 22, 2021 at 6:35 PM Gary Li  wrote:

> Hi Vinoth,
>
> Thanks for your response. For HoodieIOHandle, IMO we could define the scope
> of the Handle during the initialization, so we don't need to care about the
> timeline and table view when actually writing the data. Is that possible? A
> HoodieTable could have many Handles writing data at the same time and it
> will look cleaner if we can keep the timeline and file system view inside
> the table itself.
>
> Best,
> Gary
>
> On Sat, Sep 18, 2021 at 12:06 AM Vinoth Chandar  wrote:
>
> > Hi Gary,
> >
> > Thanks for the detailed response. Let me add my take on it.
> >
> > >>HoodieFlinkMergeOnReadTable.upsert(List) to use the
> > AppendHandle.write(HoodieRecord) directly,
> >
> > I have the same issue on JavaClient, for the Kafka Connect
> implementation.
> > I have an idea of how we can implement this. Will raise a PR and get your
> > thoughts.
> > We can then see if this can be leveraged across Flink and Java clients.
> >
> > On the IOHandle not having the Table inside, I think the File
> > reader/writer  abstraction exists already and having the Table in the io
> > layers helps us perform I/O
> > while maintaining consistency with the timeline.
> >
> > +1 on the next two points.
> >
> > I think these layers have well defined roles, and probably why we are
> able
> > to get this far :) . May be we need to pull I/O up into hudi-common ?
> >
> > For this project, we can trim the scope to code reuse and moving all the
> > different engine specific implementations up into hudi-client-common.
> >
> > What do you think?
> >
> > Thanks
> > Vinoth
> >
> >
> > On Thu, Sep 16, 2021 at 6:55 AM Gary Li  wrote:
> >
> > > Huge +1. Recently I am working on making the Flink writer in a
> streaming
> > > fashion and found the List interface is limiting the
> > > streaming power of Flink. By switching from
> > > HoodieFlinkMergeOnReadTable.upsert(List) to use the
> > > AppendHandle.write(HoodieRecord) directly, the throughput was almost
> > > doubled and the checkpoint time of the writer was reduced from minutes
> to
> > > seconds. But I found it really difficult to fit this change into the
> > > current client interface.
> > >
> > > My 2 cents:
> > >
> > >- The HoodieIOHandle should only handle the IO, and not having
> > >HoodieTable inside.
> > >- We need a more streaming-friendly Handle. For Flink, we can
> > definitely
> > >change all the batch mode List to processing
> > HoodieRecord
> > > one
> > >by one, just like the AppendHandle.write(HoodieRecord) and
> > >AppendHandle.close(). This will spread the computing cost and
> > >flattening the curve.
> > >- We can use the Handle to precisely control the JVM to avoid OOM
> and
> > >optimize the memory footprint. Then we don't need to implement
> another
> > >memory control mechanism in the compute engine itself.
> > >- HoodieClient, HoodieTable, HoodieIOHandle, HoodieTimeline,
> > >HoodieFileSystemView e.t.c should have a well-defined role and
> > > well-defined
> > >layer. We should know when to use what, it should be used by the
> > driver
> > > in
> > >a single thread or used by the worker in a distributed way.
> > >
> > > This is a big project and could benefit Hudi in long term. Happy to
> > discuss
> > > more in the design doc or PRs.
> > >
> > > Best,
> > > Gary
> > >
> > > On Thu, Sep 16, 2021 at 3:21 AM Raymond Xu <
> xu.shiyan.raym...@gmail.com>
> > > wrote:
> > >
> > > > +1 that's a great improvement.
> > > >
> > > > On Wed, Sep 15, 2021 at 10:40 AM Sivabalan 
> wrote:
> > > >
> > > > > ++1. definitely help's Hudi scale and makes it more maintainable.
> > > Thanks
> > > > > for driving this effort. Mostly devs show interest in major
> features
> > > and
> > > > > don't like to spend time in such foundational work. But as the
> > project
> > > > > scales, these foundational work will have a higher returns in the
> > long
> > > > run.
> > > > >
> > > > > On Wed, Sep 15, 2021 at 8:29 AM Vinoth Chandar 
> > > > wrote:
> > > > >
> > > > > > Another +1 ,  HoodieData abstraction will go a long way in
> reducing
> > > > LoC.
> > > > > >
> > > > > > Happy to work with you to see this through! I really encourage
> top
> > > > > > contributors to the Flink and Java clients as well,
> > > > > > actively review all PRs, given there are subtle differences
> > > everywhere.
> > > > > >
> > > > > > This will help us smoothly provide all the core features across
> > > > engines.
> > > > > > Also help us easily write a DataSet/Row based
> > > > > > client for Spark as well.
> > > > > >
> > > > > > Onwards and upwards
> > > > > > Vinoth
> > > > > >
> > > > >

Re: [DISCUSS] Refactor hudi-client module for better support of multiple engines

2021-09-22 Thread Gary Li
Hi Vinoth,

Thanks for your response. For HoodieIOHandle, IMO we could define the scope
of the Handle during the initialization, so we don't need to care about the
timeline and table view when actually writing the data. Is that possible? A
HoodieTable could have many Handles writing data at the same time and it
will look cleaner if we can keep the timeline and file system view inside
the table itself.

Best,
Gary

On Sat, Sep 18, 2021 at 12:06 AM Vinoth Chandar  wrote:

> Hi Gary,
>
> Thanks for the detailed response. Let me add my take on it.
>
> >>HoodieFlinkMergeOnReadTable.upsert(List) to use the
> AppendHandle.write(HoodieRecord) directly,
>
> I have the same issue on JavaClient, for the Kafka Connect implementation.
> I have an idea of how we can implement this. Will raise a PR and get your
> thoughts.
> We can then see if this can be leveraged across Flink and Java clients.
>
> On the IOHandle not having the Table inside, I think the File
> reader/writer  abstraction exists already and having the Table in the io
> layers helps us perform I/O
> while maintaining consistency with the timeline.
>
> +1 on the next two points.
>
> I think these layers have well defined roles, and probably why we are able
> to get this far :) . May be we need to pull I/O up into hudi-common ?
>
> For this project, we can trim the scope to code reuse and moving all the
> different engine specific implementations up into hudi-client-common.
>
> What do you think?
>
> Thanks
> Vinoth
>
>
> On Thu, Sep 16, 2021 at 6:55 AM Gary Li  wrote:
>
> > Huge +1. Recently I am working on making the Flink writer in a streaming
> > fashion and found the List interface is limiting the
> > streaming power of Flink. By switching from
> > HoodieFlinkMergeOnReadTable.upsert(List) to use the
> > AppendHandle.write(HoodieRecord) directly, the throughput was almost
> > doubled and the checkpoint time of the writer was reduced from minutes to
> > seconds. But I found it really difficult to fit this change into the
> > current client interface.
> >
> > My 2 cents:
> >
> >- The HoodieIOHandle should only handle the IO, and not having
> >HoodieTable inside.
> >- We need a more streaming-friendly Handle. For Flink, we can
> definitely
> >change all the batch mode List to processing
> HoodieRecord
> > one
> >by one, just like the AppendHandle.write(HoodieRecord) and
> >AppendHandle.close(). This will spread the computing cost and
> >flattening the curve.
> >- We can use the Handle to precisely control the JVM to avoid OOM and
> >optimize the memory footprint. Then we don't need to implement another
> >memory control mechanism in the compute engine itself.
> >- HoodieClient, HoodieTable, HoodieIOHandle, HoodieTimeline,
> >HoodieFileSystemView e.t.c should have a well-defined role and
> > well-defined
> >layer. We should know when to use what, it should be used by the
> driver
> > in
> >a single thread or used by the worker in a distributed way.
> >
> > This is a big project and could benefit Hudi in long term. Happy to
> discuss
> > more in the design doc or PRs.
> >
> > Best,
> > Gary
> >
> > On Thu, Sep 16, 2021 at 3:21 AM Raymond Xu 
> > wrote:
> >
> > > +1 that's a great improvement.
> > >
> > > On Wed, Sep 15, 2021 at 10:40 AM Sivabalan  wrote:
> > >
> > > > ++1. definitely help's Hudi scale and makes it more maintainable.
> > Thanks
> > > > for driving this effort. Mostly devs show interest in major features
> > and
> > > > don't like to spend time in such foundational work. But as the
> project
> > > > scales, these foundational work will have a higher returns in the
> long
> > > run.
> > > >
> > > > On Wed, Sep 15, 2021 at 8:29 AM Vinoth Chandar 
> > > wrote:
> > > >
> > > > > Another +1 ,  HoodieData abstraction will go a long way in reducing
> > > LoC.
> > > > >
> > > > > Happy to work with you to see this through! I really encourage top
> > > > > contributors to the Flink and Java clients as well,
> > > > > actively review all PRs, given there are subtle differences
> > everywhere.
> > > > >
> > > > > This will help us smoothly provide all the core features across
> > > engines.
> > > > > Also help us easily write a DataSet/Row based
> > > > > client for Spark as well.
> > > > >
> > > > > Onwards and upwards
> > > > > Vinoth
> > > > >
> > > > > On Wed, Sep 15, 2021 at 4:57 AM vino yang 
> > > wrote:
> > > > >
> > > > > > Hi Ethan,
> > > > > >
> > > > > > Big +1 for the proposal.
> > > > > >
> > > > > > Actually, we have discussed this topic before.[1]
> > > > > >
> > > > > > Will review your refactor PR later.
> > > > > >
> > > > > > Best,
> > > > > > Vino
> > > > > >
> > > > > > [1]:
> > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> https://lists.apache.org/thread.html/r71d96d285c735b1611920fb3e7224c9ce6fd53d09bf0e8f144f4fcbd%40%3Cdev.hudi.apache.org%3E
> > > > > >
> > > > > >
> > > > > > Y Ethan Guo  于2021年9月15日周三 下午3:34写道:
> > > > > >
> > > > > > > Hi all,
> > > > > > >
> > > > > 

Re: [DISCUSS] Refactor hudi-client module for better support of multiple engines

2021-09-17 Thread Vinoth Chandar
Hi Gary,

Thanks for the detailed response. Let me add my take on it.

>>HoodieFlinkMergeOnReadTable.upsert(List) to use the
AppendHandle.write(HoodieRecord) directly,

I have the same issue on JavaClient, for the Kafka Connect implementation.
I have an idea of how we can implement this. Will raise a PR and get your
thoughts.
We can then see if this can be leveraged across Flink and Java clients.

On the IOHandle not having the Table inside, I think the File
reader/writer  abstraction exists already and having the Table in the io
layers helps us perform I/O
while maintaining consistency with the timeline.

+1 on the next two points.

I think these layers have well defined roles, and probably why we are able
to get this far :) . May be we need to pull I/O up into hudi-common ?

For this project, we can trim the scope to code reuse and moving all the
different engine specific implementations up into hudi-client-common.

What do you think?

Thanks
Vinoth


On Thu, Sep 16, 2021 at 6:55 AM Gary Li  wrote:

> Huge +1. Recently I am working on making the Flink writer in a streaming
> fashion and found the List interface is limiting the
> streaming power of Flink. By switching from
> HoodieFlinkMergeOnReadTable.upsert(List) to use the
> AppendHandle.write(HoodieRecord) directly, the throughput was almost
> doubled and the checkpoint time of the writer was reduced from minutes to
> seconds. But I found it really difficult to fit this change into the
> current client interface.
>
> My 2 cents:
>
>- The HoodieIOHandle should only handle the IO, and not having
>HoodieTable inside.
>- We need a more streaming-friendly Handle. For Flink, we can definitely
>change all the batch mode List to processing HoodieRecord
> one
>by one, just like the AppendHandle.write(HoodieRecord) and
>AppendHandle.close(). This will spread the computing cost and
>flattening the curve.
>- We can use the Handle to precisely control the JVM to avoid OOM and
>optimize the memory footprint. Then we don't need to implement another
>memory control mechanism in the compute engine itself.
>- HoodieClient, HoodieTable, HoodieIOHandle, HoodieTimeline,
>HoodieFileSystemView e.t.c should have a well-defined role and
> well-defined
>layer. We should know when to use what, it should be used by the driver
> in
>a single thread or used by the worker in a distributed way.
>
> This is a big project and could benefit Hudi in long term. Happy to discuss
> more in the design doc or PRs.
>
> Best,
> Gary
>
> On Thu, Sep 16, 2021 at 3:21 AM Raymond Xu 
> wrote:
>
> > +1 that's a great improvement.
> >
> > On Wed, Sep 15, 2021 at 10:40 AM Sivabalan  wrote:
> >
> > > ++1. definitely help's Hudi scale and makes it more maintainable.
> Thanks
> > > for driving this effort. Mostly devs show interest in major features
> and
> > > don't like to spend time in such foundational work. But as the project
> > > scales, these foundational work will have a higher returns in the long
> > run.
> > >
> > > On Wed, Sep 15, 2021 at 8:29 AM Vinoth Chandar 
> > wrote:
> > >
> > > > Another +1 ,  HoodieData abstraction will go a long way in reducing
> > LoC.
> > > >
> > > > Happy to work with you to see this through! I really encourage top
> > > > contributors to the Flink and Java clients as well,
> > > > actively review all PRs, given there are subtle differences
> everywhere.
> > > >
> > > > This will help us smoothly provide all the core features across
> > engines.
> > > > Also help us easily write a DataSet/Row based
> > > > client for Spark as well.
> > > >
> > > > Onwards and upwards
> > > > Vinoth
> > > >
> > > > On Wed, Sep 15, 2021 at 4:57 AM vino yang 
> > wrote:
> > > >
> > > > > Hi Ethan,
> > > > >
> > > > > Big +1 for the proposal.
> > > > >
> > > > > Actually, we have discussed this topic before.[1]
> > > > >
> > > > > Will review your refactor PR later.
> > > > >
> > > > > Best,
> > > > > Vino
> > > > >
> > > > > [1]:
> > > > >
> > > > >
> > > >
> > >
> >
> https://lists.apache.org/thread.html/r71d96d285c735b1611920fb3e7224c9ce6fd53d09bf0e8f144f4fcbd%40%3Cdev.hudi.apache.org%3E
> > > > >
> > > > >
> > > > > Y Ethan Guo  于2021年9月15日周三 下午3:34写道:
> > > > >
> > > > > > Hi all,
> > > > > >
> > > > > > hudi-client module has core Hudi abstractions and client logic
> for
> > > > > > different engines like Spark, Flink, and Java.  While previous
> > effort
> > > > > > (HUDI-538 [1]) has decoupled the integration with Spark, there is
> > > quite
> > > > > > some code duplication across different engines for almost the
> same
> > > > logic
> > > > > > due to the current interface design.  Some part also has
> divergence
> > > > among
> > > > > > engines, making debugging and support difficult.
> > > > > >
> > > > > > I propose to further refactor the hudi-client module with the
> goal
> > of
> > > > > > improving the code reuse across multiple engines and reducing the
> > > > > > divergence of the logic among them, so that th

Re: [DISCUSS] Refactor hudi-client module for better support of multiple engines

2021-09-16 Thread Gary Li
Huge +1. Recently I am working on making the Flink writer in a streaming
fashion and found the List interface is limiting the
streaming power of Flink. By switching from
HoodieFlinkMergeOnReadTable.upsert(List) to use the
AppendHandle.write(HoodieRecord) directly, the throughput was almost
doubled and the checkpoint time of the writer was reduced from minutes to
seconds. But I found it really difficult to fit this change into the
current client interface.

My 2 cents:

   - The HoodieIOHandle should only handle the IO, and not having
   HoodieTable inside.
   - We need a more streaming-friendly Handle. For Flink, we can definitely
   change all the batch mode List to processing HoodieRecord one
   by one, just like the AppendHandle.write(HoodieRecord) and
   AppendHandle.close(). This will spread the computing cost and
   flattening the curve.
   - We can use the Handle to precisely control the JVM to avoid OOM and
   optimize the memory footprint. Then we don't need to implement another
   memory control mechanism in the compute engine itself.
   - HoodieClient, HoodieTable, HoodieIOHandle, HoodieTimeline,
   HoodieFileSystemView e.t.c should have a well-defined role and well-defined
   layer. We should know when to use what, it should be used by the driver in
   a single thread or used by the worker in a distributed way.

This is a big project and could benefit Hudi in long term. Happy to discuss
more in the design doc or PRs.

Best,
Gary

On Thu, Sep 16, 2021 at 3:21 AM Raymond Xu 
wrote:

> +1 that's a great improvement.
>
> On Wed, Sep 15, 2021 at 10:40 AM Sivabalan  wrote:
>
> > ++1. definitely help's Hudi scale and makes it more maintainable. Thanks
> > for driving this effort. Mostly devs show interest in major features and
> > don't like to spend time in such foundational work. But as the project
> > scales, these foundational work will have a higher returns in the long
> run.
> >
> > On Wed, Sep 15, 2021 at 8:29 AM Vinoth Chandar 
> wrote:
> >
> > > Another +1 ,  HoodieData abstraction will go a long way in reducing
> LoC.
> > >
> > > Happy to work with you to see this through! I really encourage top
> > > contributors to the Flink and Java clients as well,
> > > actively review all PRs, given there are subtle differences everywhere.
> > >
> > > This will help us smoothly provide all the core features across
> engines.
> > > Also help us easily write a DataSet/Row based
> > > client for Spark as well.
> > >
> > > Onwards and upwards
> > > Vinoth
> > >
> > > On Wed, Sep 15, 2021 at 4:57 AM vino yang 
> wrote:
> > >
> > > > Hi Ethan,
> > > >
> > > > Big +1 for the proposal.
> > > >
> > > > Actually, we have discussed this topic before.[1]
> > > >
> > > > Will review your refactor PR later.
> > > >
> > > > Best,
> > > > Vino
> > > >
> > > > [1]:
> > > >
> > > >
> > >
> >
> https://lists.apache.org/thread.html/r71d96d285c735b1611920fb3e7224c9ce6fd53d09bf0e8f144f4fcbd%40%3Cdev.hudi.apache.org%3E
> > > >
> > > >
> > > > Y Ethan Guo  于2021年9月15日周三 下午3:34写道:
> > > >
> > > > > Hi all,
> > > > >
> > > > > hudi-client module has core Hudi abstractions and client logic for
> > > > > different engines like Spark, Flink, and Java.  While previous
> effort
> > > > > (HUDI-538 [1]) has decoupled the integration with Spark, there is
> > quite
> > > > > some code duplication across different engines for almost the same
> > > logic
> > > > > due to the current interface design.  Some part also has divergence
> > > among
> > > > > engines, making debugging and support difficult.
> > > > >
> > > > > I propose to further refactor the hudi-client module with the goal
> of
> > > > > improving the code reuse across multiple engines and reducing the
> > > > > divergence of the logic among them, so that the core Hudi action
> > > > execution
> > > > > logic should be shared across engines, except for engine specific
> > > > > transformations.  Such a pattern also allows easy support of core
> > Hudi
> > > > > functionality for all engines in the future.  Specifically,
> > > > >
> > > > > (1) Abstracts the transformation boilerplates inside the
> > > > > HoodieEngineContext and implements the engine-specific data
> > > > transformation
> > > > > logic in the subclasses.  Type cast can be done inside the engine
> > > > context.
> > > > > (2) Creates new HoodieData abstraction for passing input and output
> > > along
> > > > > the flow of execution, and uses it in different Hudi abstractions,
> > > e.g.,
> > > > > HoodieTable, HoodieIOHandle, BaseActionExecutor, instead of
> enforcing
> > > > type
> > > > > parameters encountering RDD and List
> > which
> > > > are
> > > > > one source of duplication.
> > > > > (3) Extracts common execution logic to hudi-client-common module
> from
> > > > > multiple engines.
> > > > >
> > > > > As a first step and exploration for item (1) and (3) above, I've
> > tried
> > > to
> > > > > refactor the rollback actions and the PR is here [HUDI-2433][2].
> In
> > > this
> > > > > PR, I completel

Re: [DISCUSS] Refactor hudi-client module for better support of multiple engines

2021-09-15 Thread Raymond Xu
+1 that's a great improvement.

On Wed, Sep 15, 2021 at 10:40 AM Sivabalan  wrote:

> ++1. definitely help's Hudi scale and makes it more maintainable. Thanks
> for driving this effort. Mostly devs show interest in major features and
> don't like to spend time in such foundational work. But as the project
> scales, these foundational work will have a higher returns in the long run.
>
> On Wed, Sep 15, 2021 at 8:29 AM Vinoth Chandar  wrote:
>
> > Another +1 ,  HoodieData abstraction will go a long way in reducing LoC.
> >
> > Happy to work with you to see this through! I really encourage top
> > contributors to the Flink and Java clients as well,
> > actively review all PRs, given there are subtle differences everywhere.
> >
> > This will help us smoothly provide all the core features across engines.
> > Also help us easily write a DataSet/Row based
> > client for Spark as well.
> >
> > Onwards and upwards
> > Vinoth
> >
> > On Wed, Sep 15, 2021 at 4:57 AM vino yang  wrote:
> >
> > > Hi Ethan,
> > >
> > > Big +1 for the proposal.
> > >
> > > Actually, we have discussed this topic before.[1]
> > >
> > > Will review your refactor PR later.
> > >
> > > Best,
> > > Vino
> > >
> > > [1]:
> > >
> > >
> >
> https://lists.apache.org/thread.html/r71d96d285c735b1611920fb3e7224c9ce6fd53d09bf0e8f144f4fcbd%40%3Cdev.hudi.apache.org%3E
> > >
> > >
> > > Y Ethan Guo  于2021年9月15日周三 下午3:34写道:
> > >
> > > > Hi all,
> > > >
> > > > hudi-client module has core Hudi abstractions and client logic for
> > > > different engines like Spark, Flink, and Java.  While previous effort
> > > > (HUDI-538 [1]) has decoupled the integration with Spark, there is
> quite
> > > > some code duplication across different engines for almost the same
> > logic
> > > > due to the current interface design.  Some part also has divergence
> > among
> > > > engines, making debugging and support difficult.
> > > >
> > > > I propose to further refactor the hudi-client module with the goal of
> > > > improving the code reuse across multiple engines and reducing the
> > > > divergence of the logic among them, so that the core Hudi action
> > > execution
> > > > logic should be shared across engines, except for engine specific
> > > > transformations.  Such a pattern also allows easy support of core
> Hudi
> > > > functionality for all engines in the future.  Specifically,
> > > >
> > > > (1) Abstracts the transformation boilerplates inside the
> > > > HoodieEngineContext and implements the engine-specific data
> > > transformation
> > > > logic in the subclasses.  Type cast can be done inside the engine
> > > context.
> > > > (2) Creates new HoodieData abstraction for passing input and output
> > along
> > > > the flow of execution, and uses it in different Hudi abstractions,
> > e.g.,
> > > > HoodieTable, HoodieIOHandle, BaseActionExecutor, instead of enforcing
> > > type
> > > > parameters encountering RDD and List
> which
> > > are
> > > > one source of duplication.
> > > > (3) Extracts common execution logic to hudi-client-common module from
> > > > multiple engines.
> > > >
> > > > As a first step and exploration for item (1) and (3) above, I've
> tried
> > to
> > > > refactor the rollback actions and the PR is here [HUDI-2433][2].  In
> > this
> > > > PR, I completely remove all engine-specific rollback packages and
> only
> > > keep
> > > > one rollback package in hudi-client-common, adding ~350 LoC while
> > > deleting
> > > > 1.3K LoC.  My next step is to refactor the commit action which
> > > encompasses
> > > > item (2) above.
> > > >
> > > > What do you folks think and any other suggestions?
> > > >
> > > > [1] [HUDI-538] [UMBRELLA] Restructuring hudi client module for multi
> > > engine
> > > > support
> > > > https://issues.apache.org/jira/browse/HUDI-538
> > > > [2] PR: [HUDI-2433] Refactor rollback actions in hudi-client module
> > > > https://github.com/apache/hudi/pull/3664/files
> > > >
> > > > Best,
> > > > - Ethan
> > > >
> > >
> >
>
>
> --
> Regards,
> -Sivabalan
>


Re: [DISCUSS] Refactor hudi-client module for better support of multiple engines

2021-09-15 Thread Sivabalan
++1. definitely help's Hudi scale and makes it more maintainable. Thanks
for driving this effort. Mostly devs show interest in major features and
don't like to spend time in such foundational work. But as the project
scales, these foundational work will have a higher returns in the long run.

On Wed, Sep 15, 2021 at 8:29 AM Vinoth Chandar  wrote:

> Another +1 ,  HoodieData abstraction will go a long way in reducing LoC.
>
> Happy to work with you to see this through! I really encourage top
> contributors to the Flink and Java clients as well,
> actively review all PRs, given there are subtle differences everywhere.
>
> This will help us smoothly provide all the core features across engines.
> Also help us easily write a DataSet/Row based
> client for Spark as well.
>
> Onwards and upwards
> Vinoth
>
> On Wed, Sep 15, 2021 at 4:57 AM vino yang  wrote:
>
> > Hi Ethan,
> >
> > Big +1 for the proposal.
> >
> > Actually, we have discussed this topic before.[1]
> >
> > Will review your refactor PR later.
> >
> > Best,
> > Vino
> >
> > [1]:
> >
> >
> https://lists.apache.org/thread.html/r71d96d285c735b1611920fb3e7224c9ce6fd53d09bf0e8f144f4fcbd%40%3Cdev.hudi.apache.org%3E
> >
> >
> > Y Ethan Guo  于2021年9月15日周三 下午3:34写道:
> >
> > > Hi all,
> > >
> > > hudi-client module has core Hudi abstractions and client logic for
> > > different engines like Spark, Flink, and Java.  While previous effort
> > > (HUDI-538 [1]) has decoupled the integration with Spark, there is quite
> > > some code duplication across different engines for almost the same
> logic
> > > due to the current interface design.  Some part also has divergence
> among
> > > engines, making debugging and support difficult.
> > >
> > > I propose to further refactor the hudi-client module with the goal of
> > > improving the code reuse across multiple engines and reducing the
> > > divergence of the logic among them, so that the core Hudi action
> > execution
> > > logic should be shared across engines, except for engine specific
> > > transformations.  Such a pattern also allows easy support of core Hudi
> > > functionality for all engines in the future.  Specifically,
> > >
> > > (1) Abstracts the transformation boilerplates inside the
> > > HoodieEngineContext and implements the engine-specific data
> > transformation
> > > logic in the subclasses.  Type cast can be done inside the engine
> > context.
> > > (2) Creates new HoodieData abstraction for passing input and output
> along
> > > the flow of execution, and uses it in different Hudi abstractions,
> e.g.,
> > > HoodieTable, HoodieIOHandle, BaseActionExecutor, instead of enforcing
> > type
> > > parameters encountering RDD and List which
> > are
> > > one source of duplication.
> > > (3) Extracts common execution logic to hudi-client-common module from
> > > multiple engines.
> > >
> > > As a first step and exploration for item (1) and (3) above, I've tried
> to
> > > refactor the rollback actions and the PR is here [HUDI-2433][2].  In
> this
> > > PR, I completely remove all engine-specific rollback packages and only
> > keep
> > > one rollback package in hudi-client-common, adding ~350 LoC while
> > deleting
> > > 1.3K LoC.  My next step is to refactor the commit action which
> > encompasses
> > > item (2) above.
> > >
> > > What do you folks think and any other suggestions?
> > >
> > > [1] [HUDI-538] [UMBRELLA] Restructuring hudi client module for multi
> > engine
> > > support
> > > https://issues.apache.org/jira/browse/HUDI-538
> > > [2] PR: [HUDI-2433] Refactor rollback actions in hudi-client module
> > > https://github.com/apache/hudi/pull/3664/files
> > >
> > > Best,
> > > - Ethan
> > >
> >
>


-- 
Regards,
-Sivabalan


Re: [DISCUSS] Refactor hudi-client module for better support of multiple engines

2021-09-15 Thread Vinoth Chandar
Another +1 ,  HoodieData abstraction will go a long way in reducing LoC.

Happy to work with you to see this through! I really encourage top
contributors to the Flink and Java clients as well,
actively review all PRs, given there are subtle differences everywhere.

This will help us smoothly provide all the core features across engines.
Also help us easily write a DataSet/Row based
client for Spark as well.

Onwards and upwards
Vinoth

On Wed, Sep 15, 2021 at 4:57 AM vino yang  wrote:

> Hi Ethan,
>
> Big +1 for the proposal.
>
> Actually, we have discussed this topic before.[1]
>
> Will review your refactor PR later.
>
> Best,
> Vino
>
> [1]:
>
> https://lists.apache.org/thread.html/r71d96d285c735b1611920fb3e7224c9ce6fd53d09bf0e8f144f4fcbd%40%3Cdev.hudi.apache.org%3E
>
>
> Y Ethan Guo  于2021年9月15日周三 下午3:34写道:
>
> > Hi all,
> >
> > hudi-client module has core Hudi abstractions and client logic for
> > different engines like Spark, Flink, and Java.  While previous effort
> > (HUDI-538 [1]) has decoupled the integration with Spark, there is quite
> > some code duplication across different engines for almost the same logic
> > due to the current interface design.  Some part also has divergence among
> > engines, making debugging and support difficult.
> >
> > I propose to further refactor the hudi-client module with the goal of
> > improving the code reuse across multiple engines and reducing the
> > divergence of the logic among them, so that the core Hudi action
> execution
> > logic should be shared across engines, except for engine specific
> > transformations.  Such a pattern also allows easy support of core Hudi
> > functionality for all engines in the future.  Specifically,
> >
> > (1) Abstracts the transformation boilerplates inside the
> > HoodieEngineContext and implements the engine-specific data
> transformation
> > logic in the subclasses.  Type cast can be done inside the engine
> context.
> > (2) Creates new HoodieData abstraction for passing input and output along
> > the flow of execution, and uses it in different Hudi abstractions, e.g.,
> > HoodieTable, HoodieIOHandle, BaseActionExecutor, instead of enforcing
> type
> > parameters encountering RDD and List which
> are
> > one source of duplication.
> > (3) Extracts common execution logic to hudi-client-common module from
> > multiple engines.
> >
> > As a first step and exploration for item (1) and (3) above, I've tried to
> > refactor the rollback actions and the PR is here [HUDI-2433][2].  In this
> > PR, I completely remove all engine-specific rollback packages and only
> keep
> > one rollback package in hudi-client-common, adding ~350 LoC while
> deleting
> > 1.3K LoC.  My next step is to refactor the commit action which
> encompasses
> > item (2) above.
> >
> > What do you folks think and any other suggestions?
> >
> > [1] [HUDI-538] [UMBRELLA] Restructuring hudi client module for multi
> engine
> > support
> > https://issues.apache.org/jira/browse/HUDI-538
> > [2] PR: [HUDI-2433] Refactor rollback actions in hudi-client module
> > https://github.com/apache/hudi/pull/3664/files
> >
> > Best,
> > - Ethan
> >
>


Re: [DISCUSS] Refactor hudi-client module for better support of multiple engines

2021-09-15 Thread vino yang
Hi Ethan,

Big +1 for the proposal.

Actually, we have discussed this topic before.[1]

Will review your refactor PR later.

Best,
Vino

[1]:
https://lists.apache.org/thread.html/r71d96d285c735b1611920fb3e7224c9ce6fd53d09bf0e8f144f4fcbd%40%3Cdev.hudi.apache.org%3E


Y Ethan Guo  于2021年9月15日周三 下午3:34写道:

> Hi all,
>
> hudi-client module has core Hudi abstractions and client logic for
> different engines like Spark, Flink, and Java.  While previous effort
> (HUDI-538 [1]) has decoupled the integration with Spark, there is quite
> some code duplication across different engines for almost the same logic
> due to the current interface design.  Some part also has divergence among
> engines, making debugging and support difficult.
>
> I propose to further refactor the hudi-client module with the goal of
> improving the code reuse across multiple engines and reducing the
> divergence of the logic among them, so that the core Hudi action execution
> logic should be shared across engines, except for engine specific
> transformations.  Such a pattern also allows easy support of core Hudi
> functionality for all engines in the future.  Specifically,
>
> (1) Abstracts the transformation boilerplates inside the
> HoodieEngineContext and implements the engine-specific data transformation
> logic in the subclasses.  Type cast can be done inside the engine context.
> (2) Creates new HoodieData abstraction for passing input and output along
> the flow of execution, and uses it in different Hudi abstractions, e.g.,
> HoodieTable, HoodieIOHandle, BaseActionExecutor, instead of enforcing type
> parameters encountering RDD and List which are
> one source of duplication.
> (3) Extracts common execution logic to hudi-client-common module from
> multiple engines.
>
> As a first step and exploration for item (1) and (3) above, I've tried to
> refactor the rollback actions and the PR is here [HUDI-2433][2].  In this
> PR, I completely remove all engine-specific rollback packages and only keep
> one rollback package in hudi-client-common, adding ~350 LoC while deleting
> 1.3K LoC.  My next step is to refactor the commit action which encompasses
> item (2) above.
>
> What do you folks think and any other suggestions?
>
> [1] [HUDI-538] [UMBRELLA] Restructuring hudi client module for multi engine
> support
> https://issues.apache.org/jira/browse/HUDI-538
> [2] PR: [HUDI-2433] Refactor rollback actions in hudi-client module
> https://github.com/apache/hudi/pull/3664/files
>
> Best,
> - Ethan
>


[DISCUSS] Refactor hudi-client module for better support of multiple engines

2021-09-15 Thread Y Ethan Guo
Hi all,

hudi-client module has core Hudi abstractions and client logic for
different engines like Spark, Flink, and Java.  While previous effort
(HUDI-538 [1]) has decoupled the integration with Spark, there is quite
some code duplication across different engines for almost the same logic
due to the current interface design.  Some part also has divergence among
engines, making debugging and support difficult.

I propose to further refactor the hudi-client module with the goal of
improving the code reuse across multiple engines and reducing the
divergence of the logic among them, so that the core Hudi action execution
logic should be shared across engines, except for engine specific
transformations.  Such a pattern also allows easy support of core Hudi
functionality for all engines in the future.  Specifically,

(1) Abstracts the transformation boilerplates inside the
HoodieEngineContext and implements the engine-specific data transformation
logic in the subclasses.  Type cast can be done inside the engine context.
(2) Creates new HoodieData abstraction for passing input and output along
the flow of execution, and uses it in different Hudi abstractions, e.g.,
HoodieTable, HoodieIOHandle, BaseActionExecutor, instead of enforcing type
parameters encountering RDD and List which are
one source of duplication.
(3) Extracts common execution logic to hudi-client-common module from
multiple engines.

As a first step and exploration for item (1) and (3) above, I've tried to
refactor the rollback actions and the PR is here [HUDI-2433][2].  In this
PR, I completely remove all engine-specific rollback packages and only keep
one rollback package in hudi-client-common, adding ~350 LoC while deleting
1.3K LoC.  My next step is to refactor the commit action which encompasses
item (2) above.

What do you folks think and any other suggestions?

[1] [HUDI-538] [UMBRELLA] Restructuring hudi client module for multi engine
support
https://issues.apache.org/jira/browse/HUDI-538
[2] PR: [HUDI-2433] Refactor rollback actions in hudi-client module
https://github.com/apache/hudi/pull/3664/files

Best,
- Ethan