Re: [DISCUSS] Decouple Hudi and Spark

2019-09-16 Thread Vinoth Chandar
Alright then. Happy to take the lead here. But please give me a week or so, to finish up the spark bundling and other jar issues.. Too much context switching :) On Mon, Sep 16, 2019 at 6:57 PM vino yang wrote: > Hi guys, > > Currently, I am busy with HUDI-203[1] and other things. > > I agree

Re: [DISCUSS] Decouple Hudi and Spark

2019-09-16 Thread vino yang
Hi guys, Currently, I am busy with HUDI-203[1] and other things. I agree with Vinoth that we should try to find a new solution to decouple the dependency with the Spark RDD cache. It's an excellent way to start this big work. [1]: https://issues.apache.org/jira/browse/HUDI-203

Re: Merging schema's during Incremental load

2019-09-16 Thread vbal...@apache.org
Hi Gautam, Independent of using Hudi, it is best practice to manage schema of your organization's datasets using some central mechanism like schema registry. Without this, it is pretty difficult to evolve schema. It is schema-registry's responsibility for providing the correct schema for your

Re: Field not found in record HoodieException

2019-09-16 Thread Gary Li
Hello, I have seen this exception before. In my case, if the precombine key of one entry is null, then I will have this error. I'd recommend checking if there is any row has null in *last_update.* Best, Gary On Mon, Sep 16, 2019 at 12:32 PM Kabeer Ahmed wrote: > Taher, > > Let me spin a test

Re: [DISCUSS] Decouple Hudi and Spark

2019-09-16 Thread vbal...@apache.org
+1 This is a pretty large undertaking. While the community is getting their hands dirty and ramping up on Hudi internals, it would be productive if Vinoth shepherds this Balaji.VOn Monday, September 16, 2019, 11:30:44 AM PDT, Vinoth Chandar wrote: sg. :) I will wait for others on

Re: Field not found in record HoodieException

2019-09-16 Thread Kabeer Ahmed
Taher, Let me spin a test for you to test similar scenario and let me revert back to you. On Sep 16 2019, at 2:09 pm, Taher Koitawala wrote: > Hi Kabeer, hive table has everything as a string. However when fetching > data, the spark query is > .sql(String.format("select

Re: [DISCUSS] Decouple Hudi and Spark

2019-09-16 Thread Vinoth Chandar
sg. :) I will wait for others on this thread as well to chime in. On Mon, Sep 16, 2019 at 11:27 AM Taher Koitawala wrote: > Vinoth, I think right now given your experience with the project you should > be scoping out what needs to be done to take us there. So +1 for giving you > more work :) >

Re: [DISCUSS] Decouple Hudi and Spark

2019-09-16 Thread Taher Koitawala
Vinoth, I think right now given your experience with the project you should be scoping out what needs to be done to take us there. So +1 for giving you more work :) We want to reach a point where we can start scoping out addition of Flink and Beam components within. Then I think will tremendous

Re: [DISCUSS] Decouple Hudi and Spark

2019-09-16 Thread Vinoth Chandar
I still feel the key thing here is reimplementing HoodieBloomIndex without needing spark caching. https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=103093742#Design(non-global) documents the spark DAG in detail. If everyone feels, it's best for me to scope the work out, then happy

Re: [DISCUSS] [VOTE] JDBC incremental load with DeltaStreamer

2019-09-16 Thread Vinoth Chandar
It should work like any other source and none of the others are aware if whether deltaStreamer is running in continuous mode or not. Simplistically, it just needs a config to denote an incremental field - say `_last_modified_at` and we use that as a checkpoint to tail that table by including a

Re: [DISCUSS] Decouple Hudi and Spark

2019-09-16 Thread Taher Koitawala
Guys I think we are slowing down on this again. We need to start planning small small tasks towards this VC please can you help fast track this? Regards, Taher Koitawala On Thu, Aug 15, 2019, 10:07 AM Vinoth Chandar wrote: > Look forward to the analysis. A key class to read would be >

Re: [DISCUSS] [VOTE] JDBC incremental load with DeltaStreamer

2019-09-16 Thread Taher Koitawala
Will this be the same implementation as session.read.jdbc("") and then call this code continuously like how we are running HUDI in continuous mode? On Mon, Sep 16, 2019 at 9:09 PM Vinoth Chandar wrote: > Thanks, Taher! Any takers for driving this? This is something I would be > very interested

Re: [BUG] Null Pointer Exception in SourceFormatAdapter

2019-09-16 Thread Vinoth Chandar
Actually went ahead and created https://issues.apache.org/jira/browse/HUDI-253 . Question is just about the PR for this now ? :) On Mon, Sep 16, 2019 at 8:54 AM Vinoth Chandar wrote: > +1 DeltaStreamer can be much nicer in such cases.. Any interest in opening > a JIRA/PR for this? > > On Mon,

