Uploaded image for project: 'Derby'
  1. Derby
  2. DERBY-6896

XA Transaction not rolled back when client disconnects without finalizing the transaction

    XMLWordPrintableJSON

Details

    • Bug
    • Status: Open
    • Major
    • Resolution: Unresolved
    • 10.10.2.0
    • None
    • JDBC
    • None
    • Network server being access by a Network client
    • Normal
    • Deviation from standard, Regression, Seen in production

    Description

      Originally in version 54764 of XATransactionState.java (introduced in DERBY-2432), the “isFinished” attribute was set to “false" in the constructor and then then set to “true” in the “xa_finalize” method. So the effect of this was that when the “cancel” method was called, no matter from where, the XA transaction was rolled back. In DRDAXAProtocol.java, the method “rollbackTransaction” looked like

          void rollbackCurrentTransaction()
          {
              if (xid != null) {
                  boolean local  = ( xid.getFormatId() == -1);
                  try {
                      // if the transaction is not local disassociate the transaction from
                      // the connection first because the rollback can not be performed
                      // on a transaction associated with the XAResource
                      try {
                          if (!local) {
                              XAResource xaResource = getXAResource();
                              // this will throw the XAException (because TMFAIL
                              // will throw an exception)
                              xaResource.end(xid, XAResource.TMFAIL);
                          }
                      } catch (XAException e) {
                          // do not print out the exception generally thrown
                          // when TMFAIL flag is present
                          if (e.errorCode < XAException.XA_RBBASE
                              || e.errorCode > XAException.XA_RBEND) {
                              connThread.getServer().consoleExceptionPrint(e);
                          }
                      }
                      rollbackTransaction(xid, false);
                  } catch  (DRDAProtocolException e) {
                      // because we do not dump any DRDA stuff to the socket
                      // the exception can not be thrown in this case
                      // However, we will dump the exception to the console
                      connThread.getServer().consoleExceptionPrint(e);
                  }
                  xid = null;
              }
          }
      

      DERBY-2871 changed this to look like:

          void rollbackCurrentTransaction()
          {
              if (xid != null) {
                  boolean local  = ( xid.getFormatId() == -1);
                  if (!local) {
                      try {
                          XAXactId xid_im = new XAXactId(xid);
                          getResourceAdapter().cancelXATransaction(
                              xid_im,
                              MessageId.CONN_CLOSE_XA_TRANSACTION_ROLLED_BACK
                          );
                      } catch (XAException e) {
                          Monitor.logThrowable(e);
                      }
                  } else {
                      try {
                          rollbackTransaction(xid, false);
                      } catch  (DRDAProtocolException e) {
                          // because we do not dump any DRDA stuff to the socket
                          // the exception can not be thrown in this case
                          // However, we will log the exception to the monitor
                          Monitor.logThrowable(e);
                      }
                  }
                  xid = null;
              }
          }
      

      So you can see that now “rollbackTransaction” is only being called if the transaction is local and “ResourceAdapter.cancelXATransaction” is being called when the transaction is not local (a XA transaction). So now looking at what XATransactionState.cancel” which is ultimately what is called by “ResourceAdapter.cancelXATransaction”, we find that “isFinished” has been replaced by “performTimeoutRollback” and now the “cancel” looks like:

         synchronized void cancel(String messageId) throws XAException {
              // Check performTimeoutRollback just to be sure that
              // the cancellation task was not started
              // just before the xa_commit/rollback
              // obtained this object's monitor.
              if (performTimeoutRollback) {
      
                  // Log the message about the transaction cancelled
                  if (messageId != null)
                      Monitor.logTextMessage(messageId, xid.toString());
      
                  // Check whether the transaction is associated
                  // with any EmbedXAResource instance.
                  if (associationState == XATransactionState.T1_ASSOCIATED) {
                      conn.cancelRunningStatement();
                      EmbedXAResource assocRes = associatedResource;
                      end(assocRes, XAResource.TMFAIL, true);
                  }
      
                  // Rollback the global transaction
                  try {
                      conn.xa_rollback();
                  } catch (SQLException sqle) {
                      XAException ex = new XAException(XAException.XAER_RMERR);
                      ex.initCause(sqle);
                      throw ex;
                  }
      
                  // Do the cleanup on the resource
                  creatingResource.returnConnectionToResource(this, xid);
              }
          }
      

      and “performTimeoutRollback” is only set in

          synchronized void scheduleTimeoutTask(long timeoutMillis) {
              // Mark the transaction to be rolled back bby timeout
              performTimeoutRollback = true;
              // schedule a time out task if the timeout was specified
              if (timeoutMillis > 0) {
                  // take care of the transaction timeout
                  TimerTask cancelTask = new CancelXATransactionTask();
                  TimerFactory timerFactory = Monitor.getMonitor().getTimerFactory();
                  Timer timer = timerFactory.getCancellationTimer();
                  timer.schedule(cancelTask, timeoutMillis);
              } else {
                  timeoutTask = null;
              }
          }
      

      so the logic when from processing “cancel” as long as the XA transaction was not completed to processing “cancel” only when invoked from a timeout. At the same time the "DRDAXAProtocol.rollbackCurrentTransaction" went from always rolling back the transaction by calling “rollbackTransaction” to it calling “XATransctionState.cancel” (ultimately) which will do nothing from this code path as “performTimeoutRollback” will never be true.

      I understand what the check is “cancel” was trying to do, it was trying to prevent “cancel” to be processed when invoked by the timer right after it had been processed and completed by normal XA transaction work. Unfortunately what it now does is not correct. For example, I just wrote a simple test program that looks like:

          private static void startTransactionButDontComplete(XAConnection connXa, Connection conn, String schema, String table) throws SQLException, XAException {
              XAResource resXa = connXa.getXAResource();
              Xid xid = new MyXid(100, new byte[]{0x01}, new byte[]{0x02})       ;
              
              resXa.start(xid, XAResource.TMNOFLAGS);
              Statement stmt = conn.createStatement();
              stmt.executeUpdate("DELETE FROM " + schema + "." + table);
              resXa.end(xid, XAResource.TMSUCCESS);
              System.exit(1);
          }
      {no format}
      What this does is to start an XA transaction, execute an update, end the XA transaction, but kill the database client before the XA transaction is either committed or rollback.  With the code before DERBY-2871, when the the database engine detected the client disconnect, the transaction would be rolled back.  Now what happens is that there is a XA transaction that will never complete (commit or rollback) that is left in the database.  So this is what I see in the database after the application exits:
      
      

      Bretts-MacBook-Pro:~ brett$ /Applications/db-derby-10.9.1.0-bin/bin/ij
      ij version 10.9
      ij> connect 'jdbc:derby://192.169.1.31:1527/csemdb';
      ij> select * from syscs_diag.transaction_table where status != 'IDLE';
      XID |GLOBAL_XID |USERNAME |TYPE |STATUS |FIRST_INSTANT |SQL_TEXT
      -------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------
      1420140985 |(100,01,02) |CSEM |UserTransaction |ACTIVE |(113348,692000) |NULL

      1 row selected
      ij>

      {no format}

      Doing some searching it does not seem that this is the correct behavior:

      http://stackoverflow.com/questions/30608378/what-happens-to-my-jdbc-transaction-if-i-lose-my-session
      http://dba.stackexchange.com/questions/60001/what-happens-to-a-transaction-if-the-network-connection-fails

      Attachments

        Activity

          People

            Unassigned Unassigned
            bbergquist Brett Bergquist
            Votes:
            0 Vote for this issue
            Watchers:
            1 Start watching this issue

            Dates

              Created:
              Updated: