Re: [DISCUSS] Hudi Reverse Streamer

2023-06-14 Thread Rajesh Mahindra
Great. We also need it for use cases of loading data into warehouses, and
would love to help.

On Wed, Jun 14, 2023 at 9:06 AM Pratyaksh Sharma 
wrote:

> Hi,
>
> I missed this email earlier. Sure let me start an RFC this week and we can
> take it from there.
>
> On Wed, Jun 14, 2023 at 9:20 PM Nicolas Paris 
> wrote:
>
> > Hi any rfc/ongoing efforts on the reverse delta streamer ? We have a use
> > case to do hudi => Kafka and would enjoy building a more general tool.
> >
> > However we need a rfc basis to start some effort in the right way
> >
> > On April 12, 2023 3:08:22 AM UTC, Vinoth Chandar <
> > mail.vinoth.chan...@gmail.com> wrote:
> > >Cool. lets draw up a RFC for this? @pratyaksh - do you want to start
> one,
> > >given you expressed interest?
> > >
> > >On Mon, Apr 10, 2023 at 7:32 PM Léo Biscassi 
> > wrote:
> > >
> > >> +1
> > >> This would be great!
> > >>
> > >> Cheers,
> > >>
> > >> On Mon, Apr 3, 2023 at 3:00 PM Pratyaksh Sharma <
> pratyaks...@gmail.com>
> > >> wrote:
> > >>
> > >> > Hi Vinoth,
> > >> >
> > >> > I am aligned with the first reason that you mentioned. Better to
> have
> > a
> > >> > separate tool to take care of this.
> > >> >
> > >> > On Mon, Apr 3, 2023 at 9:01 PM Vinoth Chandar <
> > >> > mail.vinoth.chan...@gmail.com>
> > >> > wrote:
> > >> >
> > >> > > +1
> > >> > >
> > >> > > I was thinking that we add a new utility and NOT extend
> > DeltaStreamer
> > >> by
> > >> > > adding a Sink interface, for the following reasons
> > >> > >
> > >> > > - It will make it look like a generic Source => Sink ETL tool,
> > which is
> > >> > > actually not our intention to support on Hudi. There are plenty of
> > good
> > >> > > tools for that out there.
> > >> > > - the config management can get bit hard to understand, since we
> > >> overload
> > >> > > ingest and reverse ETL into a single tool. So break it off at
> > use-case
> > >> > > level?
> > >> > >
> > >> > > Thoughts?
> > >> > >
> > >> > > David:  PMC does not have control over that. Please see
> unsubscribe
> > >> > > instructions here. https://hudi.apache.org/community/get-involved
> > >> > > Love to keep this thread about reverse streamer discussion. So
> > kindly
> > >> > fork
> > >> > > another thread if you want to discuss unsubscribing.
> > >> > >
> > >> > > On Fri, Mar 31, 2023 at 1:47 AM Davidiam  >
> > >> > wrote:
> > >> > >
> > >> > > > Hello Vinoth,
> > >> > > >
> > >> > > > Can you please unsubscribe me?  I have been trying to
> unsubscribe
> > for
> > >> > > > months without success.
> > >> > > >
> > >> > > > Kind Regards,
> > >> > > > David
> > >> > > >
> > >> > > > Sent from Outlook for Android<https://aka.ms/AAb9ysg>
> > >> > > > 
> > >> > > > From: Vinoth Chandar 
> > >> > > > Sent: Friday, March 31, 2023 5:09:52 AM
> > >> > > > To: dev 
> > >> > > > Subject: [DISCUSS] Hudi Reverse Streamer
> > >> > > >
> > >> > > > Hi all,
> > >> > > >
> > >> > > > Any interest in building a reverse streaming tool, that does the
> > >> > reverse
> > >> > > of
> > >> > > > what the DeltaStreamer tool does? It will read Hudi table
> > >> incrementally
> > >> > > > (only source) and write out the data to a variety of sinks -
> > Kafka,
> > >> > JDBC
> > >> > > > Databases, DFS.
> > >> > > >
> > >> > > > This has come up many times with data warehouse users. Often
> > times,
> > >> > they
> > >> > > > want to use Hudi to speed up or reduce costs on their data
> > ingestion
> > >> > and
> > >> > > > ETL (using Spark/Flink), but want to move the derived data back
> > into
> > >> a
> > >> > > data
> > >> > > > warehouse or an operational database for serving.
> > >> > > >
> > >> > > > What do you all think?
> > >> > > >
> > >> > > > Thanks
> > >> > > > Vinoth
> > >> > > >
> > >> > >
> > >> >
> > >>
> > >>
> > >> --
> > >> *Léo Biscassi*
> > >> Blog - https://leobiscassi.com
> > >>
> > >>-
> > >>
> >
>


