Re: Can we use Hudi inside AWS?

2019-03-04 Thread Kabeer Ahmed
Hi Umesh, We use it on AWS. So it definitely works on AWS (S3). I have read that there is support for Presto too but we havent used Presto due to compatibility issues between Apache Ranger and Presto. Thanks, Kabeer. On Mar 4 2019, at 10:21 am, Umesh Kacha wrote: > Hi is there out of the box s

Timestamp

2019-03-14 Thread Kabeer Ahmed
handled this and has an example that can be shared, it will be much appreciated. Kabeer Ahmed, http://www.linkedin.com/in/kabeerahmed

Re: Timestamp

2019-03-23 Thread Kabeer Ahmed
hema/data generation that we can start with? > Thanks > Vinoth > > On Fri, Mar 15, 2019 at 11:19 AM Vinoth Chandar wrote: > > Hi Kabeer, > > Thanks for bringing this up. I don't think we have actually hit this > > before :) > > > > Let me spend sometime und

S3 Connections | https://github.com/apache/incubator-hudi/issues/613

2019-03-23 Thread Kabeer Ahmed
Hi, I have just raised this issue and thought to share with the community if someone else is experiencing this. Apologies in advance if this is a redundant email. Thanks Kabeer.

Re: S3 Connections | https://github.com/apache/incubator-hudi/issues/613

2019-03-24 Thread Kabeer Ahmed
ed reducing cleaner parallelism if limiting connections is your goal? > > Also some good news is, once > https://issues.apache.org/jira/browse/HUDI-1 is landed (currently being > reviewed), a follow on is to rework the cleaner incrementally on top which > should help a lot here. &g

Re: S3 Connections | https://github.com/apache/incubator-hudi/issues/613

2019-03-25 Thread Kabeer Ahmed
ira id, so I can add you as > acontributor, giving you commenting etc on jira > > Thanks > Vinoth > > > > On Sun, Mar 24, 2019 at 2:11 PM Kabeer Ahmed wrote: > > Hi Vinoth, > > Thank you for your response. I thought of reducing clear parallelism which > &g

Re: S3 Connections | https://github.com/apache/incubator-hudi/issues/613

2019-03-25 Thread Kabeer Ahmed
elps > us immensely already! :) We typically test with HDFS which just talks to > namenode for RPC. > > For context, are you running hudi in a streaming job that does not exit? > Thanks > Vinoth > > On Mon, Mar 25, 2019 at 2:58 PM Kabeer Ahmed wrote: > > Thank you V

Re: Timestamp

2019-03-26 Thread Kabeer Ahmed
2:06 am, Vinoth Chandar wrote: > Will give this a shot as well. Between this and the S3 thing, what's > blocking progress? both? ;) ? > > On Sat, Mar 23, 2019 at 7:10 PM Kabeer Ahmed wrote: > > Hi Vinoth, > > Thank you for looking into this. I am planning to try out th

Re: Hudi with duplicate key

2019-04-04 Thread Kabeer Ahmed
Omkar - there might be various reasons to have duplicates eg: handle trades in a given day from a single client, track visitor click data to the website etc. Rahul - If you can give more details about your requirements, then we can come up with a solution. I have never used INSERT & BULK_INSERT

Re: Hudi with duplicate key

2019-04-05 Thread Kabeer Ahmed
inserting. > Thanks & Regards > Rahul On Apr 5 2019, at 9:11 am, Unknown wrote: > > > On 2019/04/04 19:48:39, Kabeer Ahmed wrote: > > Omkar - there might be various reasons to have duplicates eg: handle trades > > in a given day from a single client, track visitor c

Re: [IMP] Understanding present state and planning ahead

2019-04-06 Thread Kabeer Ahmed
+1. Just submitted my input to this form. On Apr 5 2019, at 5:46 pm, Vinoth Chandar wrote: > So, please take 2 minutes and send in your responses. > > On Fri, Apr 5, 2019 at 9:45 AM Vinoth Chandar wrote: > > +1 Folks, it would be really helpful for us to manage the roadmap and also > > understand

