You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@trafodion.apache.org by se...@apache.org on 2016/10/21 19:41:22 UTC

incubator-trafodion git commit: [TRAFODION-2296] Consistent error reporting in abort, commit transaction. Changes to wait for all RS to respond when there is an exception in abort and commit transaction requests.

Repository: incubator-trafodion
Updated Branches:
  refs/heads/master 7092997e2 -> ee1cd85df


[TRAFODION-2296] Consistent error reporting in abort, commit transaction.
Changes to wait for all RS to respond when there is an exception in
abort and commit transaction requests.

When there is an error returned from commit or rollback transaction, the details of the error can be obtained in the following log files

a) $MY_SQROOT/logs/tm_.log in the node that issued this request contains the error message as seen by the TM process in the JNI side. This event may not have transaction id.
b) $MY_SQROOT/logs/trafodion.dtm.log contains more info about this error with the transaction id. These events are logged from java side of TM.
c) In the region server logs of all the regions that participated in the transaction.

These exceptions are visible as error code on the client side. To get the details about the exception, the above logs need to be browsed.

Fixes to avoid tm core upon double delete of incoming messages to it

This closes #768


Project: http://git-wip-us.apache.org/repos/asf/incubator-trafodion/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-trafodion/commit/ee1cd85d
Tree: http://git-wip-us.apache.org/repos/asf/incubator-trafodion/tree/ee1cd85d
Diff: http://git-wip-us.apache.org/repos/asf/incubator-trafodion/diff/ee1cd85d

Branch: refs/heads/master
Commit: ee1cd85dfa711dbeee083ac6205fa0dee76e1b91
Parents: 7092997
Author: selvaganesang <se...@esgyn.com>
Authored: Mon Oct 17 05:10:33 2016 +0000
Committer: selvaganesang <se...@apache.org>
Committed: Fri Oct 21 19:37:13 2016 +0000

----------------------------------------------------------------------
 core/sqf/conf/log4cxx.trafodion.tm.config       |   2 +-
 .../transactional/TransactionManager.java       | 607 ++++++++++---------
 .../client/transactional/TransactionState.java  |  79 ++-
 .../java/org/trafodion/dtm/TmAuditTlog.java     |  11 +-
 core/sqf/src/tm/tmmsg.h                         |  21 +-
 core/sqf/src/tm/tmtx.cpp                        |   4 +-
 core/sql/cli/CliExtern.cpp                      |   1 +
 core/sql/qmscommon/QRLogger.cpp                 |   3 +
 8 files changed, 415 insertions(+), 313 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-trafodion/blob/ee1cd85d/core/sqf/conf/log4cxx.trafodion.tm.config
----------------------------------------------------------------------
diff --git a/core/sqf/conf/log4cxx.trafodion.tm.config b/core/sqf/conf/log4cxx.trafodion.tm.config
index acfe6b8..214e155 100644
--- a/core/sqf/conf/log4cxx.trafodion.tm.config
+++ b/core/sqf/conf/log4cxx.trafodion.tm.config
@@ -45,5 +45,5 @@ log4j.appender.tmAppender.layout.ConversionPattern=%d, %p, %c, %m%n
 log4j.appender.tmAppender.Append=true
 
 #TM
-log4j.logger.TM=ERROR, tmAppender
+log4j.logger.TM=INFO, tmAppender
 