-- 
Take Care,
Rajesh Mahindra


Re: [DISCUSS] Hudi Community Communication Updates

2021-12-10 Thread Rajesh Mahindra
Hi folks,

We are starting to roll the first feature out. Starting today, all dev
email threads will show up on slack channel #devwork.

Thanks
Rajesh

On Wed, Nov 10, 2021 at 9:33 AM Vinoth Chandar  wrote:

> +1 for this. We will also archive all community activity on ASF
> infrastructure this way!
>
> On Wed, Nov 10, 2021 at 7:14 AM Pratyaksh Sharma 
> wrote:
>
>> Hi Rajesh,
>>
>> I do not have any strong opinions for/against point #1.
>>
>> Point #2 definitely seems useful to me.
>> I hope messages from #general channel will be formatted as respective
>> threads in either case - if the thread started on the same day or if some
>> reply comes on some ongoing thread.
>>
>> On Tue, Nov 9, 2021 at 10:48 PM Rajesh Mahindra 
>> wrote:
>>
>> > Hi Folks,
>> >
>> > We are thinking of improving community communication, and here is a
>> list of
>> > things we propose to do (some of them are already available).
>> >
>> >
>> https://cwiki.apache.org/confluence/display/HUDI/Community+Communication
>> >
>> > While we have a few things proposed, we plan to prioritize the following
>> > two features to hopefully ease communication. We plan to extend the
>> current
>> > hudi bot (slack app) to implement the features.
>> >
>> >
>> >1. As you are aware, we have 2 mailing lists: *dev* and *users*. To
>> make
>> >it easier for users to get these updates on a single platform
>> (slack),
>> > we
>> >propose to enhance hudi-bot to mirror the emails from the 2 mailing
>> > lists
>> >and post them into respective slack channels (*#dev* and *#users*).
>> The
>> >sync will happen in near-real-time (a few mins). To ease
>> readability, an
>> >email thread will be synced as a single slack thread, i.e., the
>> replies
>> > on
>> >an email will be synced as replies on the slack channel to the
>> original
>> >post.
>> >2. It is often hard to catch up on historical slack messages. To ease
>> >catching up on slack messages (*#general*) in the past few days, we
>> will
>> >further extend the functionality of the hudi bot to read all the
>> slack
>> >messages of a day, format them, and send them as a single email
>> digest
>> > to
>> >the *dev* mailing list.
>> >
>> >
>> >
>> > Please let us know what you think of this proposal, and feel free to
>> raise
>> > concerns if any.
>> >
>> > Thanks,
>> > Rajesh
>> >
>>
>

-- 
Take Care,
Rajesh Mahindra


Re: [DISCUSS] Hudi 0.10.0 Release

2021-11-19 Thread Rajesh Mahindra
Hi Danny,

I have the following blockers that have a PR up. I am working on a PR for
the Debezium Source. I am fine with Nov 26th as cut off.

   - [HUDI-2325] Implement and test Hive Sync support for Kafka Connect
   (Owner: Rajesh Mahindra)
   - [HUDI-2671] Fix record offset handling in Kafka connect transaction
   participant (Owner: Rajesh Mahindra)
   - [HUDI-2672] Avoid empty commits and rollbacks when there is no event
   from the topic (Owner: Rajesh Mahindra)

** Pending
   - [HUDI-1290] Implement Debezium avro source for Delta Streamer

Thanks
Rajesh


On Fri, Nov 19, 2021 at 4:01 PM Udit Mehrotra  wrote:

> Hi Danny,
>
> I have a blocker as well
> https://issues.apache.org/jira/browse/HUDI-2802. Nov 26th cut off date
> works fine for me.
>
> Also, just an update on the above list: HUDI-2641, HUDI-2314,
> HUDI-2362 are unblocked/merged. HUDI-2314 and HUDI-2362 can be marked
> in the highlights section. We will work on getting some doc updates
> for the same by next week.
>
> Thanks,
> Udit
>
> On Fri, Nov 19, 2021 at 3:49 PM Vinoth Chandar  wrote:
> >
> > Hi Danny,
> >
> > I have one blocker. I plan to complete it by end of next week. I am good
> > with the prior Nov 26 cutoff.
> > Does that work for everyone?
> >
> > Thanks
> > Vinoth
> >
> > On Fri, Nov 19, 2021 at 12:12 AM Danny Chan 
> wrote:
> >
> > > Hi Community,
> > >
> > > As we draw close to doing Hudi 0.10.0 release, I am happy to share a
> > > summary of the key features/improvements that would be going in the
> release
> > > and the current blockers for everyone's visibility.
> > >
> > > *Highlights*
> > >
> > >- [HUDI-1290] Implement Debezium avro source for Delta Streamer
> > >- [HUDI-1491] Support partition pruning for MOR snapshot query
> > >- [HUDI-1763] DefaultHoodieRecordPayload does not honor ordering
> value
> > >when records within multiple log files are merged
> > >- [HUDI-1827] Add ORC support in Bootstrap Op
> > >- [HUDI-1869] Upgrading Spark3 To 3.1
> > >- [HUDI-2101] support z-order for hudi
> > >- [HUDI-2276] Enable Metadata Table by default for both writers and
> > >readers
> > >- [HUDI-2581] Analyze metadata size estimate in hudi with Hfile for
> col
> > >stats partition
> > >- [HUDI-2634] Improve bootstrap performance for very large tables
> > >- [HUDI-2086] redo the logical of mor_incremental_view for hive
> > >- [HUDI-2191] Bump flink version to 1.13.1
> > >- [HUDI-2285] Metadata Table Synchronous Design
> > >- [HUDI-2316] Support Flink batch upsert
> > >- [HUDI-2371] Improve flink streaming reader
> > >- [HUDI-2394] [Kafka Connect Mileston 1] Implement kafka connect for
> > >immutable data
> > >- [HUDI-2449] Incremental read for Flink
> > >- [HUDI-2562] Embedded timeline server on JobManager
> > >
> > > *Current Blockers*
> > >
> > >- [HUDI-1856] Upstream changes made in PrestoDB to eliminate file
> > >listing to Trino (Owner: Sagar Sumit)
> > >- [HUDI-1912] Presto defaults to GenericHiveRecordCursor for all
> Hudi
> > >tables (Owner: Sagar Sumit)
> > >- [HUDI-1932] Hive Sync should not always update
> last_commit_time_sync
> > >(Owner: Raymond Xu)
> > >- [HUDI-1937] When clustering fail, generating unfinished
> replacecommit
> > >timeline. (Owner: Sagar Sumit)
> > >- [HUDI-2077] Flaky test: TestHoodieDeltaStreamer (Owner: Sagar
> Sumit)
> > >- [HUDI-2314] Add DynamoDb based lock provider (Owner: Wenning Ding)
> > >- [HUDI-2325] Implement and test Hive Sync support for Kafka Connect
> > >(Owner: Rajesh Mahindra)
> > >- [HUDI-2332] Implement scheduling of compaction/ clustering for
> Kafka
> > >Connect (Owner: Ethan Guo)
> > >- [HUDI-2362] Hudi external configuration file support (Owner:
> Wenning
> > >Ding)
> > >- [HUDI-2409] Using HBase shaded jars in Hudi presto bundle (Owner:
> > >Sagar Sumit)
> > >- [HUDI-2443] KVComparator in HFile for metadata table is tied to
> HBase
> > >version and shading (Owner: Sagar Sumit)
> > >- [HUDI-2472] Tests failure follow up when metadata is enabled by
> > >default (Owner: Manoj Govindassamy)
> > >- [HUDI-2475] Rolling Upgrade downgrade story for 0.10 & enabling
> > >metadata (Owner: Manoj Govindassamy)
> > >- [HUDI-

[DISCUSS] Propose to implement a deltastreamer source for Debezium

2021-11-19 Thread Rajesh Mahindra
Hi Community,

We intend to implement a source for ingesting Debezium Change Data Capture
(CDC) logs into Deltastreamer/ Hudi. With this capability, we can
continuously capture row-level changes that insert, update and delete
records that were committed to a database. While debezium supports multiple
databases, we will focus on postgres and mysql initially.

More details are published on the cwiki:
 https://cwiki.apache.org/confluence/display/HUDI/RFC+-+38
<https://cwiki.apache.org/confluence/display/HUDI/RFC+-+38>. This doc is
not a formal RFC yet, just to aid in the discussion here. We will publish
the RFC based on early feedback.

Thanks
Rajesh Mahindra


[DISCUSS] Hudi Community Communication Updates

2021-11-09 Thread Rajesh Mahindra
Hi Folks,

We are thinking of improving community communication, and here is a list of
things we propose to do (some of them are already available).

https://cwiki.apache.org/confluence/display/HUDI/Community+Communication

While we have a few things proposed, we plan to prioritize the following
two features to hopefully ease communication. We plan to extend the current
hudi bot (slack app) to implement the features.


   1. As you are aware, we have 2 mailing lists: *dev* and *users*. To make
   it easier for users to get these updates on a single platform (slack), we
   propose to enhance hudi-bot to mirror the emails from the 2 mailing lists
   and post them into respective slack channels (*#dev* and *#users*). The
   sync will happen in near-real-time (a few mins). To ease readability, an
   email thread will be synced as a single slack thread, i.e., the replies on
   an email will be synced as replies on the slack channel to the original
   post.
   2. It is often hard to catch up on historical slack messages. To ease
   catching up on slack messages (*#general*) in the past few days, we will
   further extend the functionality of the hudi bot to read all the slack
   messages of a day, format them, and send them as a single email digest to
   the *dev* mailing list.



Please let us know what you think of this proposal, and feel free to raise
concerns if any.

Thanks,
Rajesh


Re: New site/docs navigation

2021-10-28 Thread Rajesh Mahindra
This is awesome! Well done Kyle.

Thanks
Rajesh

On Thu, Oct 28, 2021 at 7:35 PM sagar sumit  wrote:

> This is awesome!
> I really like the separation of `Concepts` and `Services`. Very helpful for
> Hudi users in my opinion.
>
> One suggestion:
> Thinking from a newbie perspective who is thinking of adopting data lake,
> would it be better to move the `Use Cases` section right below `Overview`
> for better visibility?
> For e.g. Presto/Trino have mentioned use cases right under the Overview
> section.
>
> Regards,
> Sagar
>
> On Thu, Oct 28, 2021 at 6:55 PM Vinoth Chandar  wrote:
>
> > Awesome!
> > I think Kyle has already fixed some issues around cn docs in the PR
> above.
> > Could you review that?
> > Kyle, if you are here, please chime in. We can organize all the work
> under
> > a single umbrella JIRA.
> > https://issues.apache.org/jira/browse/HUDI-270 so its easier for any
> > volunteers to pick up?
> >
> > On Thu, Oct 28, 2021 at 6:21 AM Shawy Geng 
> > wrote:
> >
> > > Hi Vinoth,
> > >
> > > Volunteer to update the Chinese doc. Already commented at the
> > > https://issues.apache.org/jira/browse/HUDI-2628 <
> > > https://issues.apache.org/jira/browse/HUDI-2628>.
> > > Are there any other volunteers who want to work together to translate?
> > > Please contact me.
> > >
> > > > 2021年10月28日 20:35,Vinoth Chandar  写道:
> > > >
> > > > Hi all,
> > > >
> > > > https://github.com/apache/hudi/pull/3855 puts up a nice redesign of
> > the
> > > > content, that can show case all of the Hudi capabilities. Please
> chime
> > in
> > > > and help merge the PR.
> > > >
> > > > As follow on, we can also fix the Chinese site docs after this?
> > > >
> > > > Thanks
> > > > Vinoth
> > >
> > >
> >
>


-- 
Take Care,
Rajesh Mahindra


Re: [ANNOUNCE] Apache Hudi 0.9.0 released

2021-09-03 Thread Rajesh Mahindra
ore on this
> > > > > can be found in this blog
> > > > > <https://hudi.apache.org/blog/2021/08/23/async-clustering/>.
> > > > > Users can choose to drop fields used to generate partition paths.
> > > > > Added a new write operation "delete_partition" support in spark.
> > Users
> > > > can
> > > > > leverage this to delete older partitions in
> > > > > bulk, in addition to record level deletes.
> > > > > Added Support for Huawei Cloud Object Storage, BAIDU AFS storage
> > > format,
> > > > > Baidu BOS storage in Hudi.
> > > > > A pre commit validator framework
> > > > > <
> > > > >
> > > >
> > >
> >
> https://github.com/apache/hudi/blob/bf5a52e51bbeaa089995335a0a4c55884792e505/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/validator/SparkPreCommitValidator.java
> > > > > >
> > > > > has been added for spark engine, which can be used for
> DeltaStreamer
> > > and
> > > > > Spark
> > > > > Datasource writers. Users can leverage this to add any validations
> to
> > > be
> > > > > executed before committing writes to Hudi.
> > > > > Few out of the box validators are available like
> > > > > SqlQueryEqualityPreCommitValidator
> > > > > <
> > > > >
> > > >
> > >
> >
> https://github.com/apache/hudi/blob/release-0.9.0/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/validator/SqlQueryEqualityPreCommitValidator.java
> > > > > >,
> > > > > SqlQueryInequalityPreCommitValidator
> > > > > <
> > > > >
> > > >
> > >
> >
> https://github.com/apache/hudi/blob/release-0.9.0/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/validator/SqlQueryInequalityPreCommitValidator.java
> > > > > >
> > > > > and SqlQuerySingleResultPreCommitValidator
> > > > > <
> > > > >
> > > >
> > >
> >
> https://github.com/apache/hudi/blob/master/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/validator/SqlQuerySingleResultPreCommitValidator.java
> > > > > >
> > > > > .
> > > > >
> > > > > *Flink Integration Improvements*
> > > > > The Flink writer now supports propagation of CDC format for MOR
> > table,
> > > by
> > > > > turning on the option "changelog.enabled=true".
> > > > > Hudi would then persist all change flags of each record, allowing
> > users
> > > > to
> > > > > do stateful computation based on these change logs.
> > > > > Flink writing is now close to feature parity with spark writing,
> with
> > > > > addition of write operations like "bulk_insert" and
> > > > > "insert_overwrite", support for non-partitioned tables, automatic
> > > cleanup
> > > > > of uncommitted data, global indexing support, hive
> > > > > style partitioning and handling of partition path updates.
> > > > > Writing also supports a new log append mode, where no records are
> > > > > de-duplicated and base files are directly written for each flush.
> > > > > Flink readers now support streaming reads from COW/MOR tables.
> > > Deletions
> > > > > are emitted by default in streaming read mode, the
> > > > > downstream receives the "DELETE" message as a Hoodie record with
> > empty
> > > > > payload.
> > > > > Hive sync has been improved by adding support for different Hive
> > > versions
> > > > > and asynchronous execution.
> > > > > Flink Streamer tool now supports transformers.
> > > > >
> > > > > *DeltaStreamer Improvements*
> > > > > We have enhanced DeltaStreamer utility with 3 new sources. JDBC
> > > > > <
> > > > >
> > > >
> > >
> >
> https://github.com/apache/hudi/blob/release-0.9.0/hudi-utilities/src/main/java/org/apache/hudi/utilities/sources/JdbcSource.java
> > > > > >
> > > > > will help with fetching data from RDBMS sources and
> > > > > SQLSource
> > > > > <
> > > > >
> > > >
> > >
> >
> https://github.com/apache/hudi/blob/release-0.9.0/hudi-utilities/src/main/java/org/apache/hudi/utilities/sources/SqlSource.java
> > > > > >
> > > > > will assist in backfilling use cases. S3EventsHoodieIncrSource
> > > > > <
> > > > >
> > > >
> > >
> >
> https://github.com/apache/hudi/blob/release-0.9.0/hudi-utilities/src/main/java/org/apache/hudi/utilities/sources/S3EventsHoodieIncrSource.java
> > > > > >
> > > > > and S3EventsSource
> > > > > <
> > > > >
> > > >
> > >
> >
> https://github.com/apache/hudi/blob/release-0.9.0/hudi-utilities/src/main/java/org/apache/hudi/utilities/sources/S3EventsSource.java
> > > > > >
> > > > > assist in reading data from S3
> > > > > reliably and efficiently ingesting that to Hudi. In addition, we
> have
> > > > added
> > > > > support for timestamp based fetch from kafka and added
> > > > > basic auth support to schema registry.
> > > > >
> > > > > Please find more information about the release here:
> > > > > https://hudi.apache.org/releases/release-0.9.0
> > > > >
> > > > > For details on how to use Hudi, please look at the quick start page
> > > > located
> > > > > here:
> > > > > https://hudi.apache.org/docs/quick-start-guide.html
> > > > >
> > > > > If you'd like to download the source release, you can find it here:
> > > > > https://github.com/apache/hudi/releases/tag/release-0.9.0
> > > > >
> > > > > You can read more about the release (including release notes) here:
> > > > >
> > > > >
> > > >
> > >
> >
> https://issues.apache.org/jira/secure/ReleaseNote.jspa?projectId=12322822=12350027
> > > > >
> > > > > We welcome your help and feedback. For more information on how to
> > > report
> > > > > problems, and to get involved, visit the project
> > > > > website at https://hudi.apache.org/
> > > > >
> > > > > Thanks to everyone involved!
> > > > >
> > > > > Udit Mehrotra
> > > > > (on behalf of the Hudi Community)
> > > > >
> > > >
> > >
> >
>
-- 
Take Care,
Rajesh Mahindra


DISCUSS RFC RFC-32 Kafka Connect Sink for Hudi

2021-08-18 Thread Rajesh Mahindra
Hi All,

We have a new RFC, RFC-32
<https://cwiki.apache.org/confluence/display/HUDI/RFC-32+Kafka+Connect+Sink+for+Hudi>
that
details the design and implementation of a Kafka Sink for Hudi, ensuring
that Kafka connect users can easily ingest / stream kafka records directly
to Hudi Tables.

Please review the RFC and provide any feedback. Do not hesitate to ask
questions if any. Thanks a lot.

-- 
Rajesh Mahindra