Re: S3 Connections | https://github.com/apache/incubator-hudi/issues/613

2019-04-07 Thread Kabeer Ahmed
3 issue. Can you > share how you are monitoring the number of connections to S3? (This helps > me first repro the issue) > > Thanks > Vinoth > > > > > > > On Mon, Mar 25, 2019 at 6:01 PM Kabeer Ahmed wrote: > > Hi Vinoth, > > *Context* > >

Re: Hudi with duplicate key

2019-04-09 Thread Kabeer Ahmed
nfigurations.html#combineInput . > > > > > > Do any of these apply to you.. > > > Thanks > > > Vinoth > > > > > > > > > > > > On Fri, Apr 5, 2019 at 9:10 AM Kabeer Ahmed wrote: > > > > Hi Rahul, > > >

Hive Sync in HUDI

2019-04-10 Thread Kabeer Ahmed
All, Is there an API to do hive sync? The use case is for a pipeline that does update Hive after all the plumbing. I do see that there is a HiveSyncTool() functionality available. But the issue I have with that is with the hardcoded username and password to access Hive. I have currently circumv

Re: Upgrade HUDI to Hive 2.x

2019-05-19 Thread Kabeer Ahmed
Hi, I think it is OK to deprecate the Hive 1.1 support. As of 0.4.6-SNAPSHOT that I was using the latest build about 3 weeks ago, I did face issues if I did want to work with Hive 1.1 that is bundled as a part of CDH 5.13 docker image. I did have to make manual tweaks listed at: https://github

Re: Read RO table in Spark as hive table | No records returned

2019-05-19 Thread Kabeer Ahmed
Hi Vinod, On the backdrop of your email, are there any examples where HiveSyncTool has been used programmatically to drive creation and management of the Hive Table? I have a need to create HiveTable and manage it programmatically. Any example would help. Or others who can chip in here and say t

Re: Upgrade HUDI to Hive 2.x

2019-05-19 Thread Kabeer Ahmed
notice that older > version of Hudi (pre-0.4.6) worked fine with CDH 5.13 ? > Balaji.V > > > > > > > > > On Sunday, May 19, 2019, 11:45:24 AM PDT, Kabeer Ahmed > wrote: > Hi, > I think it is OK to deprecate the Hive 1.1 support. As of 0.4.6-SNAPSHOT that > I

Out of Heap Error when inserting into Hudi dataset

2019-07-02 Thread Kabeer Ahmed
Hi Vinoth and other HUDI Experts, I am stuck while processing inserts into HUDI. The process picks up CSV files and loads them into HUDI. The process seems to be stuck at: https://github.com/apache/incubator-hudi/blob/master/hoodie-client/src/main/java/com/uber/hoodie/table/HoodieCopyOnWriteTabl

Re: Out of Heap Error when inserting into Hudi dataset

2019-07-03 Thread Kabeer Ahmed
at and paste > the results (that you can share). > > Thanks, > Nishith > > On Tue, Jul 2, 2019 at 4:53 PM Kabeer Ahmed wrote: > > Hi Vinoth and other HUDI Experts, > > I am stuck while processing inserts into HUDI. The process picks up CSV > > files and loads

Re: Out of Heap Error when inserting into Hudi dataset

2019-07-03 Thread Kabeer Ahmed
h","type":{"type":"string","avro.java.string":"String"}},{"name":"policy","type":{"type":"string","avro.java.string":"String"}},{"name":"deletePathPatterns","typ

Re: Out of Heap Error when inserting into Hudi dataset

2019-07-03 Thread Kabeer Ahmed
":"partitionMetadata","type":{"type":"map","values":{"type":"record","name":"HoodieCleanPartitionMetadata","fields":[{"name":"partitionPath","type":{"type":"strin

Re: Out of Heap Error when inserting into Hudi dataset

