Re: DataSourceV2 : Transactional Write support

2019-08-05 Thread Reynold Xin
Monday, August 5, 2019 at 7:40 PM > *To: *Shiv Prashant Sood > *Cc: *Ryan Blue , Jungtaek Lim , > Spark Dev List > *Subject: *Re: DataSourceV2 : Transactional Write support > > > > I agree with the temp table approach. One idea is: maybe we only need one > temp table, an

Re: DataSourceV2 : Transactional Write support

2019-08-05 Thread Matt Cheah
There might be some help from the staging table catalog as well. -Matt Cheah From: Wenchen Fan Date: Monday, August 5, 2019 at 7:40 PM To: Shiv Prashant Sood Cc: Ryan Blue , Jungtaek Lim , Spark Dev List Subject: Re: DataSourceV2 : Transactional Write support I agree with the temp

Re: DataSourceV2 : Transactional Write support

2019-08-05 Thread Wenchen Fan
I agree with the temp table approach. One idea is: maybe we only need one temp table, and each task writes to this temp table. At the end we read the data from the temp table and write it to the target table. AFAIK JDBC can handle concurrent table writing very well, and it's better than creating th

Re: DataSourceV2 : Transactional Write support

2019-08-05 Thread Shiv Prashant Sood
Thanks all for the clarification. Regards, Shiv On Sat, Aug 3, 2019 at 12:49 PM Ryan Blue wrote: > > What you could try instead is intermediate output: inserting into > temporal table in executors, and move inserted records to the final table > in driver (must be atomic) > > I think that this i

Re: DataSourceV2 : Transactional Write support

2019-08-03 Thread Ryan Blue
> What you could try instead is intermediate output: inserting into temporal table in executors, and move inserted records to the final table in driver (must be atomic) I think that this is the approach that other systems (maybe sqoop?) have taken. Insert into independent temporary tables, which c

Re: DataSourceV2 : Transactional Write support

2019-08-02 Thread Jungtaek Lim
I asked similar question for end-to-end exactly-once with Kafka, and you're correct distributed transaction is not supported. Introducing distributed transaction like "two-phase commit" requires huge change on Spark codebase and the feedback was not positive. What you could try instead is intermed

Re: DataSourceV2 : Transactional Write support

2019-08-02 Thread Matt Cheah
Can we check that the latest staging APIs work for the JDBC use case in a single transactional write? See https://github.com/apache/spark/pull/24798/files#diff-c9d2f9c9d20452939b7c28ebdae0503dR53 But also acknowledge that transactions from a more traditional RDBMS sense tend to have pretty s