Re: [BUG] Null Pointer Exception in SourceFormatAdapter

2019-09-16 Thread Vinoth Chandar
+1 DeltaStreamer can be much nicer in such cases.. Any interest in opening a JIRA/PR for this? On Mon, Sep 16, 2019 at 2:02 AM vbal...@apache.org wrote: > Yes, It makes sense to add validations with descriptive messages. Please > open a ticket and send a PR for this. > Thanks,Balaji.VOn

Re: [DISCUSS] [VOTE] JDBC incremental load with DeltaStreamer

2019-09-16 Thread Vinoth Chandar
Thanks, Taher! Any takers for driving this? This is something I would be very interested in getting involved with. Dont have the bandwidth atm :/ On Sun, Sep 15, 2019 at 11:15 PM Taher Koitawala wrote: > Thank you all for your support. JIRA filed at >

Re: [BUG] Exception when running HoodieDeltaStreamer

2019-09-16 Thread Pratyaksh Sharma
It works. Thank you. On Fri, Sep 13, 2019 at 7:28 PM vbal...@apache.org wrote: > > Hi Pratyaksh, > For boolean flags, you don't need to pass true or false. It is implicit. > Just pass "--enable-hive-sync" without additional true/false in the command > line. > Balaji.VOn Friday, September

Re: Field not found in record HoodieException

2019-09-16 Thread Taher Koitawala
Hi Kabeer, hive table has everything as a string. However when fetching data, the spark query is .sql(String.format("select contact_id,country,cast(last_update as TIMESTAMP) as last_update from %s",hiveTable)) On Mon, Sep 16, 2019 at 6:18 PM Kabeer Ahmed wrote: > Is last_update a timestamp? Can

Re: Field not found in record HoodieException

2019-09-16 Thread Kabeer Ahmed
Is last_update a timestamp? Can you please throw the hive schema that you are using to create table. You could run show create table and send us the output please? On Sep 16 2019, at 1:32 pm, Taher Koitawala wrote: > Hi Kaber, Same issue when last_update is converted to long. > >

Re: Field not found in record HoodieException

2019-09-16 Thread Taher Koitawala
Hi Kaber, Same issue when last_update is converted to long. HoodieSparkSQLWriter: Registered avro schema : { "type" : "record", "name" : "s3_master_contacts_list_hudi_record", "namespace" : "hoodie.s3_master_contacts_list_hudi", "fields" : [ { "name" : "contact_id", "type" : [

Re: Field not found in record HoodieException

2019-09-16 Thread Kabeer Ahmed
Taher, This error of field not found exception with HUDI is mostly because of 2 cases: The data types of the fields do not match with the types listed in hive tables. The field may really not be preset - which doesnt seem to be your case. I looked into the schema in your log which is below.

Re: [BUG] Null Pointer Exception in SourceFormatAdapter

2019-09-16 Thread vbal...@apache.org
Yes, It makes sense to add validations with descriptive messages. Please open a ticket and send a PR for this. Thanks,Balaji.VOn Monday, September 16, 2019, 01:11:12 AM PDT, Pratyaksh Sharma wrote: Hi Balaji, I get your point. However I feel in such cases, instead of throwing a Null

Re: [BUG] Null Pointer Exception in SourceFormatAdapter

2019-09-16 Thread Pratyaksh Sharma
Hi Balaji, I get your point. However I feel in such cases, instead of throwing a Null Pointer, we should handle the case gracefully. The exception should be thrown with proper user-facing message. Please let me know your thoughts on this. On Fri, Sep 13, 2019 at 7:26 PM Balaji Varadarajan

Re: [DISCUSS] [VOTE] JDBC incremental load with DeltaStreamer

2019-09-16 Thread Taher Koitawala
Thank you all for your support. JIRA filed at https://issues.apache.org/jira/browse/HUDI-251 Regards, Taher Koitawala On Mon, Sep 16, 2019 at 11:34 AM Taher Koitawala wrote: > Since everyone is fully onboard. I am creating a JIRA to track this. > > On Sun, Sep 15, 2019 at 9:47 AM

Re: [DISCUSS] [VOTE] JDBC incremental load with DeltaStreamer

2019-09-16 Thread Taher Koitawala
Since everyone is fully onboard. I am creating a JIRA to track this. On Sun, Sep 15, 2019 at 9:47 AM vbal...@apache.org wrote: > > +1. Agree with everyone's point. Go for it Taher !! > Balaji.VOn Saturday, September 14, 2019, 07:44:04 PM PDT, Bhavani > Sudha Saktheeswaran wrote: > > +1 I