2019-07-03 Thread Kabeer Ahmed
ng the newest instant > with non-zero records written and then using it for average record > calculation. Let us know if you are interested in working on the fix. > Balaji.V > On Wednesday, July 3, 2019, 8:25:33 AM PDT, Kabeer Ahmed > wrote: > > Hi Nishith and All, > I think I

Re: Out of Heap Error when inserting into Hudi dataset

2019-07-11 Thread Kabeer Ahmed
t; We can write a simple loop that looks for the first non-zero size commit or > fallback to default configs. > > On Wed, Jul 3, 2019 at 8:35 AM Kabeer Ahmed wrote: > > Hi Balaji, > > My confidence isnt great when it comes to edit the code to find the newest > > non

Re: [VOTE] Proposal to clone default JIRA workflow for Hudi project

2019-07-23 Thread Kabeer Ahmed
+1. On Jul 23 2019, at 11:36 pm, Vinoth Chandar wrote: > Thanks for the feedback! > > In case my description was misleading, only change we want to do at this > point is : letting non assignees also move a task from "open" to say "in > progress". > This makes board maintainability easier IMHO. >

Spark 2.4 and Timestamp Type

2019-07-23 Thread Kabeer Ahmed
Hello All, After scanning a few PRs I think the relevant Spark 2.4 changes are in the pull request: https://github.com/apache/incubator-hudi/pull/638 (https://link.getmailspring.com/link/1c80cc45-323f-44f2-80ea-28b9fcf3f...@getmailspring.com/0?redirect=https%3A%2F%2Fgithub.com%2Fapache%2Fincubat

Re: [QUESTION] May I ask if the Hudi contributor JIRA group can receive the notification email.

2019-08-08 Thread Kabeer Ahmed
+1 On Aug 7 2019, at 10:24 pm, Vinoth Chandar wrote: > Hi Luciano, > > > > please consider having a notifications list to avoid flooding the mailing > list as depending on the workflow > > https://issues.apache.org/jira/projects/HUDI/summary should already give us > a feed like you are mentioning

Upsert after Delete