http://git-wip-us.apache.org/repos/asf/incubator-trafodion/blob/ee1cd85d/core/sqf/src/seatrans/hbase-trx/src/main/java/org/apache/hadoop/hbase/client/transactional/TransactionManager.java
----------------------------------------------------------------------
diff --git a/core/sqf/src/seatrans/hbase-trx/src/main/java/org/apache/hadoop/hbase/client/transactional/TransactionManager.java b/core/sqf/src/seatrans/hbase-trx/src/main/java/org/apache/hadoop/hbase/client/transactional/TransactionManager.java
index 863f21c..a6b5055 100644
--- a/core/sqf/src/seatrans/hbase-trx/src/main/java/org/apache/hadoop/hbase/client/transactional/TransactionManager.java
+++ b/core/sqf/src/seatrans/hbase-trx/src/main/java/org/apache/hadoop/hbase/client/transactional/TransactionManager.java
@@ -288,6 +288,8 @@ public class TransactionManager {
 
         if( TRANSACTION_ALGORITHM == AlgorithmType.MVCC){
         do {
+          retry = false;
+          refresh = false;
           try {
 
             if (LOG.isDebugEnabled()) LOG.debug("doCommitX -- ENTRY txid: " + transactionId
@@ -319,41 +321,40 @@ public class TransactionManager {
                         table.toString() + " startKey: " + new String(startKey, "UTF-8") + " endKey: " + new String(endKey, "UTF-8"));
                  result = table.coprocessorService(TrxRegionService.class, startKey, endKey, callable);
                } catch (ServiceException se) {
-                  String msg = new String ("ERROR occurred while calling doCommitX coprocessor service in doCommitX for transaction: "
-                              + transactionId + " participantNum " + participantNum );
-                  LOG.error(msg, se);
+                  String msg = new String ("ERROR occurred while calling coprocessor service in doCommitX for transaction "
+                              + transactionId + " participantNum " + participantNum + " Location " + location.getRegionInfo().getRegionNameAsString());
+                  LOG.warn(msg, se);
                   throw new RetryTransactionException(msg,se);
                } catch (Throwable e) {
-                  String msg = new String ("ERROR occurred while calling doCommitX coprocessor service in doCommitX for transaction: "
-                              + transactionId + " participantNum " + participantNum );
+                  String msg = new String ("ERROR occurred while calling coprocessor service in doCommitX for transaction "
+                              + transactionId + " participantNum " + participantNum + " Location " + location.getRegionInfo().getRegionNameAsString());
                   LOG.error(msg, e);
-                  transactionState.requestPendingCountDec(true);
-                  throw new DoNotRetryIOException(msg,e);
+                  DoNotRetryIOException dnre =  new DoNotRetryIOException(msg,e);
+                  transactionState.requestPendingCountDec(dnre);
+                  throw dnre;
                }
-               if(result.size() == 0) {
-                  if(LOG.isTraceEnabled()) LOG.trace("doCommitX,received incorrect result size: " + result.size() + " txid: "
-                        + transactionId + " location: " + location.getRegionInfo().getRegionNameAsString());
-                  
+               if (result.size() == 0) {
+                  LOG.error("doCommitX, received incorrect result size: " + result.size() + " txid: "
+                       + transactionId + " participantNum " + participantNum + " Location " + location.getRegionInfo().getRegionNameAsString());
                   refresh = true;
                   retry = true;
                   //if transaction for DDL operation, it is possible this table is disabled
                   //as part of prepare if the table was intended for a drop. If this is the case
                   //this exception can be ignored.
                   if(transactionState.hasDDLTx())
-		  {
+                  {
                      if(LOG.isTraceEnabled()) LOG.trace("doCommitX, checking against DDL Drop list:  result size: " +
                          result.size() + " txid: " + transactionId + " location: " + location.getRegionInfo().getRegionNameAsString() + 
                          "table: " + table.getName().getNameAsString());
-                     
                      ArrayList<String> createList = new ArrayList<String>(); //This list is ignored.
            	     ArrayList<String> dropList = new ArrayList<String>();
-		     ArrayList<String> truncateList = new ArrayList<String>();
-  		     StringBuilder state = new StringBuilder ();
+                     ArrayList<String> truncateList = new ArrayList<String>();
+                     StringBuilder state = new StringBuilder ();
                      tmDDL.getRow(transactionState.getTransactionId(), state, createList, dropList, truncateList);
                      if(state.toString().equals("VALID") && dropList.size() > 0)
                      {
                        Iterator<String> di = dropList.iterator();
-		       while (di.hasNext())
+                       while (di.hasNext())
                        {
                          if(table.getName().getNameAsString().equals(di.next().toString()))
                          {
@@ -365,55 +366,27 @@ public class TransactionManager {
                          }
                        }
                      }
-		  }
-		  else
-		  {
+                  }
+                  else
+                  {
                   	LOG.error("doCommitX, received incorrect result size: " + result.size() + " txid: "
-                       	+ transactionId + " location: " + location.getRegionInfo().getRegionNameAsString());
-		  }
-               }
-               else if(result.size() == 1){
-                  // size is 1
-                  for (CommitResponse cresponse : result.values()){
-                    if(cresponse.getHasException()) {
-                      String exceptionString = new String (cresponse.getException());
-                      LOG.error("doCommitX - exceptionString: " + exceptionString);
-                      if (exceptionString.contains("UnknownTransactionException")) {
-                        if (ignoreUnknownTransaction == true) {
-                          if (LOG.isTraceEnabled()) LOG.trace("doCommitX, ignoring UnknownTransactionException in cresponse");
-                        }
-                        else {
-                          LOG.error("doCommitX, coprocessor UnknownTransactionException: " + cresponse.getException());
-                          throw new UnknownTransactionException(cresponse.getException());
-                        }
-                      }
-                      else {
-                        if (LOG.isTraceEnabled()) LOG.trace("doCommitX coprocessor exception: " + cresponse.getException());
-                        throw new RetryTransactionException(cresponse.getException());
-                      }
+                       	+ transactionId + " participantNum " + participantNum + " location: " + location.getRegionInfo().getRegionNameAsString());
                   }
                }
-               retry = false;
-             }
-             else {
-                  for (CommitResponse cresponse : result.values()){
-                    if(cresponse.getHasException()) {
+               else {
+                  for (CommitResponse cresponse : result.values()) {
+                    if (cresponse.getHasException()) {
                       String exceptionString = new String (cresponse.getException());
-                      LOG.error("doCommitX - exceptionString: " + exceptionString);
                       if (exceptionString.contains("UnknownTransactionException")) {
-                        if (ignoreUnknownTransaction == true) {
-                          if (LOG.isTraceEnabled()) LOG.trace("doCommitX, ignoring UnknownTransactionException in cresponse");
-                        }
-                        else {
-                          LOG.error("doCommitX, coprocessor UnknownTransactionException: " + cresponse.getException());
                           throw new UnknownTransactionException(cresponse.getException());
-                        }
                       }
-                      else if(exceptionString.contains("Asked to commit a non-pending transaction")) {
-                        if (LOG.isTraceEnabled()) LOG.trace("doCommitX, ignoring 'commit non-pending transaction' in cresponse");
+                      else if (exceptionString.contains("DUPLICATE")) {
+                         throw new UnknownTransactionException(cresponse.getException());
+                      }
+                      else if (exceptionString.contains("org.apache.hadoop.hbase.exceptions.FailedSanityCheckException")) {
+                         throw new org.apache.hadoop.hbase.exceptions.FailedSanityCheckException(cresponse.getException());
                       }
                       else {
-                        if (LOG.isTraceEnabled()) LOG.trace("doCommitX coprocessor exception: " + cresponse.getException());
                         throw new RetryTransactionException(cresponse.getException());
                       }
                   }
@@ -423,22 +396,39 @@ public class TransactionManager {
 
           }
           catch (UnknownTransactionException ute) {
-             LOG.error("Got unknown exception in doCommitX by participant " + participantNum
-                       + " for transaction: " + transactionId, ute);
-             transactionState.requestPendingCountDec(true);
-             throw ute;
+             String errMsg = new String("doCommitX UnknownTransactionException for transaction "
+                              + transactionId + " participantNum " + participantNum + " Location " + location.getRegionInfo().getRegionNameAsString());
+             if (ignoreUnknownTransaction) {
+                if (LOG.isInfoEnabled()) LOG.info(errMsg + " ,but ignored", ute);
+                transactionState.requestPendingCountDec(null);
+             }
+             else {
+                LOG.error(errMsg, ute);
+                transactionState.logUteDetails();
+                transactionState.requestPendingCountDec(null);
+             }
+          }
+          catch (org.apache.hadoop.hbase.exceptions.FailedSanityCheckException fsce) {
+              LOG.error("doCommitX FailedSanityCheckException for transaction " + transactionId + " participantNum " + participantNum + 
+                 " Location " + location.getRegionInfo().getRegionNameAsString(), fsce);
+              refresh = false;
+              retry = false;
+              transactionState.requestPendingCountDec(fsce);
+              throw fsce;
           }
           catch (RetryTransactionException rte) {
              if(retryCount == RETRY_ATTEMPTS) {
-                LOG.error("Exceeded retry attempts (" + retryCount + ") in doCommitX for transaction: " + transactionId);
+                String errMsg;
+                errMsg = new String("Exceeded " + retryCount + " retry attempts in doCommitX for transaction "
+                        + transactionId + " participantNum " + participantNum + " Location " + location.getRegionInfo().getRegionNameAsString());
                 // We have received our reply in the form of an exception,
                 // so decrement outstanding count and wake up waiters to avoid
                 // getting hung forever
-                transactionState.requestPendingCountDec(true);
-                throw new CommitUnsuccessfulException("Exceeded retry attempts (" + retryCount + ") in doCommitX for transaction: " +
-                             transactionId, rte);
+                CommitUnsuccessfulException cue = new CommitUnsuccessfulException(errMsg, rte);
+                transactionState.requestPendingCountDec(cue);
+                throw cue;
              }
-             LOG.error("doCommitX retrying transaction: " + transactionId + " due to Exception: ", rte);
+             LOG.error("doCommitX retrying transaction " + transactionId + " due to Exception: ", rte);
              refresh = true;
              retry = true;
           }
@@ -448,7 +438,7 @@ public class TransactionManager {
              HRegionInfo     lv_hri = lv_hrl.getRegionInfo();
 
              if (LOG.isTraceEnabled()) LOG.trace("doCommitX -- location being refreshed : " + location.getRegionInfo().getRegionNameAsString() + " endKey: "
-                     + Hex.encodeHexString(location.getRegionInfo().getEndKey()) + " for transaction: " + transactionId);
+                     + Hex.encodeHexString(location.getRegionInfo().getEndKey()) + " for transaction " + transactionId);
              if (LOG.isWarnEnabled()) LOG.warn("doCommitX -- " + table.toString() + " location being refreshed");
              if (LOG.isWarnEnabled()) LOG.warn("doCommitX -- lv_hri: " + lv_hri);
              if (LOG.isWarnEnabled()) LOG.warn("doCommitX -- location.getRegionInfo(): " + location.getRegionInfo());
@@ -463,6 +453,8 @@ public class TransactionManager {
 
         if( TRANSACTION_ALGORITHM == AlgorithmType.SSCC){
         do {
+          retry = false;
+          refresh = false;
           try {
 
             if (LOG.isTraceEnabled()) LOG.trace("doCommitX -- ENTRY txid: " + transactionId +
@@ -493,17 +485,21 @@ public class TransactionManager {
                         table.toString() + " startKey: " + new String(startKey, "UTF-8") + " endKey: " + new String(endKey, "UTF-8"));
                  result = table.coprocessorService(SsccRegionService.class, startKey, endKey, callable);
                } catch (ServiceException se) {
-                  String msg = "ERROR occurred while calling doCommitX coprocessor service in doCommitX";
-                  LOG.error(msg + ":", se);
+                  String msg = new String("ERROR occurred while calling coprocessor service in doCommitX for transaction " 
+                     + transactionId + " participantNum " + participantNum + " Location " + location.getRegionInfo().getRegionNameAsString());
+                  LOG.warn(msg + ":", se);
                   throw new RetryTransactionException(msg,se);
                } catch (Throwable e) {
-                  String msg = "ERROR occurred while calling doCommitX coprocessor service in doCommitX";
+                  String msg = new String("ERROR occurred while calling coprocessor service in doCommitX for transaction " 
+                     + transactionId + " participantNum " + participantNum + " Location " + location.getRegionInfo().getRegionNameAsString());
                   LOG.error(msg + ":", e);
-                  transactionState.requestPendingCountDec(true);
-                  throw new DoNotRetryIOException(msg, e);
+                  DoNotRetryIOException dnr = new DoNotRetryIOException(msg, e);
+                  transactionState.requestPendingCountDec(dnr);
+                  throw dnr;
                }
-               if(result.size() != 1) {
-                  LOG.error("doCommitX, received incorrect result size: " + result.size() + " txid: " + transactionId);
+               if (result.size() != 1) {
+                  LOG.error("doCommitX, received incorrect result size: " + result.size() + " in doCommitX for transaction "
+                     + transactionId + " participantNum " + participantNum + " Location " + location.getRegionInfo().getRegionNameAsString());
                   refresh = true;
                   retry = true;
                }
@@ -513,16 +509,9 @@ public class TransactionManager {
                     if(cresponse.getHasException()) {
                       String exceptionString = new String (cresponse.getException());
                       if (exceptionString.contains("UnknownTransactionException")) {
-                        if (ignoreUnknownTransaction == true) {
-                          if (LOG.isTraceEnabled()) LOG.trace("doCommitX, ignoring UnknownTransactionException in cresponse");
-                        }
-                        else {
-                          LOG.error("doCommitX, coprocessor UnknownTransactionException: " + cresponse.getException());
                           throw new UnknownTransactionException(cresponse.getException());
-                        }
                       }
                       else {
-                        if (LOG.isTraceEnabled()) LOG.trace("doCommitX coprocessor exception: " + cresponse.getException());
                         throw new RetryTransactionException(cresponse.getException());
                       }
                   }
@@ -531,21 +520,28 @@ public class TransactionManager {
              }
           }
           catch (UnknownTransactionException ute) {
-              String errMsg = new String("Got unknown exception in doCommitX by participant " + participantNum
-              		  + " for transaction: " + transactionId);
-              LOG.error(errMsg, ute);
-              transactionState.requestPendingCountDec(true);
-              throw ute;
+             String errMsg = new String("doCommitX, UnknownTransactionException  for transaction " 
+                + transactionId + " participantNum " + participantNum + " Location " + location.getRegionInfo().getRegionNameAsString());
+             if (ignoreUnknownTransaction) {
+                LOG.info(errMsg + " ,but ignored", ute);
+                transactionState.requestPendingCountDec(null);
+             }
+             else {
+                LOG.error(errMsg, ute);
+                transactionState.logUteDetails();
+                transactionState.requestPendingCountDec(null);
+             }
           }
           catch (RetryTransactionException rte) {
              if (retryCount == RETRY_ATTEMPTS) {
-                LOG.error("Exceeded retry attempts (" + retryCount + ") in doCommitX for transaction: " + transactionId, rte);
+                String errMsg = new String("Exceeded " + retryCount + " retry attempts in doCommitX for transaction "
+                        + transactionId + " participantNum " + participantNum + " Location " + location.getRegionInfo().getRegionNameAsString());
                 // We have received our reply in the form of an exception,
                 // so decrement outstanding count and wake up waiters to avoid
                 // getting hung forever
-                transactionState.requestPendingCountDec(true);
-                throw new CommitUnsuccessfulException("Exceeded retry attempts (" + retryCount + ") in doCommitX for transaction: " + transactionId,
-                            rte);
+                CommitUnsuccessfulException cue = new CommitUnsuccessfulException(errMsg, rte);
+                transactionState.requestPendingCountDec(cue);
+                throw cue;
              }
 
              LOG.error("doCommitX participant " + participantNum + " retrying transaction "
@@ -559,7 +555,7 @@ public class TransactionManager {
              HRegionInfo     lv_hri = lv_hrl.getRegionInfo();
 
              if (LOG.isTraceEnabled()) LOG.trace("doCommitX -- location being refreshed : " + location.getRegionInfo().getRegionNameAsString() + "endKey: "
-                     + Hex.encodeHexString(location.getRegionInfo().getEndKey()) + " for transaction: " + transactionId);
+                     + Hex.encodeHexString(location.getRegionInfo().getEndKey()) + " for transaction " + transactionId);
                 if (LOG.isWarnEnabled()) LOG.warn("doCommitX -- " + table.toString() + " location being refreshed");
                 if (LOG.isWarnEnabled()) LOG.warn("doCommitX -- lv_hri: " + lv_hri);
                 if (LOG.isWarnEnabled()) LOG.warn("doCommitX -- location.getRegionInfo(): " + location.getRegionInfo());
@@ -573,7 +569,7 @@ public class TransactionManager {
 
         }
         // We have received our reply so decrement outstanding count
-        transactionState.requestPendingCountDec(false);
+        transactionState.requestPendingCountDec(null);
 
         if (LOG.isTraceEnabled()) LOG.trace("doCommitX -- EXIT txid: " + transactionId);
         return 0;
@@ -600,6 +596,8 @@ public class TransactionManager {
 
        if( TRANSACTION_ALGORITHM == AlgorithmType.MVCC){
        do {
+          retry = false;
+          refresh = false;
           try {
              Batch.Call<TrxRegionService, CommitRequestResponse> callable =
                 new Batch.Call<TrxRegionService, CommitRequestResponse>() {
@@ -626,18 +624,21 @@ public class TransactionManager {
              try {
                 result = table.coprocessorService(TrxRegionService.class, startKey, endKey, callable);
              } catch (ServiceException se) {
-                String errMsg  = new String("doPrepareX coprocessor error for " + Bytes.toString(regionName) + " txid: " + transactionId + ":");
-                LOG.error(errMsg, se);
-                throw new RetryTransactionException("Unable to call prepare, coprocessor error", se);
+                String errMsg = new String("ERROR occurred while calling coprocessor service in doPrepareX for transaction " 
+                     + transactionId + " participantNum " + participantNum + " Location " + location.getRegionInfo().getRegionNameAsString());
+                LOG.warn(errMsg, se);
+                throw new RetryTransactionException(errMsg, se);
              } catch (Throwable e) {
-                String errMsg  = new String("doPrepareX coprocessor error for " + Bytes.toString(regionName) + " txid: " + transactionId + ":");
+                String errMsg = new String("ERROR occurred while calling coprocessor service in doPrepareX for transaction " 
+                     + transactionId + " participantNum " + participantNum + " Location " + location.getRegionInfo().getRegionNameAsString());
                 LOG.error(errMsg, e);
-                transactionState.requestPendingCountDec(true);
-                throw new CommitUnsuccessfulException("Unable to call prepare, coprocessor error", e);
+                CommitUnsuccessfulException cue =  new CommitUnsuccessfulException(errMsg, e);
+                throw cue;
              }
 
              if(result.size() == 0)  {
-                LOG.error("doPrepareX, received incorrect result size: " + result.size());
+                LOG.error("doPrepareX(MVCC), received incorrect result size: " + result.size() + " for transaction "
+                     + transactionId + " participantNum " + participantNum + " Location " + location.getRegionInfo().getRegionNameAsString());
                 refresh = true;
                 retry = true;
              }
@@ -714,15 +715,14 @@ public class TransactionManager {
              }
           }
           catch(RetryTransactionException rte) {
-             if (retryCount == RETRY_ATTEMPTS){
-                LOG.error("Exceeded retry attempts in doPrepareX: " + retryCount, rte);
-                // We have received our reply in the form of an exception,
-                // so decrement outstanding count and wake up waiters to avoid
-                // getting hung forever
-                transactionState.requestPendingCountDec(true);
-                throw new CommitUnsuccessfulException("Exceeded retry attempts in doPrepareX: " + retryCount, rte);
+             if (retryCount == RETRY_ATTEMPTS) {
+                String errMsg = new String("Exceeded retry attempts in doPrepareX for transaction " 
+                     + transactionId + " participantNum " + participantNum + " Location " + location.getRegionInfo().getRegionNameAsString());
+                LOG.error(errMsg, rte);
+                CommitUnsuccessfulException cue = new CommitUnsuccessfulException(errMsg, rte);
+                throw cue;
              }
-             LOG.error("doPrepareX participant " + participantNum + " retrying transaction "
+             LOG.warn("doPrepareX participant " + participantNum + " retrying transaction "
                           + transactionId + " due to Exception: " , rte);
              refresh = true;
              retry = true;
@@ -733,7 +733,7 @@ public class TransactionManager {
              HRegionInfo     lv_hri = lv_hrl.getRegionInfo();
 
              if (LOG.isTraceEnabled()) LOG.trace("doPrepareX -- location being refreshed : " + location.getRegionInfo().getRegionNameAsString() + "endKey: "
-                     + Hex.encodeHexString(location.getRegionInfo().getEndKey()) + " for transaction: " + transactionId);
+                     + Hex.encodeHexString(location.getRegionInfo().getEndKey()) + " for transaction " + transactionId);
                 if (LOG.isWarnEnabled()) LOG.warn("doPrepareX -- " + table.toString() + " location being refreshed");
                 if (LOG.isWarnEnabled()) LOG.warn("doPrepareX -- lv_hri: " + lv_hri);
                 if (LOG.isWarnEnabled()) LOG.warn("doPrepareX -- location.getRegionInfo(): " + location.getRegionInfo());
@@ -752,6 +752,8 @@ public class TransactionManager {
        }
        if( TRANSACTION_ALGORITHM == AlgorithmType.SSCC){
        do {
+          retry = false;
+          refresh = false;
           try {
              Batch.Call<SsccRegionService, SsccCommitRequestResponse> callable =
                 new Batch.Call<SsccRegionService, SsccCommitRequestResponse>() {
@@ -774,11 +776,18 @@ public class TransactionManager {
 
              try {
                 result = table.coprocessorService(SsccRegionService.class, startKey, endKey, callable);
+             } catch (ServiceException se) {
+                String errMsg = new String("ERROR occurred while calling coprocessor service in doPrepareX for transaction " 
+                     + transactionId + " participantNum " + participantNum + " Location " + location.getRegionInfo().getRegionNameAsString());
+                LOG.warn(errMsg, se);
+                throw new RetryTransactionException("Unable to call prepare, coprocessor error", se);
              } catch (Throwable e) {
-                LOG.error("doPrepareX coprocessor error for " + Bytes.toString(regionName) + " txid: " + transactionId, e);
-                throw new CommitUnsuccessfulException("Unable to call prepare, coprocessor error", e);
+                String errMsg = new String("ERROR occurred while calling coprocessor service in doPrepareX for transaction " 
+                     + transactionId + " participantNum " + participantNum + " Location " + location.getRegionInfo().getRegionNameAsString());
+                LOG.error(errMsg, e);
+                CommitUnsuccessfulException cue =  new CommitUnsuccessfulException(errMsg, e);
+                throw cue;
              }
-
              if(result.size() != 1)  {
                 LOG.error("doPrepareX, received incorrect result size: " + result.size());
                 refresh = true;
@@ -798,17 +807,16 @@ public class TransactionManager {
                 retry = false;
              }
           }
-          catch (RetryTransactionException e) {
+          catch (RetryTransactionException rte) {
              if (retryCount == RETRY_ATTEMPTS) {
-                LOG.error("Exceeded retry attempts in doPrepareX: " + retryCount, e);
-                // We have received our reply in the form of an exception,
-                // so decrement outstanding count and wake up waiters to avoid
-                // getting hung forever
-                transactionState.requestPendingCountDec(true);
-                throw new CommitUnsuccessfulException("Exceeded retry attempts in doPrepareX: " + retryCount, e);
+                String errMsg = new String("Exceeded " + retryCount + " retry attempts in doPrepareX for transaction " 
+                     + transactionId + " participantNum " + participantNum + " Location " + location.getRegionInfo().getRegionNameAsString());
+                LOG.error(errMsg, rte);
+                CommitUnsuccessfulException cue =  new CommitUnsuccessfulException(errMsg, rte);
+                throw cue;
              }
-             LOG.error("doPrepareX participant " + participantNum + " retrying transaction "
-                      + transactionId + " due to Exception: ", e);
+             LOG.warn("doPrepareX participant " + participantNum + " retrying transaction "
+                      + transactionId + " due to Exception: ", rte);
              refresh = true;
              retry = true;
           }
@@ -818,7 +826,7 @@ public class TransactionManager {
              HRegionInfo     lv_hri = lv_hrl.getRegionInfo();
 
              if (LOG.isTraceEnabled()) LOG.trace("doPrepareX -- location being refreshed : " + location.getRegionInfo().getRegionNameAsString() + "endKey: "
-                     + Hex.encodeHexString(location.getRegionInfo().getEndKey()) + " for transaction: " + transactionId);
+                     + Hex.encodeHexString(location.getRegionInfo().getEndKey()) + " for transaction " + transactionId);
                 if (LOG.isWarnEnabled()) LOG.warn("doPrepareX -- " + table.toString() + " location being refreshed");
                 if (LOG.isWarnEnabled()) LOG.warn("doPrepareX -- lv_hri: " + lv_hri);
                 if (LOG.isWarnEnabled()) LOG.warn("doPrepareX -- location.getRegionInfo(): " + location.getRegionInfo());
@@ -836,6 +844,8 @@ public class TransactionManager {
        if (LOG.isTraceEnabled()) LOG.trace("commitStatus for transId(" + transactionId + "): " + commitStatus
                                                                        + " TableName " + table.toString()
                                                                        + " Region Name " + Bytes.toString(regionName));
+       String errMsg = new String("Received invalid return code " + commitStatus + " from commit request in doPrepareX for transaction " 
+                     + transactionId + " participantNum " + participantNum + " Location " + location.getRegionInfo().getRegionNameAsString());
        boolean canCommit = true;
        boolean readOnly = false;
 
@@ -848,15 +858,16 @@ public class TransactionManager {
             readOnly = true;
             break;
           case TransactionalReturn.COMMIT_UNSUCCESSFUL_FROM_COPROCESSOR:
-            if (LOG.isTraceEnabled()) LOG.trace("Received COMMIT_UNSUCCESSFUL_FROM_COPROCESSOR return code from requestCommit " + commitStatus + " for transaction " + transactionId);
           case TransactionalReturn.COMMIT_CONFLICT:
           case TransactionalReturn.COMMIT_UNSUCCESSFUL:
+             LOG.error(errMsg);
              canCommit = false;
              transactionState.addRegionToIgnore(location); // No need to re-abort.
              break;
           default:
-             LOG.warn("Received invalid return code from requestCommit " + commitStatus + " for transaction " + transactionId + " throwing CommitUnsuccessfulException");
-             throw new CommitUnsuccessfulException("Unexpected return code from prepareCommit: " + commitStatus);
+             CommitUnsuccessfulException cue = new CommitUnsuccessfulException(errMsg);
+             LOG.error(errMsg, cue);
+             throw cue;
        }
 
        if (!canCommit) {
@@ -885,7 +896,8 @@ public class TransactionManager {
      * Return  : Ignored
      * Purpose : Call abort for a given regionserver
      */
-    public Integer doAbortX(final byte[] regionName, final long transactionId, final int participantNum, final boolean dropTableRecorded) throws IOException{
+    public Integer doAbortX(final byte[] regionName, final long transactionId, final int participantNum, 
+               final boolean dropTableRecorded, boolean ignoreUnknownTransaction) throws IOException{
         if(LOG.isDebugEnabled()) LOG.debug("doAbortX -- ENTRY txID: " + transactionId + " participantNum "
                         + participantNum + " region " + regionName.toString());
         boolean retry = false;
@@ -896,6 +908,8 @@ public class TransactionManager {
         Admin admin = connection.getAdmin();
         if( TRANSACTION_ALGORITHM == AlgorithmType.MVCC) {
         do {
+            retry = false;
+            refresh = false;
             try {
 
               Batch.Call<TrxRegionService, AbortTransactionResponse> callable =
@@ -924,32 +938,29 @@ public class TransactionManager {
                  result = table.coprocessorService(TrxRegionService.class, startKey, endKey, callable);
               } catch (ServiceException se) {
                   String msg = "ERROR occurred while calling doAbortX coprocessor service";
-                  LOG.error(msg,  se);
+                  LOG.warn(msg,  se);
                   throw new RetryTransactionException(msg, se);
               } catch (Throwable t) {
                   String msg = "ERROR occurred while calling doAbortX coprocessor service";
                   LOG.error(msg,  t);
-                  transactionState.requestPendingCountDec(true);
-                  throw new DoNotRetryIOException(msg, t);
+                  DoNotRetryIOException dnre = new DoNotRetryIOException(msg, t);
+                  transactionState.requestPendingCountDec(dnre);
+                  throw dnre;
               }
               
 
               if(result.size() == 0) {
-                 LOG.error("doAbortX, received 0 region results for transaction: " + transactionId
+                 LOG.error("doAbortX, received 0 region results for transaction " + transactionId
                 		   + " participantNum: " + participantNum + " region: " + Bytes.toString(regionName));
                  refresh = true;
                  retry = true;
               }
               else {
                  for (AbortTransactionResponse cresponse : result.values()) {
-                   if(cresponse.getHasException()) {
-                     String exceptionString = new String (cresponse.getException());
-		     String errMsg = new String("Abort of transaction: " + transactionId
-                          + " participantNum: " + participantNum + " region: " + Bytes.toString(regionName)
-                          + " threw Exception: " + exceptionString);
-                     LOG.error(errMsg);
-                     if(exceptionString.contains("UnknownTransactionException")) {
-                       throw new UnknownTransactionException(errMsg);
+                   String exceptionString = cresponse.getException();
+                   if (exceptionString != null) {
+                     if (exceptionString.contains("UnknownTransactionException")) {
+                       throw new UnknownTransactionException(exceptionString);
                      }
                      throw new RetryTransactionException(cresponse.getException());
                    }
@@ -958,26 +969,37 @@ public class TransactionManager {
               }
            }
           catch (UnknownTransactionException ute) {
-             LOG.error("Got unknown exception in doAbortX by participant " + participantNum
-                       + " for transaction: " + transactionId, ute);
-             transactionState.requestPendingCountDec(true);
-             throw ute;
+             String errMsg = new String("doAbortX UnknownTransactionException for transaction "
+                              + transactionId + " participantNum " + participantNum + " Location " + location.getRegionInfo().getRegionNameAsString());
+             if (ignoreUnknownTransaction) {
+                LOG.info(errMsg + " ,but ignored", ute);
+                transactionState.requestPendingCountDec(null);
+             }
+             else {
+                LOG.error(errMsg, ute);
+                transactionState.logUteDetails();
+                transactionState.requestPendingCountDec(null);
+             }
           }
           catch (RetryTransactionException rte) {
               if (retryCount == RETRY_ATTEMPTS) {
-                  String errMsg = "Exceeded retry attempts in doAbortX: " + retryCount;
-                  LOG.error(errMsg, rte); 
-                  transactionState.requestPendingCountDec(true);
-                  throw new DoNotRetryIOException(errMsg, rte);
+                 String errMsg = new String("Exceeded " + retryCount + " retry attempts in doAbortX for transaction " 
+                     + transactionId + " participantNum " + participantNum + " Location " + location.getRegionInfo().getRegionNameAsString());
+                 DoNotRetryIOException dnre = new DoNotRetryIOException(errMsg, rte);
+                 LOG.error(errMsg, dnre);
+                 transactionState.requestPendingCountDec(dnre);
+                 throw dnre;
               }
               else if (rte.toString().contains("Asked to commit a non-pending transaction ")) {
-                 LOG.error(" doCommitX will not retry transaction: " + transactionId , rte);
+                 String errMsg = new String("doAbortX will not retry transaction" 
+                     + transactionId + " participantNum " + participantNum + " Location " + location.getRegionInfo().getRegionNameAsString());
+                 LOG.error(errMsg,rte);
                  refresh = false;
                  retry = false;
               }
               else {
-                  LOG.error("doAbortX retrying transaction: " + transactionId + " participantNum: "
-                       + participantNum + " region: " + Bytes.toString(regionName) + " due to Exception: " ,rte );
+                  LOG.warn("doAbortX retrying " + retryCount + " time for transaction " + transactionId + " participantNum: "
+                      + participantNum + " Location " + location.getRegionInfo().getRegionNameAsString());
                  refresh = true;
                  retry = true;
               }
@@ -987,21 +1009,30 @@ public class TransactionManager {
                  HRegionInfo     lv_hri = lv_hrl.getRegionInfo();
 
                  if (LOG.isTraceEnabled()) LOG.trace("doAbortX -- location being refreshed : " + location.getRegionInfo().getRegionNameAsString() + "endKey: "
-                     + Hex.encodeHexString(location.getRegionInfo().getEndKey()) + " for transaction: " + transactionId);
-                    if (LOG.isWarnEnabled()) LOG.warn("doAbortX -- " + table.toString() + " location being refreshed");
-                    if (LOG.isWarnEnabled()) LOG.warn("doAbortX -- lv_hri: " + lv_hri);
-                    if (LOG.isWarnEnabled()) LOG.warn("doAbortX -- location.getRegionInfo(): " + location.getRegionInfo());
+                     + Hex.encodeHexString(location.getRegionInfo().getEndKey()) + " for transaction " + transactionId);
+                 if (LOG.isWarnEnabled()) LOG.warn("doAbortX -- " + table.toString() + " location being refreshed");
+                 if (LOG.isWarnEnabled()) LOG.warn("doAbortX -- lv_hri: " + lv_hri);
+                 if (LOG.isWarnEnabled()) LOG.warn("doAbortX -- location.getRegionInfo(): " + location.getRegionInfo());
+                 if (admin.isTableEnabled(TableName.valueOf(table.getTableName()))) {
                     table.getRegionLocation(startKey, true);
-                 if (LOG.isTraceEnabled()) LOG.trace("doAbortX -- setting retry, count: " + retryCount);
+                 }
+                 else {
+                    LOG.error("doAbortX -- table: " + table.toString() + " is disabled, ignoring table and returning");
+                    transactionState.requestPendingCountDec(null);
+                    return 0;                    	
+                 }
+                 if (LOG.isWarnEnabled()) LOG.warn("doAbortX -- setting retry, count: " + retryCount);
                  refresh = false;
             }
             if (retry)
                retrySleep = retry(retrySleep);
           } while (retry && retryCount++ <= RETRY_ATTEMPTS);
-
         }
+
         if( TRANSACTION_ALGORITHM == AlgorithmType.SSCC){
         do {
+             retry = false;
+             refresh = false;
              try {
 
               Batch.Call<SsccRegionService, SsccAbortTransactionResponse> callable =
@@ -1023,54 +1054,61 @@ public class TransactionManager {
 
             Map<byte[], SsccAbortTransactionResponse> result = null;
               try {
-                      if (LOG.isTraceEnabled()) LOG.trace("doAbortX -- before coprocessorService txid: " + transactionId + " table: " +
+                  if (LOG.isTraceEnabled()) LOG.trace("doAbortX -- before coprocessorService txid: " + transactionId + " table: " +
                         table.toString() + " startKey: " + new String(startKey, "UTF-8") + " endKey: " + new String(endKey, "UTF-8"));
-                      result = table.coprocessorService(SsccRegionService.class, startKey, endKey, callable);
+                  result = table.coprocessorService(SsccRegionService.class, startKey, endKey, callable);
               } catch (ServiceException se) {
                   String msg = "ERROR occurred while calling doAbortX coprocessor service";
-                  LOG.error(msg + ":",  se);
+                  LOG.warn(msg + ":",  se);
                   throw new RetryTransactionException(msg, se);
               } catch (Throwable e) {
                   String msg = "ERROR occurred while calling doAbortX coprocessor service";
                   LOG.error(msg + ":",  e);
-                  transactionState.requestPendingCountDec(true);
-                  throw new DoNotRetryIOException(msg,e);
+                  DoNotRetryIOException dnre = new DoNotRetryIOException(msg,e);
+                  transactionState.requestPendingCountDec(dnre);
+                  throw dnre;
               }
 
-              if(result.size() != 1) {
-                     LOG.error("doAbortX, received incorrect result size: " + result.size());
-                     refresh = true;
-                     retry = true;
-                  }
-                  else {
-                     for (SsccAbortTransactionResponse cresponse : result.values()) {
-                if(cresponse.getHasException()) {
-                  String exceptionString = cresponse.getException();
-                  String errMsg = new String("Abort of transaction: " + transactionId + " threw Exception: " + exceptionString);
-                  LOG.error(errMsg);
-                  if(exceptionString.contains("UnknownTransactionException")) {
-                         throw new UnknownTransactionException(errMsg);
-                  }
-                  throw new RetryTransactionException(cresponse.getException());
-                }
+              if (result.size() != 1) {
+                 LOG.error("doAbortX, received incorrect result size: " + result.size());
+                 refresh = true;
+                 retry = true;
               }
-              retry = false;
+              else {
+                 for (SsccAbortTransactionResponse cresponse : result.values()) {
+                    if (cresponse.getHasException()) {
+                       String exceptionString = cresponse.getException();
+                       if (exceptionString.contains("UnknownTransactionException")) {
+                          throw new UnknownTransactionException(exceptionString);
+                       }
+                       throw new RetryTransactionException(cresponse.getException());
+                    }
+                 }
               }
+              retry = false;
           }
           catch (UnknownTransactionException ute) {
-             LOG.error("Got unknown exception in doAbortX by participant " + participantNum
-                       + " for transaction: " + transactionId, ute);
-             transactionState.requestPendingCountDec(true);
-             throw ute;
+             String errMsg = new String("Got unknown exception in doAbortX by participant " + participantNum
+                       + " for transaction " + transactionId);
+             if (ignoreUnknownTransaction) {
+                LOG.info(errMsg + " ,but ignored", ute);
+                transactionState.requestPendingCountDec(null);
+             }
+             else {
+                LOG.error(errMsg, ute);
+                transactionState.logUteDetails();
+                transactionState.requestPendingCountDec(null);
+             }
           }
           catch (RetryTransactionException rte) {
               if (retryCount == RETRY_ATTEMPTS){
                    String errMsg = new String ("Exceeded retry attempts in doAbortX: " + retryCount + " (Not ingoring)");
                    LOG.error(errMsg);
-                   transactionState.requestPendingCountDec(true);
-                   throw new RollbackUnsuccessfulException(errMsg, rte);  
+                   RollbackUnsuccessfulException rue = new RollbackUnsuccessfulException(errMsg, rte);  
+                   transactionState.requestPendingCountDec(rue);
+                   throw rue;
               }
-              LOG.error("doAbortX participant " + participantNum + " retrying transaction "
+              LOG.warn("doAbortX participant " + participantNum + " retrying transaction "
                       + transactionId + " due to Exception: " + rte);
               refresh = true;
               retry = true;
@@ -1081,12 +1119,19 @@ public class TransactionManager {
                  HRegionInfo     lv_hri = lv_hrl.getRegionInfo();
 
                  if (LOG.isTraceEnabled()) LOG.trace("doAbortX -- location being refreshed : " + location.getRegionInfo().getRegionNameAsString() + "endKey: "
-                     + Hex.encodeHexString(location.getRegionInfo().getEndKey()) + " for transaction: " + transactionId);
-                    if (LOG.isWarnEnabled()) LOG.warn("doAbortX -- " + table.toString() + " location being refreshed");
-                    if (LOG.isWarnEnabled()) LOG.warn("doAbortX -- lv_hri: " + lv_hri);
-                    if (LOG.isWarnEnabled()) LOG.warn("doAbortX -- location.getRegionInfo(): " + location.getRegionInfo());
+                     + Hex.encodeHexString(location.getRegionInfo().getEndKey()) + " for transaction " + transactionId);
+                 if (LOG.isWarnEnabled()) LOG.warn("doAbortX -- " + table.toString() + " location being refreshed");
+                 if (LOG.isWarnEnabled()) LOG.warn("doAbortX -- lv_hri: " + lv_hri);
+                 if (LOG.isWarnEnabled()) LOG.warn("doAbortX -- location.getRegionInfo(): " + location.getRegionInfo());
+                 if (admin.isTableEnabled(TableName.valueOf(table.getTableName()))) {
                     table.getRegionLocation(startKey, true);
-                 if (LOG.isTraceEnabled()) LOG.trace("doAbortX -- setting retry, count: " + retryCount);
+                 }
+                 else {
+                    LOG.error("doAbortX -- table: " + table.toString() + " is disabled, ignoring table and returning");
+                    transactionState.requestPendingCountDec(null);
+                    return 0;                    	
+                 }
+                 if (LOG.isWarnEnabled()) LOG.warn("doAbortX -- setting retry, count: " + retryCount);
                  refresh = false;
               }
               if (retry) 
@@ -1096,13 +1141,8 @@ public class TransactionManager {
         }
       admin.close();
       // We have received our reply so decrement outstanding count
-      transactionState.requestPendingCountDec(false);
+      transactionState.requestPendingCountDec(null);
 
-      // forget the transaction if all replies have been received.
-      //  otherwise another thread will do it
-//      if (transactionState.requestAllComplete())
-//      {
-//      }
       if(LOG.isTraceEnabled()) LOG.trace("doAbortX -- EXIT txID: " + transactionId);
       return 0;
     }
@@ -1114,6 +1154,8 @@ public class TransactionManager {
 
         int retryCount = 0;
         do {
+          retry = false;
+          refresh = false;
           try {
 
             if (LOG.isTraceEnabled()) LOG.trace("doCommitX - Batch -- ENTRY txid: " + transactionId
@@ -1134,18 +1176,21 @@ public class TransactionManager {
                 TrxRegionService.BlockingInterface trxService = TrxRegionService.newBlockingStub(channel);
                 commitMultipleResponse = trxService.commitMultiple(null, commitMultipleRequest);
                 retry = false;
-		    } catch (ServiceException se) {
-		      String msg = "doCommitX coprocessor error for " + Bytes.toString(locations.iterator().next().getRegionInfo().getRegionName()) + " txid: " + transactionId;
-		      LOG.error(msg, se);
-		      refresh = true;
-		      throw new RetryTransactionException(msg, se);
-		    } catch (Throwable e) {
-		      String errMsg = "doCommitX coprocessor error for " + Bytes.toString(locations.iterator().next().getRegionInfo().getRegionName()) + " txid: " + transactionId;
-		      LOG.error(errMsg,e);
-                      transactionState.requestPendingCountDec(true);
-		      throw new CommitUnsuccessfulException(errMsg, e);
-		    }
-		  if(!retry) {
+            } catch (ServiceException se) {
+                 String errMsg = new String ("ERROR occurred while calling coprocessor service in doCommitX for transaction "
+                              + transactionId + " participantNum " + participantNum + " Table " + location.getRegionInfo().getTable().toString());
+                 LOG.warn(errMsg, se);
+                 refresh = true;
+                 throw new RetryTransactionException(errMsg, se);
+            } catch (Throwable e) {
+                 String errMsg = new String ("ERROR occurred while calling coprocessor service in doCommitX for transaction "
+                        + transactionId + " participantNum " + participantNum + " Table " + location.getRegionInfo().getTable().toString());
+                 LOG.error(errMsg,e);
+                 CommitUnsuccessfulException cue = new CommitUnsuccessfulException(errMsg, e);
+                 transactionState.requestPendingCountDec(cue);
+                 throw cue;
+            }
+		 if(!retry) {
 		      List<String> exceptions = commitMultipleResponse.getExceptionList();
 
 		      checkException(transactionState, locations, exceptions);
@@ -1156,12 +1201,14 @@ public class TransactionManager {
 		}
 		catch (RetryTransactionException rte) {
 		   if(retryCount == RETRY_ATTEMPTS) {
-		      LOG.error("Exceeded retry attempts (" + retryCount + ") in doCommitX for transaction: " + transactionId, rte);
-		      transactionState.requestPendingCountDec(true);
-		      throw new CommitUnsuccessfulException("Exceeded retry attempts (" + retryCount + ") in doCommitX for transaction: " + transactionId, 
-				 rte);
+                      String errMsg = new String("Exceeded " + retryCount + " retry attempts in doCommitX for transaction "
+                            + transactionId + " participantNum " + participantNum + " Table " + location.getRegionInfo().getTable().toString());
+		      LOG.error(errMsg, rte);
+		      CommitUnsuccessfulException cue = new CommitUnsuccessfulException(errMsg, rte);
+		      transactionState.requestPendingCountDec(cue);
+                      throw cue;
 		   }
-		   LOG.error("doCommitX retrying transaction " + transactionId
+		   LOG.warn("doCommitX retrying transaction " + transactionId
 				   + " participant " + participantNum + " due to Exception: ", rte);
 		   refresh = true;
 		   retry = true;
@@ -1172,7 +1219,7 @@ public class TransactionManager {
 		   HRegionInfo     lv_hri = lv_hrl.getRegionInfo();
 
 		   if (LOG.isTraceEnabled()) LOG.trace("doCommitX -- location being refreshed : " + location.getRegionInfo().getRegionNameAsString() + "endKey: "
-			   + Hex.encodeHexString(location.getRegionInfo().getEndKey()) + " for transaction: " + transactionId);
+			   + Hex.encodeHexString(location.getRegionInfo().getEndKey()) + " for transaction " + transactionId);
 
 		 if (LOG.isWarnEnabled()) {
 		   LOG.warn("doCommitX -- " + table.toString() + " location being refreshed");
@@ -1187,7 +1234,7 @@ public class TransactionManager {
 	    }  while (retry && retryCount++ <= RETRY_ATTEMPTS);
 
 
-	    transactionState.requestPendingCountDec(false);
+	    transactionState.requestPendingCountDec(null);
 
 	    if (LOG.isTraceEnabled()) LOG.trace("doCommitX - Batch -- EXIT txid: " + transactionId);
 	    return 0;
@@ -1203,6 +1250,8 @@ public class TransactionManager {
 	    int retryCount = 0;
 	    List<Integer> results = null;
 	    do {
+               retry = false;
+               refresh = false;
 	       try {
 
 		  TrxRegionProtos.CommitRequestMultipleRequest.Builder builder = CommitRequestMultipleRequest.newBuilder();
@@ -1220,15 +1269,17 @@ public class TransactionManager {
 		      commitMultipleResponse = trxService.commitRequestMultiple(null, commitMultipleRequest);
 		      retry = false;
 		  } catch (ServiceException se) {
-                       String errMsg = new String("doPrepareX coprocessor error for " + Bytes.toString(locations.iterator().next().getRegionInfo().getRegionName()) + " txid: " + transactionId );
-                       LOG.error(errMsg, se);
+                       String errMsg = new String ("ERROR occurred while calling coprocessor service in doPrepareX for transaction "
+                              + transactionId + " participantNum " + participantNum + " Table " + location.getRegionInfo().getTable().toString());
+                       LOG.warn(errMsg, se);
                        refresh = true;
   		       throw new RetryTransactionException(errMsg, se);
                   } catch (Throwable e) {
-                      String errMsg = "doPrepareX coprocessor error for " + Bytes.toString(locations.iterator().next().getRegionInfo().getRegionName()) + " txid: " + transactionId; 
+                      String errMsg = new String ("ERROR occurred while calling coprocessor service in doPrepareX for transaction "
+                              + transactionId + " participantNum " + participantNum + " Table " + location.getRegionInfo().getTable().toString());
                       LOG.error(errMsg, e);
-                      transactionState.requestPendingCountDec(true);
-                      throw new CommitUnsuccessfulException("Unable to call prepare, coprocessor error", e);
+                      CommitUnsuccessfulException cue = new CommitUnsuccessfulException("Unable to call prepare, coprocessor error", e);
+                      throw cue;
                    }
                    if(!retry) {
               results = commitMultipleResponse.getResultList();
@@ -1238,16 +1289,15 @@ public class TransactionManager {
 
           }
        }
-       catch(RetryTransactionException rte) {
-         if(retryCount == RETRY_ATTEMPTS){
-            LOG.error("Exceeded retry attempts in doPrepareX: " + retryCount, rte);
-            // We have received our reply in the form of an exception,
-            // so decrement outstanding count and wake up waiters to avoid
-            // getting hung forever
-            transactionState.requestPendingCountDec(true);
-            throw new CommitUnsuccessfulException("Exceeded retry attempts in doPrepareX: " + retryCount, rte);
+       catch (RetryTransactionException rte) {
+         if (retryCount == RETRY_ATTEMPTS) {
+            String errMsg = new String("Exceeded " + retryCount + " retry attempts in doPrepareX for transaction "
+                        + transactionId + " participantNum " + participantNum + " Table " + location.getRegionInfo().getTable().toString());
+            LOG.error(errMsg, rte);
+            CommitUnsuccessfulException cue = new CommitUnsuccessfulException(errMsg, rte);
+            throw cue;
           }
-          LOG.error("doPrepareX - Batch - retrying for participant "
+          LOG.warn("doPrepareX - Batch - retrying for participant "
                    + participantNum + " transaction " + transactionId + " due to Exception: ", rte);
           refresh = true;
           retry = true;
@@ -1257,7 +1307,7 @@ public class TransactionManager {
          HRegionInfo     lv_hri = lv_hrl.getRegionInfo();
 
          if (LOG.isTraceEnabled()) LOG.trace("doPrepareX -Batch- location being refreshed : " + location.getRegionInfo().getRegionNameAsString() + "endKey: "
-                  + Hex.encodeHexString(location.getRegionInfo().getEndKey()) + " for transaction: " + transactionId);
+                  + Hex.encodeHexString(location.getRegionInfo().getEndKey()) + " for transaction " + transactionId);
          if (LOG.isWarnEnabled()) {
             LOG.warn("doPrepareX -Batch- " + table.toString() + " location being refreshed");
             LOG.warn("doPrepareX -Batch- lv_hri: " + lv_hri);
@@ -1281,6 +1331,8 @@ public class TransactionManager {
     int resultCount = 0;
 
     for(Integer commitStatus : results) {
+       String errMsg = new String("Received invalid return code " + commitStatus + " from commit request in doPrepareX for transaction " 
+                     + transactionId + " participantNum " + participantNum + " Table " + location.getRegionInfo().getTable().toString());
        switch (commitStatus) {
           case 0:
             break;
@@ -1300,11 +1352,13 @@ public class TransactionManager {
                  commitErr = TransactionalReturn.COMMIT_UNSUCCESSFUL;
              canCommit = false;
              readOnly = false;
+             LOG.warn(errMsg);
              transactionState.addRegionToIgnore(locations.get(resultCount)); // No need to re-abort.
              break;
           default:
-             LOG.warn("Received invalid return code from requestCommit " + commitStatus + " for transaction " + transactionId + " throwing CommitUnsuccessfulException");
-             throw new CommitUnsuccessfulException("Unexpected return code from prepareCommit: " + commitStatus);
+             CommitUnsuccessfulException cue = new CommitUnsuccessfulException(errMsg);
+             LOG.warn(errMsg, cue);
+             throw cue;
        }
        resultCount++;
     }
@@ -1315,9 +1369,6 @@ public class TransactionManager {
      LOG.warn("Region [" + location.getRegionInfo().getRegionNameAsString() + "] votes "
              +  "to abort" + (readOnly ? " Read-only ":"") + " transaction "
              + transactionState.getTransactionId());
-     //System.out.println("Region [" + location.getRegionInfo().getRegionNameAsString() + "] votes "
-     //        +  "to abort" + (readOnly ? " Read-only ":"") + " transaction "
-     //        + transactionState.getTransactionId());
      if(commitErr == TransactionalReturn.COMMIT_CONFLICT) {
           return TM_COMMIT_FALSE_CONFLICT;
      }
@@ -1338,6 +1389,8 @@ public class TransactionManager {
     boolean refresh = false;
     int retryCount = 0;
     do {
+      retry = false;
+      refresh = false;
       try {
           TrxRegionProtos.AbortTransactionMultipleRequest.Builder builder = AbortTransactionMultipleRequest.newBuilder();
           builder.setTransactionId(transactionId);
@@ -1353,15 +1406,17 @@ public class TransactionManager {
               abortTransactionMultipleResponse = trxService.abortTransactionMultiple(null, abortTransactionMultipleRequest);
               retry = false;
           } catch (ServiceException se) {
-              String errMsg = "doAbortX - Batch - coprocessor error for " + Bytes.toString(locations.iterator().next().getRegionInfo().getRegionName()) + " txid: " + transactionId;
-              refresh = true;
-              LOG.error(errMsg, se);
-              throw new RetryTransactionException(errMsg, se);
+              String errMsg = new String ("ERROR occurred while calling coprocessor service in doAbortX for transaction "
+                          + transactionId + " participantNum " + participantNum + " Table " + location.getRegionInfo().getTable().toString());
+              LOG.warn(errMsg, se);
+              throw new RetryTransactionException(errMsg,se);
           } catch (Throwable e) {
-              String errMsg = "doAbortX - Batch - coprocessor error for " + Bytes.toString(locations.iterator().next().getRegionInfo().getRegionName()) + " txid: " + transactionId; 
-             LOG.error(errMsg, e);
-              transactionState.requestPendingCountDec(true);
-              throw new RollbackUnsuccessfulException("doAbortX, Batch - coprocessor error", e);
+              String errMsg = new String ("ERROR occurred while calling coprocessor service in doAbortX for transaction "
+                              + transactionId + " participantNum " + participantNum + " Table " + location.getRegionInfo().getTable().toString());
+              LOG.error(errMsg, e);
+              RollbackUnsuccessfulException rue = new RollbackUnsuccessfulException("doAbortX, Batch - coprocessor error", e);
+              transactionState.requestPendingCountDec(rue);
+              throw rue;
           }
           if(!retry) {
               List<String> exceptions = abortTransactionMultipleResponse.getExceptionList();
@@ -1372,13 +1427,15 @@ public class TransactionManager {
           }
          }
          catch (RetryTransactionException rte) {
-            if(retryCount == RETRY_ATTEMPTS){
-               String errMsg = "Exceeded retry attempts in doAbortX: " + retryCount + " (not ingoring)";
+            if (retryCount == RETRY_ATTEMPTS) {
+               String errMsg = new String("Exceeded " + retryCount + " retry attempts in doAbortX for transaction "
+                        + transactionId + " participantNum " + participantNum + " Table " + location.getRegionInfo().getTable().toString());
                LOG.error(errMsg, rte);
-               transactionState.requestPendingCountDec(true);
-               throw new RollbackUnsuccessfulException("doAbortX, Batch - coprocessor error", rte);
+               RollbackUnsuccessfulException rue = new RollbackUnsuccessfulException("doAbortX, Batch - coprocessor error", rte);
+               transactionState.requestPendingCountDec(rue);
+               throw rue;
             }
-            LOG.error("doAbortX - Batch - participant " + participantNum + " retrying transaction "
+            LOG.warn("doAbortX - Batch - participant " + participantNum + " retrying transaction "
                         + transactionId + " due to Exception: ", rte);
             refresh = true;
             retry = true;
@@ -1390,7 +1447,7 @@ public class TransactionManager {
             if (LOG.isTraceEnabled()) LOG.trace("doAbortX - Batch - participant " + participantNum
             		+ "-- location being refreshed : " + location.getRegionInfo().getRegionNameAsString()
                     + " endKey: " + Hex.encodeHexString(location.getRegionInfo().getEndKey())
-                    + " for transaction: " + transactionId);
+                    + " for transaction " + transactionId);
            if (LOG.isWarnEnabled()) {
              LOG.warn("doAbortX - Batch - -- " + table.toString() + " location being refreshed");
              LOG.warn("doAbortX - Batch - -- lv_hri: " + lv_hri);
@@ -1404,7 +1461,7 @@ public class TransactionManager {
       }  while (retry && retryCount++ <= RETRY_ATTEMPTS);
 
 
-      transactionState.requestPendingCountDec(false);
+      transactionState.requestPendingCountDec(null);
       if(LOG.isTraceEnabled()) LOG.trace("doAbortX - Batch -- EXIT txID: " + transactionId);
       return 0;
    }
@@ -1614,7 +1671,7 @@ public class TransactionManager {
       else {
          if (LOG.isTraceEnabled()) LOG.trace("beginTransaction NOT retrieving new startId");
       }
-      if (LOG.isTraceEnabled()) LOG.trace("beginTransaction setting transaction: [" + ts.getTransactionId() +
+      if (LOG.isTraceEnabled()) LOG.trace("beginTransaction setting transaction [" + ts.getTransactionId() +
                       "], startEpoch: " + ts.getStartEpoch() + " and startId: " + startIdVal);
       ts.setStartId(startIdVal);
       return ts;
@@ -1913,7 +1970,7 @@ public class TransactionManager {
     public void tryCommit(final TransactionState transactionState)
         throws CommitUnsuccessfulException, UnsuccessfulDDLException, IOException {
         long startTime = EnvironmentEdgeManager.currentTime();
-        if (LOG.isTraceEnabled()) LOG.trace("Attempting to commit transaction: " + transactionState.toString());
+        if (LOG.isTraceEnabled()) LOG.trace("Attempting to commit transaction " + transactionState.toString());
         int status = prepareCommit(transactionState);
 
         if (status == TransactionalReturn.COMMIT_OK) {
@@ -1940,7 +1997,7 @@ public class TransactionManager {
           for (TransactionRegionLocation location : transactionState.getRetryRegions()) {
             loopCount++;
             final int participantNum = loopCount;
-            if(LOG.isTraceEnabled()) LOG.trace("retryCommit retrying commit for transaction: "
+            if(LOG.isTraceEnabled()) LOG.trace("retryCommit retrying commit for transaction "
                     + transactionState.getTransactionId() + ", participant: " + participantNum + ", region "
                     + location.getRegionInfo().getRegionNameAsString());
             threadPool.submit(new TransactionManagerCallable(transactionState, location, connection) {
@@ -2014,16 +2071,16 @@ public class TransactionManager {
           List<TransactionRegionLocation> completedList = new ArrayList<TransactionRegionLocation>();
           int loopCount = 0;
           for (TransactionRegionLocation location : transactionState.getRetryRegions()) {
-             loopCount++;
-             final int participantNum = loopCount;
-             if(LOG.isTraceEnabled()) LOG.trace("retryAbort retrying abort for transaction: "
-                      + transactionState.getTransactionId() + ", participant: "
-              		+ participantNum + ", region: " + location.getRegionInfo().getRegionNameAsString());
-              threadPool.submit(new TransactionManagerCallable(transactionState, location, connection) {
+            loopCount++;
+            final int participantNum = loopCount;
+            if(LOG.isTraceEnabled()) LOG.trace("retryAbort retrying abort for transaction "
+                    + transactionState.getTransactionId() + ", participant: "
+            		+ participantNum + ", region: " + location.getRegionInfo().getRegionNameAsString());
+             threadPool.submit(new TransactionManagerCallable(transactionState, location, connection) {
                   public Integer call() throws CommitUnsuccessfulException, IOException {
 
                       return doAbortX(location.getRegionInfo().getRegionName(),
-                              transactionState.getTransactionId(), participantNum, location.isTableRecodedDropped());
+                              transactionState.getTransactionId(), participantNum, location.isTableRecodedDropped(), false);
                   }
               });
               completedList.add(location);
@@ -2032,6 +2089,7 @@ public class TransactionManager {
       }
       if(LOG.isTraceEnabled()) LOG.trace("retryAbort -- EXIT -- txid: " + transactionState.getTransactionId());
     }
+
     /**
      * Do the commit. This is the 2nd phase of the 2-phase protocol.
      *
@@ -2149,6 +2207,11 @@ public class TransactionManager {
                 loopExit = true; 
               } 
               catch (InterruptedException ie) {}
+              catch (IOException e) {
+                 loopExit = true;
+                 LOG.error("Exception at the time of committing DML before processing DDL ", e); 
+                 throw e;
+              }
             } while (loopExit == false);
 
             if (LOG.isDebugEnabled()) LOG.debug("doCommit() [" + transactionState.getTransactionId()
@@ -2182,7 +2245,7 @@ public class TransactionManager {
             catch(IOException e){
                 LOG.info("Exception in doCommitDDL, Step: getRow. txID: " + transactionState.getTransactionId() + "Exception: " , e);
 
-                if(retryCount == RETRY_ATTEMPTS)
+                if (retryCount == RETRY_ATTEMPTS)
                 {
                     LOG.error("Fatal Exception in doCommitDDL, Step: getRow. Raising CommitUnsuccessfulException txID: " + transactionState.getTransactionId() + "Exception: " + e);
 
@@ -2278,16 +2341,16 @@ public class TransactionManager {
       if(LOG.isTraceEnabled()) LOG.trace("Abort -- ENTRY txID: " + transactionState.getTransactionId());
         int loopCount = 0;
 
-      transactionState.setStatus(TransState.STATE_ABORTED);
-      // (Asynchronously send aborts
-      if (batchRegionServer && (TRANSACTION_ALGORITHM == AlgorithmType.MVCC)) {
-        ServerName servername;
-        List<TransactionRegionLocation> regionList;
-        Map<ServerName, List<TransactionRegionLocation>> locations = new HashMap<ServerName, List<TransactionRegionLocation>>();
+          transactionState.setStatus(TransState.STATE_ABORTED);
+          // (Asynchronously send aborts
+          if (batchRegionServer && (TRANSACTION_ALGORITHM == AlgorithmType.MVCC)) {
+            ServerName servername;
+            List<TransactionRegionLocation> regionList;
+            Map<ServerName, List<TransactionRegionLocation>> locations = new HashMap<ServerName, List<TransactionRegionLocation>>();
 
         for (TransactionRegionLocation location : transactionState.getParticipatingRegions()) {
             if (transactionState.getRegionsToIgnore().contains(location)) {
-                continue;
+               continue;
             }
             servername = location.getServerName();
 
@@ -2329,7 +2392,7 @@ public class TransactionManager {
             		+ participantNum + ", region: " + location.getRegionInfo().getRegionNameAsString());
             threadPool.submit(new TransactionManagerCallable(transactionState, location, connection) {
               public Integer call() throws IOException {
-                return doAbortX(regionName, transactionState.getTransactionId(), participantNum, location.isTableRecodedDropped());
+                return doAbortX(regionName, transactionState.getTransactionId(), participantNum, location.isTableRecodedDropped(), false);
               }
             });
         }
@@ -2338,7 +2401,7 @@ public class TransactionManager {
         transactionState.completeSendInvoke(loopCount);
     }
          
-        CommitUnsuccessfulException savedCue = null;
+       IOException savedException = null;
 
         //if DDL is involved with this transaction, need to unwind it.
         if(transactionState.hasDDLTx())
@@ -2354,15 +2417,15 @@ public class TransactionManager {
                 loopExit = true; 
               } 
               catch (InterruptedException ie) {}
-              catch (CommitUnsuccessfulException cue) {
+              catch (IOException e) {
                  loopExit = true;
-                 LOG.error("Exception at the time of aborting DDL transaction", cue); 
-                 savedCue = cue;
+                 LOG.error("Exception at the time of aborting DML before processing DDL", e); 
+                 savedException = e;
               }
             } while (loopExit == false);
             abortDDL(transactionState);
-            if (savedCue != null)
-               throw savedCue; 
+            if (savedException != null)
+               throw savedException; 
         }
 
         if(LOG.isTraceEnabled()) LOG.trace("Abort -- EXIT txID: " + transactionState.getTransactionId());

http://git-wip-us.apache.org/repos/asf/incubator-trafodion/blob/ee1cd85d/core/sqf/src/seatrans/hbase-trx/src/main/java/org/apache/hadoop/hbase/client/transactional/TransactionState.java
----------------------------------------------------------------------
diff --git a/core/sqf/src/seatrans/hbase-trx/src/main/java/org/apache/hadoop/hbase/client/transactional/TransactionState.java b/core/sqf/src/seatrans/hbase-trx/src/main/java/org/apache/hadoop/hbase/client/transactional/TransactionState.java
index dc02056..a49c1d1 100644
--- a/core/sqf/src/seatrans/hbase-trx/src/main/java/org/apache/hadoop/hbase/client/transactional/TransactionState.java
+++ b/core/sqf/src/seatrans/hbase-trx/src/main/java/org/apache/hadoop/hbase/client/transactional/TransactionState.java
@@ -35,6 +35,7 @@ import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.hbase.HRegionLocation;
 import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.HConstants;
 
 /**
  * Holds client-side transaction information. Client's use them as opaque objects passed around to transaction
@@ -64,12 +65,13 @@ public class TransactionState {
     private Object countLock;
     private boolean commitSendDone;
     private Object commitSendLock;
-    private boolean hasError;
+    private Throwable hasError;
     private boolean localTransaction;
     private boolean ddlTrans;
     private static boolean useConcurrentHM = false;
     private static boolean getCHMVariable = true;
     private boolean hasRetried = false;
+    private boolean uteLogged = false;
 
     public Set<String> tableNames = Collections.synchronizedSet(new HashSet<String>());
     public Set<TransactionRegionLocation> participatingRegions;
@@ -93,7 +95,7 @@ public class TransactionState {
         requestPendingCount = 0;
         requestReceivedCount = 0;
         commitSendDone = false;
-        hasError = false;
+        hasError = null;
         ddlTrans = false;
 
         if(getCHMVariable) {
@@ -175,34 +177,29 @@ public class TransactionState {
     {
         synchronized (countLock)
         {
-            hasError = false;  // reset, just in case
+            hasError = null;  // reset, just in case
             requestPendingCount = count;
         }
     }
 
     /**
      * 
-     * Method  : requestPendingCountDec
-     * Params  : None
-     * Return  : void
-     * Purpose : Decrease number of outstanding replies needed and wake up any waiters
-     *           if we receive the last one or if the wakeUp value is true (which means
-     *           we received an exception)
+     * method  : requestpendingcountdec
+     * params  : none
+     * return  : void
+     * purpose : decrease number of outstanding replies needed and wake up any waiters
+     *           if we receive the last one 
      */
-    public void  requestPendingCountDec(boolean wakeUp)
+    public void  requestPendingCountDec(Throwable exception)
     {
-        synchronized (countLock)
-        {
-            requestReceivedCount++;
-            if ((requestReceivedCount == requestPendingCount) || (wakeUp == true))
-            {
-                //Signal waiters that an error occurred
-                if (wakeUp == true)
-                    hasError = true;
-
-                countLock.notify();
-        }
-    }
+       synchronized (countLock)
+       {
+          requestReceivedCount++;
+          if (exception != null && hasError == null)
+             hasError = exception;
+          if (requestReceivedCount == requestPendingCount)
+             countLock.notify();
+       }
     }
 
     /**
@@ -212,7 +209,7 @@ public class TransactionState {
      * Return  : Void
      * Purpose : Hang thread until all replies have been received
      */
-    public void completeRequest() throws InterruptedException, CommitUnsuccessfulException
+    public void completeRequest() throws InterruptedException, IOException
     {
         // Make sure we've completed sending all requests first, if not, then wait
         synchronized (commitSendLock)
@@ -230,9 +227,10 @@ public class TransactionState {
                 countLock.wait();
         }
 
-        if (hasError)
-            throw new CommitUnsuccessfulException();
-
+        if (hasError != null)  {
+            hasError.fillInStackTrace();
+            throw new IOException("Exception at completeRequest()", hasError);
+        }
         return;
 
     }
@@ -500,4 +498,33 @@ public class TransactionState {
       return this.hasRetried;
     }
 
+    public void logUteDetails()
+    {
+       if (uteLogged)
+          return;
+       int participantNum = 0;
+       byte[] startKey;
+       byte[] endKey_orig;
+       byte[] endKey;
+
+       for (TransactionRegionLocation location : getParticipatingRegions()) {
+          participantNum++;
+          final byte[] regionName = location.getRegionInfo().getRegionName();
+
+          startKey = location.getRegionInfo().getStartKey();
+          endKey_orig = location.getRegionInfo().getEndKey();
+          if (endKey_orig == null || endKey_orig == HConstants.EMPTY_END_ROW)
+              endKey = null;
+          else
+              endKey = TransactionManager.binaryIncrementPos(endKey_orig, -1);
+
+          LOG.warn("UTE for transId: " + getTransactionId()
+                    + " participantNum " + participantNum
+                    + " location " + location.getRegionInfo().getRegionNameAsString()
+                    + " startKey " + ((startKey != null)? Hex.encodeHexString(startKey) : "NULL")
+                    + " endKey " +  ((endKey != null) ? Hex.encodeHexString(endKey) : "NULL")
+                    + " RegionEndKey " + ((endKey_orig != null) ? Hex.encodeHexString(endKey_orig) : "NULL"));
+       }
+       uteLogged = true;
+    }
 }

http://git-wip-us.apache.org/repos/asf/incubator-trafodion/blob/ee1cd85d/core/sqf/src/seatrans/tm/hbasetmlib2/src/main/java/org/trafodion/dtm/TmAuditTlog.java
----------------------------------------------------------------------
diff --git a/core/sqf/src/seatrans/tm/hbasetmlib2/src/main/java/org/trafodion/dtm/TmAuditTlog.java b/core/sqf/src/seatrans/tm/hbasetmlib2/src/main/java/org/trafodion/dtm/TmAuditTlog.java
index ede3b8c..a66d880 100644
--- a/core/sqf/src/seatrans/tm/hbasetmlib2/src/main/java/org/trafodion/dtm/TmAuditTlog.java
+++ b/core/sqf/src/seatrans/tm/hbasetmlib2/src/main/java/org/trafodion/dtm/TmAuditTlog.java
@@ -272,8 +272,9 @@ public class TmAuditTlog {
                   // We have received our reply in the form of an exception,
                   // so decrement outstanding count and wake up waiters to avoid
                   // getting hung forever
-                  transactionState.requestPendingCountDec(true);
-                  throw new IOException("Exceeded retry attempts (" + retryCount + ") in deleteEntriesOlderThanASNX for ASN: " + auditSeqNum);
+                  IOException ie = new IOException("Exceeded retry attempts (" + retryCount + ") in deleteEntriesOlderThanASNX for ASN: " + auditSeqNum);
+                  transactionState.requestPendingCountDec(ie);
+                  throw ie;
                }
 
                if (LOG.isWarnEnabled()) LOG.warn("deleteEntriesOlderThanASNX -- " + table.toString() + " location being refreshed");
@@ -297,7 +298,7 @@ public class TmAuditTlog {
             }
        } while (retryCount < TLOG_RETRY_ATTEMPTS && retry == true);
        // We have received our reply so decrement outstanding count
-       transactionState.requestPendingCountDec(false);
+       transactionState.requestPendingCountDec(null);
 
        if (LOG.isTraceEnabled()) LOG.trace("deleteEntriesOlderThanASNX -- EXIT ASN: " + auditSeqNum);
        return 0;
@@ -944,7 +945,7 @@ public class TmAuditTlog {
    }
 
    public long addControlPoint (final Map<Long, TransactionState> map) throws IOException {
-      if (LOG.isTraceEnabled()) LOG.trace("addControlPoint start with map size " + map.size());
+      if (LOG.isDebugEnabled()) LOG.debug("addControlPoint start with map size " + map.size());
       long lvCtrlPt = 0L;
       long agedAsn;  // Writes older than this audit seq num will be deleted
       long lvAsn;    // local copy of the asn
@@ -1000,7 +1001,7 @@ public class TmAuditTlog {
                throw e;
             }
          }
-      if (LOG.isTraceEnabled()) LOG.trace("addControlPoint returning " + lvCtrlPt);
+      if (LOG.isDebugEnabled()) LOG.debug("addControlPoint returning " + lvCtrlPt);
       return lvCtrlPt;
    } 
 

http://git-wip-us.apache.org/repos/asf/incubator-trafodion/blob/ee1cd85d/core/sqf/src/tm/tmmsg.h
----------------------------------------------------------------------
diff --git a/core/sqf/src/tm/tmmsg.h b/core/sqf/src/tm/tmmsg.h
index 6afffe2..c927a16 100644
--- a/core/sqf/src/tm/tmmsg.h
+++ b/core/sqf/src/tm/tmmsg.h
@@ -46,23 +46,23 @@ public:
          abort();
       memset(&iv_req, 0, sizeof(Tm_Req_Msg_Type));
       memcpy(&iv_req, pp_req, sizeof(Tm_Req_Msg_Type));
-      memset(ia_EID, 0, EID_SIZE+1);
-      //set_EID();
+      set_EID();
    }
    CTmMessage(short pv_reqType)
    {
       memset(&iv_req, 0, sizeof(Tm_Req_Msg_Type));
       requestType(pv_reqType);
-      memset(ia_EID, 0, EID_SIZE+1);
-      //set_EID();
+      set_EID();
    }
    CTmMessage(CTmMessage *pp_msg)
    {
       memcpy(&iv_req, pp_msg->request(), sizeof(iv_req));
+      set_EID();
+   }
+   ~CTmMessage()
+   {
       memset(ia_EID, 0, EID_SIZE+1);
-      //set_EID();
    }
-   ~CTmMessage() {}
 
    Tm_Req_Msg_Type * request() {return &iv_req;}
 
@@ -73,12 +73,17 @@ public:
    {
        strcpy((char *) &ia_EID, (char *) EID_CTmMessage);
    }
-    void validate()
+
+   bool validate(bool detectDoubleDelete = false)
     {
        if (strcmp((char *) &ia_EID, (char *) &EID_CTmMessage) != 0)
        {
-          ;// abort();
+          if (! detectDoubleDelete)
+             abort();
+          else
+             return false;
        }
+       return true;
     } //validate
 };
 

http://git-wip-us.apache.org/repos/asf/incubator-trafodion/blob/ee1cd85d/core/sqf/src/tm/tmtx.cpp
----------------------------------------------------------------------
diff --git a/core/sqf/src/tm/tmtx.cpp b/core/sqf/src/tm/tmtx.cpp
index a57d0d1..83b6e38 100644
--- a/core/sqf/src/tm/tmtx.cpp
+++ b/core/sqf/src/tm/tmtx.cpp
@@ -1113,8 +1113,10 @@ void TM_TX_Info::process_eventQ()
       } // switch
 
       // Protect message as registerRegion could try to reply from the main thread.
+      bool detectDoubleDelete = true;
       lock();
-      delete lp_msg;
+      if (lp_msg->validate(detectDoubleDelete))
+         delete lp_msg;
       unlock();
       // Multithreaded only:
       // Worker threads release the transaction after every request

http://git-wip-us.apache.org/repos/asf/incubator-trafodion/blob/ee1cd85d/core/sql/cli/CliExtern.cpp
----------------------------------------------------------------------
diff --git a/core/sql/cli/CliExtern.cpp b/core/sql/cli/CliExtern.cpp
index 899ddae..2f179ef 100644
--- a/core/sql/cli/CliExtern.cpp
+++ b/core/sql/cli/CliExtern.cpp
@@ -90,6 +90,7 @@ CLISemaphore globalSemaphore ;
 #include "SqlStats.h"
 #include "ComExeTrace.h"
 #include "Context.h"
+#include <unistd.h>
 #include "QRLogger.h"
 
 #ifndef CLI_PRIV_SRL

http://git-wip-us.apache.org/repos/asf/incubator-trafodion/blob/ee1cd85d/core/sql/qmscommon/QRLogger.cpp
----------------------------------------------------------------------
diff --git a/core/sql/qmscommon/QRLogger.cpp b/core/sql/qmscommon/QRLogger.cpp
index 7874c35..548532b 100644
--- a/core/sql/qmscommon/QRLogger.cpp
+++ b/core/sql/qmscommon/QRLogger.cpp
@@ -186,6 +186,9 @@ void getMyNidSuffix(char stringNidSuffix[])
 // **************************************************************************
 NABoolean QRLogger::initLog4cxx(const char* configFileName)
 {
+  if (gv_QRLoggerInitialized_)
+     return TRUE;
+
   NAString logFileName;
 
   if (gv_QRLoggerInitialized_)