clickhouse-java icon indicating copy to clipboard operation
clickhouse-java copied to clipboard

Transaction support

Open zhicwu opened this issue 3 years ago • 1 comments

Server

  • Transaction ID: Tuple(snapshot_version UInt64, local_txid_counter UInt64, host_id UUID)
  • functions: transactionID(), transactionLatestSnapshot(), transactionOldestSnapshot()
  • settings: throw_on_unsupported_query_inside_transaction, wait_changes_become_visible_after_commit_mode, implicit_transaction
  • statements: begin transaction, commit, rollback, set transaction snapshot <snapshot id>, kill transaction where tid=<transaction id>

Java Client

  • [x] new client option transaction_timeout, defaults to session_timeout
  • [x] add ClickHouseTransaction with methods for begin/commit/rollback transaction and snapshot
  • [x] add ClickHouseRequestManager for generating query id and session id as well as transaction creation
  • [x] enhance ClickHouseRequest by adding methods getTransaction and setTransaction - multiple requests can take part in one transaction
  • [x] throw ClickHouseTransactionException for Code: 649. DB::Exception: Cannot execute query because current transaction failed. Expecting ROLLBACK statement. (INVALID_TRANSACTION) - server will rollback automatically but still needs client to explicitly to acknowledge that by issue rollback(using connection.rollback())
  • [x] always create new session when begin transaction and enforce session_check until transaction completed
  • [x] consider transaction in retry and failover(avoid node switching)

JDBC Driver

  • [x] new JDBC option transactionSupport (defaults to false)
  • [x] use implicit_transaction in auto commit mode, fallback to TCL when it's not supported
  • [x] replace FakeTransaction when transaction support is enabled
  • [x] change default transaction isolation level to repeatable-read(why?)
  • [ ] parse TCL to maintain transaction consistency between server and client in 0.3.3?
  • [ ] update SIT base classes in 0.3.3 to use docker-compose instead of standalone container and then enable tests for HA and TX in CI

zhicwu avatar Jul 01 '22 00:07 zhicwu

Usage(as of 0.3.2-patch11):

  • Java Client

    ClickHouseNode server = ClickHouseNode.of("http://localhost:48011/system?transactionSupport");
    try (ClickHouseClient client = ClickHouseClient.newInstance(server.getProtocol())) {
        // implicit transaction
        try (ClickHouseResponse response = client.connect(server)
            .format(ClickHouseFormat.RowBinaryWithNamesAndTypes)
            .query("select 1")
            // blocking call to create and start a new transaction, issue query, and then commit/rollback automatically
            .executeWithinTransaction()) {
            ...
        }
    
        ClickHouseRequest<?> reqWithManualTx = client.connect(server)
            .format(ClickHouseFormat.RowBinaryWithNamesAndTypes)
            .transaction() // create a new transaction and call its begin() method right now
            .query("select 1");
        try (ClickHouseResponse response = reqWithManualTx.executeAndWait()) {
            ...
        }
        reqWithManualTx.getTransaction().commit(); // commit the transaction right away
    }
    
  • JDBC Driver

    // auto-commit
    try (Connection conn = DriverManager.getConnection("jdbc:ch:http://localhost:48011/system?transactionSupport");
        Statement stmt = conn.createStatement()) {
        stmt.executeQuery("select 1");
        ...
    }
    
    // manual
    try (Connection conn = DriverManager.getConnection("jdbc:ch:http://localhost:48011/system?!autoCommit&transactionSupport");
        Statement stmt = conn.createStatement()) {
        stmt.executeQuery("select 1");
        conn.commit();
    }
    

Performance penalty for transaction support is acceptable(on client side):

image

zhicwu avatar Jul 27 '22 01:07 zhicwu

Hello, can you provide an example of transactional data insertion using PreparedStatement about JDBC Driver?

hantaoliulinux avatar Mar 11 '23 09:03 hantaoliulinux

Hello, can you provide an example of transactional data insertion using PreparedStatement about JDBC Driver?

Hi @hantaoliulinux, did you run into any issue using above examples? The usage is same for Statement and PreparedStatement. FYI, transaction won't start in 0.4.0 and 0.4.1, please use v0.3.2-patch11(or wait for 0.4.2) for transaction support.

zhicwu avatar Mar 13 '23 00:03 zhicwu