2019-08-22 Thread Kabeer Ahmed
Hudi experts and Users, Has anyone attempted an upsert after a delete? Here is a weird thing that I have bumped into and it is a shame that this has come up when someone in the team tested this whilst I failed to run this test. Use case: Insert data into a table. Say records (1, kabeer | 2, vino

Upsert after Delete

2019-08-22 Thread Kabeer Ahmed
lete Date: Aug 22 2019, at 8:38 pm To: dev@hudi.apache.org That’s interesting. Can you also share details on storage type and how you are issuing the deletes and also the table/view (ro, rt) that you are querying? On Thu, Aug 22, 2019 at 9:49 AM Kabeer Ahmed wrote: > Hudi experts and Users, >

Re: Upsert after Delete

2019-08-22 Thread Kabeer Ahmed
And if you meant HUDI storage type, I have left it to default COW - Copy On Write. If anyone has tried this please let me know if you have hit similar issue. Any experience would be greatly helpful. On Aug 22 2019, at 11:01 pm, Kabeer Ahmed wrote: > Hi Vinoth - thanks for the quick respo

Re: Upsert after Delete

2019-08-28 Thread Kabeer Ahmed
this, we can > debug from there. > > > > > > > On Thu, Aug 22, 2019 at 3:06 PM Kabeer Ahmed wrote: > > And if you meant HUDI storage type, I have left it to default COW - Copy > > On Write. > > > > If anyone has tried this please let me know if you ha

Re: Upsert after Delete

2019-08-30 Thread Kabeer Ahmed
er, > > > I have requested some information in the github ticket. > > > Balaji.V On Wednesday, August 28, 2019, 10:46:04 AM PDT, Kabeer Ahmed > > > > < > > > kab...@linuxmail.org> wrote: > > > > > > Thanks for the quick response Vinoth.

Re: Upsert after Delete

2019-08-30 Thread Kabeer Ahmed
> Hi Kabeer, > > I have requested some information in the github ticket. > > Balaji.V On Wednesday, August 28, 2019, 10:46:04 AM PDT, Kabeer Ahmed < > > kab...@linuxmail.org> wrote: > > > > Thanks for the quick response Vinoth. That is what I would have though

Re: Upsert after Delete

2019-09-02 Thread Kabeer Ahmed
en last step is deleting data. Query after delete was > > failing because HUDI was not able to read schema from last record as it was > > written by empty payload. I am not sure whether it's fixed or not now. > > Maybe Vinoth can confirm it. > > > > Thanks, >

Re: Dropping support for Spark 2.2 and lower

2019-09-10 Thread Kabeer Ahmed
+1. I am on spark 2.3 but would love to move to Spark 2.4. On Sep 10 2019, at 12:16 am, Vinoth Chandar wrote: > Hello all, > > I am trying to gauge what spark version everyone is on. We would like to > move the spark version to 2.4 and simplify a whole bunch of stuff. Any > objections? As a best

Re: Using hudi with pyspark

2019-09-10 Thread Kabeer Ahmed
Hi Rodrigo, Welcome to the HUDI users group. The entire Hudi code base is Java and Scala based. But there is nothing stopping you from using it through Python (pyspark). You should be able to copy all the packaging jars into your Spark installation and use them. But please note that you wouldnt

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. Basi

Re: Field not found in record HoodieException

2019-09-16 Thread Kabeer Ahmed
"contact_id", > "type" : [ "string", "null" ] > }, { > "name" : "country", > "type" : [ "string", "null" ] > }, { > "name" : "last_update", > "type" : [ "lo

Re: Field not found in record HoodieException

2019-09-16 Thread Kabeer Ahmed
d,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 you please throw the hive schema that you > > are using to create table. You could run show create

Re: Field not found in record HoodieException

2019-09-17 Thread Kabeer Ahmed
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, 201

Re: Field not found in record HoodieException

2019-09-18 Thread Kabeer Ahmed
s thread! > > > > On Tue, Sep 17, 2019 at 8:04 PM Taher Koitawala > > wrote: > > > > > No there are no nulls in the data and I am getting the same error. > > > On Wed, Sep 18, 2019, 3:33 AM Kabeer Ahmed wrote: > > > > Taher - did you find an

Re: Field not found in record HoodieException

2019-09-24 Thread Kabeer Ahmed
, at 5:13 pm, Taher Koitawala wrote: > Hi Kabeer, > Really appreciate the help. Take your time nothing urgent. > > Regards, > Taher Koitawala > > On Wed, Sep 18, 2019, 9:38 PM Kabeer Ahmed wrote: > > Taher, > > I have a half baked code for test. I shall complete it an

Re: How to deploy Hudi

2019-10-02 Thread Kabeer Ahmed
Qian Welcome! Are you able to tell us a bit more about your use case? Eg: type of the project, industry, complexity of the pipeline that you plan to write (eg: pulling data from external APIs like New York taxi dataset and writing them into Hive for analysis) etc. This will give us a bit more c

Re: Questions about using Hudi

2019-10-11 Thread Kabeer Ahmed
Hi Qian, If there are no nulls in the data, then most likey it is issue with the data types being stored. I have seen this issue again and again and in the recent one it was due to me storing double value when I had actually declared the schema as IntegerType. I can reproduce this with an examp

Re: Questions about using Hudi

2019-10-11 Thread Kabeer Ahmed
gt; > > Best, > > > Qian > > > On Oct 11, 2019, 1:57 PM -0700, Qian Wang , wrote: > > > > Hi Kabeer, > > > > > > > > Thanks for your detailed explanation. I will try it again. Will update > > you the result. > > > > >

Re: Parquet issue

2019-10-15 Thread Kabeer Ahmed
Shahida Welcome to Hudi. I am not an expert with DeltaStreamer as I do not use it. In general, I think this points to the issue with build of the fat jar. This looks to me that either you didnt build the fat jar to include all the dependencies or your class path didnt include the jar needed. Fo

Re: Parquet issue

2019-10-15 Thread Kabeer Ahmed
n after > including in build. > > Strange is, I have even provided the parquet-avro jar via spark-submit, and > it behave differently for 1.7 and 1.8 > Seems like there is some configuration missing with respect to > HoodieStorageConfig.PARQUET_COMPRESSION_CODEC. > > >

Re: Parquet issue

2019-10-15 Thread Kabeer Ahmed
*Looks like this * > > * 1.8.1* > > > > > > > > > > > > *org.apache.parquetparquet-avro${parquet.version}org.apache.parquetparquet-hadoop > ${parquet.version} * > > > Regards, > *Shahida R. Khan* > > > On Tue, Oct 15, 2019 at 7:12 PM Kabeer Ahmed wrote: >

Re: Parquet issue

2019-10-17 Thread Kabeer Ahmed
xml#L67 > > > > > > Heads up : you may hit issues with Hive since CDH hive is still 1.x (world > > is moving to Hive 3+ slowly, all other cloud/distro vendors are on Hive > > 2.x). > > > > On Tue, Oct 15, 2019 at 8:33 AM Kabeer Ahmed > (mailto:kab...@lin

Re: Parquet issue

2019-10-17 Thread Kabeer Ahmed
%2Fe189734a07b8782ea1d21b3c780dfc61c2ab8f2b&recipient=ZGV2QGh1ZGkuYXBhY2hlLm9yZw%3D%3D) Email below just for context of my earlier discussion on this thread. On Sunday, May 19, 2019, 11:45:24 AM PDT, Kabeer Ahmed https://link.getmailspring.com/link/0fcca6a8-0063-4c51-b50d-00e52fa9f...@getmailspring.com/1?redi

Re: Inline storage of parquet data in logs

2019-10-23 Thread Kabeer Ahmed
Hi Vinoth, Have crazy week and the next 2 to 3 weeks are going to be very busy. I havent had a chance to look into this. My thoughts are around security. The ideas of building external indexes come with loads of advantages and throwing user data into the logs etc makes me anxious. Let me do a d

Re: Inline storage of parquet data in logs

2019-10-23 Thread Kabeer Ahmed
t. Please correct me if I am missing something. > > > > Thanks, > > Jaimin > > > > On Wednesday, 23 October 2019, Vinoth Chandar wrote: > > > Sure. Take your time! Just to clarify, here log refers to the Hudi > > append > > > log, not user's lo

Re: new committer: vinoyang/Hua Yang

2019-11-02 Thread Kabeer Ahmed
Hua Yang and Shaofeng Li - Congratulations on your achievement! On Nov 2 2019, at 8:36 pm, Vinoth Chandar wrote: > Hello all, > > The Podling Project Management Committee (PPMC) for Apache Hudi > (Incubating) has invited Hua Yang to become a committer and we are pleased > to announce that he has

Re: [Nov 5, 2019] Weekly Sync meeting

2019-11-06 Thread Kabeer Ahmed
Sudha, Balaji, and All, Thank you so much for this initiative. 9pm PST is 5am GMT and quite inconvenient for people in Europe. I think a fair way would be to poll based on existing users and then decide on a time. Getting a convenient time for people in West Coast to far east is always challeng

Re: [Discuss] Convenient time for weekly sync meeting

2019-11-07 Thread Kabeer Ahmed
Dear Sudha Really appreciate the initiative to promptly start this thread. My preferences are as below: Any weekday: 10PM PST to 11PM PST OR 10AM PST TO 2PM PST thank you On Nov 7 2019, at 6:46 am, Pratyaksh Sharma wrote: > Interested. > > Timings: > Mon-Fri 6AM-7.30AM PST > > On Thu, Nov 7,

Re: New Committer : bhavanisudha

2019-11-08 Thread Kabeer Ahmed
Congratulations Sudha! On Nov 8 2019, at 3:47 am, Y. Ethan Guo wrote: > Congrats, Sudha! > > On Thu, Nov 7, 2019 at 7:45 PM Taher Koitawala wrote: > > Congratulations Bhavani Sudha! Well deserved > > On Fri, Nov 8, 2019, 9:10 AM Vinoth Chandar wrote: > > > Congrats sudha! > > > On Thu, Nov

Re: [Discuss] Convenient time for weekly sync meeting

2019-11-09 Thread Kabeer Ahmed
d half. > - There is also a proposal about adding another meeting for covering the > US/EU times. We can try that, but am not able to find a overlapping 1 hour > slot that would cover the people who cannot attend the first meeting. > > > Thanks, > Sudha > > > On Thu, Nov

Re: Spark v2.3.2 : Duplicate entries found for each primary Key

2019-11-15 Thread Kabeer Ahmed
Adding to Vinoth's response, in spark-shell you just need to copy and paste the below line. Let us know if it still doesnt work. spark.sparkContext.hadoopConfiguration.setClass("mapreduce.input.pathFilter.class", classOf[org.apache.hudi.hadoop.HoodieROTablePathFilter], classOf[org.apache.hadoop

Re: Spark v2.3.2 : Duplicate entries found for each primary Key

2019-11-19 Thread Kabeer Ahmed
eer. It resolved my problem. > > > > > > Regards, > > > Purushotham Pushpavanth > > > > > > > > > > > > On Fri, 15 Nov 2019 at 20:16, Kabeer Ahmed > wrote: > > > > > > > Adding to Vinoth's response, in spark-she

Re: Spark v2.3.2 : Duplicate entries found for each primary Key

2019-11-20 Thread Kabeer Ahmed
Sharma wrote: > Thank you for the explanation Kabeer/Sudha. > > Let me go through the flow and revert back in case of any further queries. > On Wed, Nov 20, 2019 at 6:21 AM Kabeer Ahmed wrote: > > Pratyaksh, > > +1 to what Sudha has written. Lets zoom a bit closer. >

Re: [Discuss] Convenient time for weekly sync meeting

2019-11-24 Thread Kabeer Ahmed
> > On Sun, Nov 10, 2019 at 5:41 PM Vinoth Chandar > > > wrote: > > > > > > > > > @kabeer I can additionally join a bi-weekly/monthly call that works > > for > > > > > Europe and other time zones. Weekly would be hard. Any of the

Re: [DISCUSS] Scaling community support

2019-12-09 Thread Kabeer Ahmed
Hi Vinoth Regarding [1], I recommend that people who cannot triage or find a owner also chip in. i.e. make this open ended so that anyone from community could answer it. But if the thread reaches a dead end or doesnt progress, then such threads should be picked up by someone more experienced eg

Re: [QUESTION] Encountering exceptions while upserting with Deltastreamer

2019-12-19 Thread Kabeer Ahmed
Hi Ethan, It is often tricky to debug or help with issues when I do not have an idea of the data. My "guess" is that your schema is changing. This could be related to: https://stackoverflow.com/a/42946528/4517001 (https://link.getmailspring.com/link/966cdc69-f6ce-4745-88bd-0e5553efa...@getmails

Re: [QUESTION] Encountering exceptions while upserting with Deltastreamer

2019-12-19 Thread Kabeer Ahmed
t; > > > On Thu, Dec 19, 2019 at 1:51 PM nishith agarwal > > > wrote: > > > > > > > Ethan, > > > > Unless this is a backwards incompatible schema change, this seems > > > > related to a parquet-avro reader bug we've seen before, find more >

Re: [DISCUSS] SQL Support using Apache Calcite

2020-12-15 Thread Kabeer Ahmed
Vinoth and All, Users on gmail might be missing out on these emails as Gmail is down and emails sent to gmail.com domain are bouncing back. At 11pm UK time below is the google update: https://www.google.com/appsstatus#hl=en&v=issue&sid=1&iid=a8b67908fadee664c68c240ff9f529ab Best to bump this thre