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 specific semantics we don’t support in the V2 API. For example, 
one cannot commit multiple write operations in a single transaction right now. 
That would require changes to the DDL and a pretty substantial change to the 
design of Spark-SQL more broadly.

 

-Matt Cheah

 

From: Shiv Prashant Sood <shivprash...@gmail.com>
Date: Friday, August 2, 2019 at 12:56 PM
To: Spark Dev List <dev@spark.apache.org>
Subject: DataSourceV2 : Transactional Write support

 

All,

 

I understood that DataSourceV2 supports Transactional write and wanted to  
implement that in JDBC DataSource V2 connector ( PR#25211 [github.com] ). 

 

Don't see how this is feasible for JDBC based connector.  The FW suggest that 
EXECUTOR send a commit message  to DRIVER, and actual commit should only be 
done by DRIVER after receiving all commit confirmations. This will not work for 
JDBC  as commits have to happen on the JDBC Connection which is maintained by 
the EXECUTORS and JDBCConnection  is not serializable that it can be sent to 
the DRIVER. 

 

Am i right in thinking that this cannot be supported for JDBC? My goal is to 
either fully write or roll back the dataframe write operation.

 

Thanks in advance for your help.

 

Regards, 

Shiv

Attachment: smime.p7s
Description: S/MIME cryptographic signature

Reply via email to