Uploaded image for project: 'Debezium'
  1. Debezium
  2. DBZ-960

ConnectException: Only REPEATABLE READ isolation level is supported for START TRANSACTION WITH CONSISTENT SNAPSHOT in RocksDB Storage Engine

    XMLWordPrintable

Details

    • Bug
    • Resolution: Done
    • Major
    • 0.9.0.Beta2
    • 0.7.2
    • mysql-connector
    • None

    Description

      dbz version: 0.7.2

      kafka version: 1.0.0

      mysql version: 5.6.0


      In my product env, we use ali cloud rds;

      Alibaba cloud rds provided rocksdb as storage engine. We have used the mysql which support rocks db as storage engine;

      When a new dbz mysql task created, we gotten an exception

      "org.apache.kafka.connect.errors.ConnectException: Only REPEATABLE READ isolation level is supported for START TRANSACTION WITH CONSISTENT SNAPSHOT in RocksDB Storage Engine. Error code: 3045; SQLSTATE: HY000.\n\t
      at io.debezium.connector.mysql.AbstractReader.wrap(AbstractReader.java:178)\n\t
      at io.debezium.connector.mysql.AbstractReader.failed(AbstractReader.java:156)\n\t
      at io.debezium.connector.mysql.SnapshotReader.execute(SnapshotReader.java:668)\n\t
      at java.lang.Thread.run(Thread.java:748)\n
      Caused by: java.sql.SQLException: Only REPEATABLE READ isolation level is supported for START TRANSACTION WITH CONSISTENT SNAPSHOT in RocksDB Storage Engine.\n\t
      at com.mysql.jdbc.SQLError.createSQLException(SQLError.java:964)\n\t
      at com.mysql.jdbc.MysqlIO.checkErrorPacket(MysqlIO.java:3970)\n\t
      at com.mysql.jdbc.MysqlIO.checkErrorPacket(MysqlIO.java:3906)\n\t
      at com.mysql.jdbc.MysqlIO.sendCommand(MysqlIO.java:2524)\n\t
      at com.mysql.jdbc.MysqlIO.sqlQueryDirect(MysqlIO.java:2677)\n\t
      at com.mysql.jdbc.ConnectionImpl.execSQL(ConnectionImpl.java:2545)\n\t
      at com.mysql.jdbc.ConnectionImpl.execSQL(ConnectionImpl.java:2503)\n\t
      at com.mysql.jdbc.StatementImpl.executeInternal(StatementImpl.java:839)\n\t
      at com.mysql.jdbc.StatementImpl.execute(StatementImpl.java:739)\n\t
      at io.debezium.jdbc.JdbcConnection.lambda$execute$2(JdbcConnection.java:294)\n\t
      at io.debezium.jdbc.JdbcConnection.execute(JdbcConnection.java:311)\n\t
      at io.debezium.jdbc.JdbcConnection.execute(JdbcConnection.java:288)\n\t
      at io.debezium.connector.mysql.SnapshotReader.execute(SnapshotReader.java:285)\n\t... 1 more\n",
      

      According to the exception, ' START TRANSACTION WITH CONSISTENT SNAPSHOT' is not supported by rocksdb engine;

      In order to test that, we tried to execute sql in ali cloud console;
      We tries to execute below sqls firstly:
      1. set autocommit = false;
      2. SET TRANSACTION ISOLATION LEVEL READ COMMITTED
      3. START TRANSACTION WITH CONSISTENT SNAPSHOT
      4. commit

      Results:
      When step 3 executing, exception was throwed out; ('Only REPEATABLE READ isolation level is supported for START TRANSACTION WITH CONSISTENT SNAPSHOT in RocksDB Storage Engine.')

      Then we execute these sqls:
      1. set autocommit = false;
      2. SET TRANSACTION ISOLATION LEVEL REPEATABLE READ
      3. START TRANSACTION WITH CONSISTENT SNAPSHOT
      4. commit

      Results:
      All the sqls were executed correctly;


      The default transaction isolation of our mysql is 'rc';

      In dbz SnapshotReader.java 'SET TRANSACTION ISOLATION LEVEL REPEATABLE READ' will be executed firstly and then 'START TRANSACTION WITH CONSISTENT SNAPSHOT' will be executed;

      Because of the default transaction isolation that is 'rc', task throws out the exception; But from the code, we have changed the transaction isolation to 'rr'; It seems the change is not work when executing "START TRANSACTION WITH CONSISTENT SNAPSHOT";

      Now I think the transaction when executing two sql is not same;


      In SnapshotReader.java, sql will be executed at JdbcConnection.java;

          public JdbcConnection execute(String... sqlStatements) throws SQLException {
              return execute(statement -> {
                  for (String sqlStatement : sqlStatements) {
                      if (sqlStatement != null) {
                          if (LOGGER.isTraceEnabled()) {
                              LOGGER.trace("executing '{}'", sqlStatement);
                          }
                          statement.execute(sqlStatement);
                      }
                  }
              });
          }
      
          public JdbcConnection execute(Operations operations) throws SQLException {
              Connection conn = connection();
              try (Statement statement = conn.createStatement();) {
                  operations.apply(statement);
                  if (!conn.getAutoCommit()) conn.commit();
              }
              return this;
          }
      

      if (!conn.getAutoCommit()) conn.commit();

      According to the code, conn.commit() will be called when any sql executed;

      I am very strange about this code; When any sql executed, commit will be executed too. So 'SET TRANSACTION ISOLATION LEVEL REPEATABLE READ' will be not worked when 'START TRANSACTION WITH CONSISTENT SNAPSHOT' executed;

      Attachments

        1. 1.PNG
          1.PNG
          67 kB
        2. 2.PNG
          2.PNG
          6 kB
        3. 捕获.PNG
          捕获.PNG
          6 kB

        Activity

          People

            jpechane Jiri Pechanec
            sweatott wenze hu (Inactive)
            Votes:
            0 Vote for this issue
            Watchers:
            3 Start watching this issue

            Dates

              Created:
              Updated:
              Resolved: