You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@trafodion.apache.org by sa...@apache.org on 2017/05/12 20:25:44 UTC

[09/22] incubator-trafodion git commit: This is a large contribution of changes from Esgyn TransactionManager and libraries that are collectively much better tested and hardened than Trafodion, but are too numerous and complex to cherry pick individually

This is a large contribution of changes from Esgyn
TransactionManager and libraries that are collectively
much better tested and hardened than Trafodion, but are too
numerous and complex to cherry pick individually.

The changes include:
Dozens of bug fixes
Improved recovery after HBase failures
Better error reporting and tracing, which includes region keys for easier debugging
Reduced UnknownTransactionExceptions resulting from duplicate transaction registration
Better TLOG management including reducing the default number of tables


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

Branch: refs/heads/master
Commit: 91794b534788a1fc578192b381bcaccddb1680e1
Parents: b5eb1cf
Author: Sean Broeder <sb...@edev05.novalocal>
Authored: Fri Apr 28 20:53:19 2017 +0000
Committer: Sean Broeder <sb...@edev05.novalocal>
Committed: Fri Apr 28 20:53:19 2017 +0000

----------------------------------------------------------------------
 .../hbase/client/transactional/RMInterface.java |  292 +-
 .../hbase/client/transactional/TmDDL.java       |   28 +-
 .../hbase/client/transactional/TransState.java  |    6 +-
 .../transactional/TransactionManager.java       |  355 +-
 .../TransactionRegionLocation.java              |  137 +-
 .../client/transactional/TransactionState.java  |  157 +-
 .../TransactionalAggregationClient.java         |   83 +-
 .../TransactionalScanner.java.tmpl              |   38 +-
 .../transactional/TransactionalTable.java       |  162 +-
 .../transactional/TrxRegionEndpoint.java.tmpl   | 3437 +++++----
 .../generated/SsccRegionProtos.java             |   21 -
 .../generated/TrxRegionProtos.java              | 6828 ++++++++++--------
 .../transactional/SsccTransactionState.java     |    4 +-
 .../transactional/TransactionState.java         |  108 +-
 .../TransactionalRegionScannerHolder.java       |    7 +-
 .../transactional/TrxTransactionState.java.tmpl |  337 +-
 .../hbase-trx/src/main/protobuf/TrxRegion.proto |  175 +-
 .../trafodion/dtm/HBaseAuditControlPoint.java   |  431 +-
 .../java/org/trafodion/dtm/HBaseTxClient.java   |  344 +-
 .../java/org/trafodion/dtm/TmAuditTlog.java     | 1712 +++--
 .../java/org/trafodion/sql/HBaseClient.java     |    3 +-
 .../java/org/trafodion/sql/HTableClient.java    |   44 +-
 22 files changed, 8804 insertions(+), 5905 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-trafodion/blob/91794b53/core/sqf/src/seatrans/hbase-trx/src/main/java/org/apache/hadoop/hbase/client/transactional/RMInterface.java
----------------------------------------------------------------------
diff --git a/core/sqf/src/seatrans/hbase-trx/src/main/java/org/apache/hadoop/hbase/client/transactional/RMInterface.java b/core/sqf/src/seatrans/hbase-trx/src/main/java/org/apache/hadoop/hbase/client/transactional/RMInterface.java
index 9a42a02..f783b13 100644
--- a/core/sqf/src/seatrans/hbase-trx/src/main/java/org/apache/hadoop/hbase/client/transactional/RMInterface.java
+++ b/core/sqf/src/seatrans/hbase-trx/src/main/java/org/apache/hadoop/hbase/client/transactional/RMInterface.java
@@ -27,24 +27,36 @@ package org.apache.hadoop.hbase.client.transactional;
 import java.io.IOException;
 import java.io.ByteArrayOutputStream;
 import java.io.DataOutputStream;
+import java.io.File;
+import java.io.InterruptedIOException;
+import java.io.IOException;
 import java.io.PrintWriter;
 import java.io.StringWriter;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.commons.codec.binary.Hex;
+
+import org.apache.hadoop.fs.Path;
+
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.HRegionLocation;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.ZooKeeperConnectionException;
+import org.apache.hadoop.hbase.client.coprocessor.Batch;
 import org.apache.hadoop.hbase.client.Connection;
 import org.apache.hadoop.hbase.client.HTable;
 import org.apache.hadoop.hbase.client.Delete;
 import org.apache.hadoop.hbase.client.Get;
 import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.client.RegionLocator;
 import org.apache.hadoop.hbase.client.Result;
 import org.apache.hadoop.hbase.client.ResultScanner;
 import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.client.Table;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.client.transactional.TransactionManager;
 import org.apache.hadoop.hbase.client.transactional.TransactionState;
@@ -67,6 +79,10 @@ import org.apache.hadoop.hbase.regionserver.transactional.IdTm;
 import org.apache.hadoop.hbase.regionserver.transactional.IdTmException;
 import org.apache.hadoop.hbase.regionserver.transactional.IdTmId;
 
+import org.apache.hadoop.hbase.coprocessor.transactional.generated.TrxRegionProtos.TrxRegionService;
+import org.apache.hadoop.hbase.coprocessor.transactional.generated.TrxRegionProtos.PushEpochRequest;
+import org.apache.hadoop.hbase.coprocessor.transactional.generated.TrxRegionProtos.PushEpochResponse;
+
 import org.apache.hadoop.hbase.ipc.BlockingRpcCallback;
 import org.apache.hadoop.hbase.ipc.ServerRpcController;
 
@@ -81,6 +97,12 @@ import java.util.List;
 import java.util.Set;
 import java.util.HashSet;
 import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorCompletionService;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Callable;
+import java.util.concurrent.CompletionService;
 
 public class RMInterface {
     static final Log LOG = LogFactory.getLog(RMInterface.class);
@@ -90,7 +112,11 @@ public class RMInterface {
     public AlgorithmType TRANSACTION_ALGORITHM;
     static Map<Long, Set<RMInterface>> mapRMsPerTransaction = new HashMap<Long,  Set<RMInterface>>();
     private TransactionalTableClient ttable = null;
+    private ExecutorService threadPool;
+    private CompletionService<Integer> compPool;
+    private int intThreads = 16;
     private Connection connection;
+    static TransactionManager txm;
     static {
         System.loadLibrary("stmlib");
         String envset = System.getenv("TM_USE_SSCC");
@@ -121,11 +147,15 @@ public class RMInterface {
     private AlgorithmType transactionAlgorithm;
 
     public RMInterface(final String tableName, Connection connection) throws IOException {
-        //super(conf, Bytes.toBytes(tableName));
+        if (LOG.isTraceEnabled()) LOG.trace("RMInterface constructor:"
+					    + " tableName: " + tableName);
+
         this.connection = connection;
         transactionAlgorithm = envTransactionAlgorithm;
         if( transactionAlgorithm == AlgorithmType.MVCC) //MVCC
         {
+           if (LOG.isTraceEnabled()) LOG.trace("Algorithm type: MVCC"
+						+ " tableName: " + tableName);
             ttable = new TransactionalTable(Bytes.toBytes(tableName), connection);
         }
         else if(transactionAlgorithm == AlgorithmType.SSCC)
@@ -141,8 +171,132 @@ public class RMInterface {
 
     }
 
-    public synchronized TransactionState registerTransaction(final long transactionID, final byte[] row) throws IOException {
-        if (LOG.isTraceEnabled()) LOG.trace("Enter registerTransaction, transaction ID: " + transactionID);
+    public void pushRegionEpoch (HTableDescriptor desc, final TransactionState ts) throws IOException {
+       if (LOG.isDebugEnabled()) LOG.debug("pushRegionEpoch start; transId: " + ts.getTransactionId());
+
+       TransactionalTable ttable1 = new TransactionalTable(Bytes.toBytes(desc.getNameAsString()), connection);
+       long lvTransid = ts.getTransactionId();
+       RegionLocator rl = connection.getRegionLocator(desc.getTableName());
+       List<HRegionLocation> regionList = rl.getAllRegionLocations();
+
+       boolean complete = false;
+       int loopCount = 0;
+       int result = 0;
+
+       for (HRegionLocation location : regionList) {
+          final byte[] regionName = location.getRegionInfo().getRegionName();
+          if (compPool == null){
+              LOG.info("pushRegionEpoch compPool is null");
+              threadPool = Executors.newFixedThreadPool(intThreads);
+              compPool = new ExecutorCompletionService<Integer>(threadPool);
+          }
+
+          final HRegionLocation lv_location = location;
+          compPool.submit(new RMCallable2(ts, lv_location, connection ) {
+             public Integer call() throws IOException {
+                return pushRegionEpochX(ts, lv_location, connection);
+             }
+          });
+          try {
+            result = compPool.take().get();
+          } catch(Exception ex) {
+            throw new IOException(ex);
+          }
+          if ( result != 0 ){
+             LOG.error("pushRegionEpoch result " + result + " returned from region "
+                          + location.getRegionInfo().getRegionName());
+             throw new IOException("pushRegionEpoch result " + result + " returned from region "
+                      + location.getRegionInfo().getRegionName());
+          }
+       }
+       if (LOG.isTraceEnabled()) LOG.trace("pushRegionEpoch end transid: " + ts.getTransactionId());
+       return;
+    }
+    private abstract class RMCallable2 implements Callable<Integer>{
+       TransactionState transactionState;
+       HRegionLocation  location;
+       Connection connection;
+       HTable table;
+       byte[] startKey;
+       byte[] endKey_orig;
+       byte[] endKey;
+
+       RMCallable2(TransactionState txState, HRegionLocation location, Connection connection) {
+          this.transactionState = txState;
+          this.location = location;
+          this.connection = connection;
+          try {
+             table = new HTable(location.getRegionInfo().getTable(), connection);
+          } catch(IOException e) {
+             LOG.error("Error obtaining HTable instance " + e);
+             table = null;
+          }
+          startKey = location.getRegionInfo().getStartKey();
+          endKey_orig = location.getRegionInfo().getEndKey();
+          endKey = TransactionManager.binaryIncrementPos(endKey_orig, -1);
+
+       }
+
+       public Integer pushRegionEpochX(final TransactionState txState,
+                          final HRegionLocation location, Connection connection) throws IOException {
+          if (LOG.isTraceEnabled()) LOG.trace("pushRegionEpochX -- Entry txState: " + txState
+                   + " location: " + location);
+
+          Batch.Call<TrxRegionService, PushEpochResponse> callable =
+              new Batch.Call<TrxRegionService, PushEpochResponse>() {
+                 ServerRpcController controller = new ServerRpcController();
+                 BlockingRpcCallback<PushEpochResponse> rpcCallback =
+                    new BlockingRpcCallback<PushEpochResponse>();
+
+                 @Override
+                 public PushEpochResponse call(TrxRegionService instance) throws IOException {
+                    org.apache.hadoop.hbase.coprocessor.transactional.generated.TrxRegionProtos.PushEpochRequest.Builder
+                    builder = PushEpochRequest.newBuilder();
+                    builder.setTransactionId(txState.getTransactionId());
+                    builder.setEpoch(txState.getStartEpoch());
+                    builder.setRegionName(ByteString.copyFromUtf8(Bytes.toString(location.getRegionInfo().getRegionName())));
+                    instance.pushOnlineEpoch(controller, builder.build(), rpcCallback);
+                    return rpcCallback.get();
+                 }
+              };
+
+              Map<byte[], PushEpochResponse> result = null;
+              try {
+                 if (LOG.isTraceEnabled()) LOG.trace("pushRegionEpochX -- before coprocessorService: startKey: "
+                     + new String(startKey, "UTF-8") + " endKey: " + new String(endKey, "UTF-8"));
+                 result = table.coprocessorService(TrxRegionService.class, startKey, endKey, callable);
+              } catch (Throwable e) {
+                 String msg = "ERROR occurred while calling pushRegionEpoch coprocessor service in pushRegionEpochX";
+                 LOG.error(msg + ":" + e);
+                 throw new IOException(msg);
+              }
+
+              if(result.size() == 1){
+                 // size is 1
+                 for (PushEpochResponse eresponse : result.values()){
+                   if(eresponse.getHasException()) {
+                     String exceptionString = new String (eresponse.getException().toString());
+                     LOG.error("pushRegionEpochX - coprocessor exceptionString: " + exceptionString);
+                     throw new IOException(eresponse.getException());
+                   }
+                 }
+              }
+              else {
+                  LOG.error("pushRegionEpochX, received incorrect result size: " + result.size() + " txid: "
+                          + txState.getTransactionId() + " location: " + location.getRegionInfo().getRegionNameAsString());
+                  return 1;
+              }
+              if (LOG.isTraceEnabled()) LOG.trace("pushRegionEpochX -- Exit txState: " + txState
+                      + " location: " + location);
+              return 0;
+       }
+    }
+
+    public synchronized TransactionState registerTransaction(final TransactionalTableClient pv_table,
+							     final long transactionID,
+							     final byte[] row) throws IOException {
+        if (LOG.isTraceEnabled()) LOG.trace("Enter registerTransaction, transaction ID: " + transactionID
+             + " row " + Hex.encodeHexString(row));
         boolean register = false;
         short ret = 0;
 
@@ -194,6 +348,9 @@ public class RMInterface {
         }
         HRegionLocation location = ttable.getRegionLocation(row, false /*reload*/);
 
+        if (LOG.isTraceEnabled()) LOG.trace("RMInterface:registerTransaction - retrieved location with startKey="
+              + Hex.encodeHexString(location.getRegionInfo().getStartKey()) + ", endKey="
+              + Hex.encodeHexString(location.getRegionInfo().getEndKey()) + " row= " + Hex.encodeHexString(row));
         TransactionRegionLocation trLocation = new TransactionRegionLocation(location.getRegionInfo(),
                                                                              location.getServerName());
         if (LOG.isTraceEnabled()) LOG.trace("RMInterface:registerTransaction, created TransactionRegionLocation [" + trLocation.getRegionInfo().getRegionNameAsString() + "], endKey: "
@@ -202,8 +359,10 @@ public class RMInterface {
         // if this region hasn't been registered as participating in the transaction, we need to register it
         if (ts.addRegion(trLocation)) {
           register = true;
-          if (LOG.isTraceEnabled()) LOG.trace("RMInterface:registerTransaction, added TransactionRegionLocation [" + trLocation.getRegionInfo().getRegionNameAsString() +  "\nEncodedName: [" + trLocation.getRegionInfo().getEncodedName() + "], endKey: "
-                  + Hex.encodeHexString(trLocation.getRegionInfo().getEndKey()) + " to transaction [" + transactionID + "]");
+          if (LOG.isTraceEnabled()) LOG.trace("RMInterface:registerTransaction, added TransactionRegionLocation ["
+                  + trLocation.getRegionInfo().getRegionNameAsString() + "], endKey: "
+                  + Hex.encodeHexString(trLocation.getRegionInfo().getEndKey()) + " to transaction " + transactionID
+                  + " with " + ts.getParticipantCount() + " participants");
         }
 
         // register region with TM.
@@ -225,6 +384,65 @@ public class RMInterface {
         return ts;
     }
 
+    public synchronized TransactionState registerTransaction(final long transactionID,
+							     final byte[] row) throws IOException {
+
+       if (LOG.isTraceEnabled()) LOG.trace("Enter registerTransaction,"
+					    + " transaction ID: " + transactionID);
+
+       TransactionState ts = registerTransaction(ttable, transactionID, row);
+
+       if (LOG.isTraceEnabled()) LOG.trace("Exit registerTransaction, transaction ID: " + transactionID);
+       return ts;
+    }
+
+    public static synchronized TransactionState registerTransaction(final TransactionalTableClient pv_table,
+		     TransactionState ts,
+		     final byte[] row) throws IOException {
+       if (LOG.isTraceEnabled()) LOG.trace("Enter static registerTransaction, trans: " + ts
+              + " row " + Hex.encodeHexString(row));
+       boolean register = false;
+       final long transactionID = ts.getTransactionId();
+       short ret = 0;
+       HRegionLocation location = pv_table.getRegionLocation(row, false /*reload*/);
+
+       if (LOG.isTraceEnabled()) LOG.trace("static RMInterface:registerTransaction - retrieved location with startKey="
+               + Hex.encodeHexString(location.getRegionInfo().getStartKey()) + ", endKey="
+               + Hex.encodeHexString(location.getRegionInfo().getEndKey()) + " row= " + Hex.encodeHexString(row));
+       HRegionInfo hri = location.getRegionInfo();
+       TransactionRegionLocation trLocation = new TransactionRegionLocation(hri, location.getServerName());
+
+       if (LOG.isTraceEnabled()) LOG.trace("static RMInterface:registerTransaction, created TransactionRegionLocation [" + trLocation.getRegionInfo().getRegionNameAsString() + "], endKey: "
+               + Hex.encodeHexString(trLocation.getRegionInfo().getEndKey()) + " and transaction [" + transactionID + "]");
+
+       // if this region hasn't been registered as participating in the transaction, we need to register it
+       if (ts.addRegion(trLocation)) {
+          register = true;
+          if (LOG.isTraceEnabled()) LOG.trace("static RMInterface:registerTransaction, added TransactionRegionLocation ["
+                + trLocation.getRegionInfo().getRegionNameAsString() + "], endKey: "
+                + Hex.encodeHexString(trLocation.getRegionInfo().getEndKey()) + " to transaction " + transactionID
+                + " with " + ts.getParticipantCount() + " participants");
+       }
+
+       // register region with TM.
+       if (register) {
+          ts.registerLocation(trLocation);
+          if (LOG.isTraceEnabled()) LOG.trace("static RMInterface:registerTransaction, called registerLocation TransactionRegionLocation [" + trLocation.getRegionInfo().getRegionNameAsString() +  "\nEncodedName: [" + trLocation.getRegionInfo().getEncodedName() + "], endKey: "
+                   + Hex.encodeHexString(trLocation.getRegionInfo().getEndKey()) + " to transaction [" + transactionID + "]");
+       }
+       else {
+          if (LOG.isTraceEnabled()) LOG.trace("static RMInterface:registerTransaction did not send registerRegion for transaction " + ts.getTransactionId());
+       }
+
+       if ((ts == null) || (ret != 0)) {
+          LOG.error("static registerTransaction failed, TransactionState is NULL");
+          throw new IOException("static registerTransaction failed with error.");
+       }
+
+       if (LOG.isTraceEnabled()) LOG.trace("Exit static registerTransaction, transaction ID: " + transactionID + ", startId: " + ts.getStartId());
+       return ts;
+    }
+
     public long getTmId() throws IOException {
         if (LOG.isTraceEnabled()) LOG.trace("Enter getTmId");
 
@@ -296,11 +514,11 @@ public class RMInterface {
     }   
 
     static public void clearTransactionStates(final long transactionID) {
-      if (LOG.isTraceEnabled()) LOG.trace("cts1 Enter txid: " + transactionID);
+      if (LOG.isTraceEnabled()) LOG.trace("clearTransactionStates enter txid: " + transactionID);
 
       unregisterTransaction(transactionID);
 
-      if (LOG.isTraceEnabled()) LOG.trace("cts2 txid: " + transactionID);
+      if (LOG.isTraceEnabled()) LOG.trace("clearTransactionStates exit txid: " + transactionID);
     }
 
     static public synchronized void unregisterTransaction(final long transactionID) {
@@ -320,6 +538,16 @@ public class RMInterface {
         if (LOG.isTraceEnabled()) LOG.trace("Exit unregisterTransaction ts: " + ts.getTransactionId());
     }
 
+    public synchronized TransactionState getTransactionState(final long transactionID) throws IOException {
+        if (LOG.isTraceEnabled()) LOG.trace("getTransactionState txid: " + transactionID);
+        TransactionState ts = mapTransactionStates.get(transactionID);
+        if (ts == null) {
+            if (LOG.isTraceEnabled()) LOG.trace("TransactionState for txid: " + transactionID + " not found; throwing IOException");
+            throw new IOException("TransactionState for txid: " + transactionID + " not found" );
+        }
+        if (LOG.isTraceEnabled()) LOG.trace("EXIT getTransactionState");
+        return ts;
+    }
     public synchronized Result get(final long transactionID, final Get get) throws IOException {
         if (LOG.isTraceEnabled()) LOG.trace("get txid: " + transactionID);
         TransactionState ts = registerTransaction(transactionID, get.getRow());
@@ -336,8 +564,9 @@ public class RMInterface {
 
     public synchronized void deleteRegionTx(final Delete delete, final boolean autoCommit) throws IOException {
         long tid = getTmId();
-        if (LOG.isTraceEnabled()) LOG.trace("deleteRegionTx tid: " + tid);
+        if (LOG.isTraceEnabled()) LOG.trace("deleteRegionTx tid: " + tid  + " autoCommit " + autoCommit);
         ttable.deleteRegionTx(tid, delete, autoCommit);
+        if (LOG.isTraceEnabled()) LOG.trace("deleteRegionTx EXIT tid: " + tid);
     }
 
     public synchronized void delete(final long transactionID, final List<Delete> deletes) throws IOException {
@@ -354,7 +583,12 @@ public class RMInterface {
     }
 
     public synchronized ResultScanner getScanner(final long transactionID, final Scan scan) throws IOException {
-        if (LOG.isTraceEnabled()) LOG.trace("getScanner txid: " + transactionID);
+        if (LOG.isTraceEnabled()) LOG.trace("getScanner txid: " + transactionID
+           + " scan startRow=" + (Bytes.equals(scan.getStartRow(), HConstants.EMPTY_START_ROW) ?
+                   "INFINITE" : Hex.encodeHexString(scan.getStartRow())) + ", endRow="
+           + (Bytes.equals(scan.getStopRow(), HConstants.EMPTY_END_ROW) ?
+                   "INFINITE" : Hex.encodeHexString(scan.getStopRow())));
+
         TransactionState ts = registerTransaction(transactionID, scan.getStartRow());
         ResultScanner res = ttable.getScanner(ts, scan);
         if (LOG.isTraceEnabled()) LOG.trace("EXIT getScanner");
@@ -362,10 +596,11 @@ public class RMInterface {
     }
 
     public synchronized void putRegionTx(final Put put, final boolean autoCommit) throws IOException {
-        long tsId = getTmId();
-        if (LOG.isTraceEnabled()) LOG.trace("Enter putRegionTx, autoCommit: " + autoCommit + ", tsId " + tsId);
-        ttable.putRegionTx(tsId, put, autoCommit);
-        if (LOG.isTraceEnabled()) LOG.trace("Exit putRegionTx tsId: " + tsId);
+        long tid = getTmId();
+        if (LOG.isTraceEnabled()) LOG.trace("Enter putRegionTx, autoCommit: " + autoCommit
+               + ", tid " + tid);
+        ttable.putRegionTx(tid, put, autoCommit);
+        if (LOG.isTraceEnabled()) LOG.trace("putRegionTx Exit tid: " + tid);
     }
 
     public synchronized void put(final long transactionID, final Put put) throws IOException {
@@ -385,11 +620,15 @@ public class RMInterface {
            ts = mapTransactionStates.get(transactionID);
         }
         ttable.put(ts, puts);
-         if (LOG.isTraceEnabled()) LOG.trace("Exit put (list of puts) txid: " + transactionID);
+        if (LOG.isTraceEnabled()) LOG.trace("Exit put (list of puts) txid: " + transactionID);
     }
 
-    public synchronized boolean checkAndPut(final long transactionID, byte[] row, byte[] family, byte[] qualifier,
-                       byte[] value, Put put) throws IOException {
+    public synchronized boolean checkAndPut(final long transactionID,
+                                            final byte[] row,
+                                            final byte[] family,
+                                            final byte[] qualifier,
+                                            final byte[] value,
+                                            final Put put) throws IOException {
 
         if (LOG.isTraceEnabled()) LOG.trace("Enter checkAndPut txid: " + transactionID);
         TransactionState ts = registerTransaction(transactionID, row);
@@ -399,15 +638,19 @@ public class RMInterface {
     public synchronized boolean checkAndPutRegionTx(byte[] row, byte[] family,
     		byte[] qualifier, byte[] value, Put put, final boolean autoCommit) throws IOException {
 
-        long tsId = getTmId();
-        if (LOG.isTraceEnabled()) LOG.trace("Enter checkAndPutRegionTx tsId: " + tsId
-        		           + ": autoCommit " + autoCommit );
-        return ttable.checkAndPutRegionTx(tsId, row, family, qualifier, value,
-        		                          put, autoCommit);
+        long tid = getTmId();
+        if (LOG.isTraceEnabled()) LOG.trace("checkAndPutRegionTx, autoCommit: " + autoCommit
+                           + ", tid " + tid);
+        return ttable.checkAndPutRegionTx(tid, row, family, qualifier, value,
+                   put, autoCommit);
     }
 
-    public synchronized boolean checkAndDelete(final long transactionID, byte[] row, byte[] family, byte[] qualifier,
-                       byte[] value, Delete delete) throws IOException {
+    public synchronized boolean checkAndDelete(final long transactionID,
+                                               final byte[] row,
+                                               final byte[] family,
+                                               final byte[] qualifier,
+                                               final byte[] value,
+                                               final Delete delete) throws IOException {
 
         if (LOG.isTraceEnabled()) LOG.trace("Enter checkAndDelete txid: " + transactionID);
         TransactionState ts = registerTransaction(transactionID, row);
@@ -417,7 +660,8 @@ public class RMInterface {
     public synchronized boolean checkAndDeleteRegionTx(byte[] row, byte[] family, byte[] qualifier,
             byte[] value, Delete delete, final boolean autoCommit) throws IOException {
        long tid = getTmId();
-       if (LOG.isTraceEnabled()) LOG.trace("Enter checkAndDeleteRegionTx tid: " + tid);
+       if (LOG.isTraceEnabled()) LOG.trace("checkAndDeleteRegionTx, autoCommit: " + autoCommit
+               + ", tid " + tid);
        return ttable.checkAndDeleteRegionTx(tid, row, family, qualifier, value, delete, autoCommit);
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-trafodion/blob/91794b53/core/sqf/src/seatrans/hbase-trx/src/main/java/org/apache/hadoop/hbase/client/transactional/TmDDL.java
----------------------------------------------------------------------
diff --git a/core/sqf/src/seatrans/hbase-trx/src/main/java/org/apache/hadoop/hbase/client/transactional/TmDDL.java b/core/sqf/src/seatrans/hbase-trx/src/main/java/org/apache/hadoop/hbase/client/transactional/TmDDL.java
index af4c2e5..43444ce 100644
--- a/core/sqf/src/seatrans/hbase-trx/src/main/java/org/apache/hadoop/hbase/client/transactional/TmDDL.java
+++ b/core/sqf/src/seatrans/hbase-trx/src/main/java/org/apache/hadoop/hbase/client/transactional/TmDDL.java
@@ -80,11 +80,29 @@ public class TmDDL {
 
       boolean tDDLTableExists = admin.tableExists(tablename);
 
-      if (tDDLTableExists==false && dtmid ==0) {
-            HTableDescriptor desc = new HTableDescriptor(tablename);
-            desc.addFamily(new HColumnDescriptor(TDDL_FAMILY));
-            admin.createTable(desc);
-      }
+      boolean loopExit = false;
+      do
+      {
+        try {
+           if (tDDLTableExists==false) {
+              if (dtmid == 0){
+                 HTableDescriptor desc = new HTableDescriptor(tablename);
+                 desc.addFamily(new HColumnDescriptor(TDDL_FAMILY));
+                 admin.createTable(desc);
+              }
+              else {
+                 Thread.sleep(3000);  // Sleep 3 seconds to allow table creation
+              }
+           }
+           loopExit = true;
+        }
+        catch (InterruptedException ie) {}
+        catch (IOException e) {
+           loopExit = true;
+           LOG.error("Exception at the time of creating TmDDL ", e);
+           throw e;
+        }
+      } while (loopExit == false);
 
       tablePutLock = new Object();
 

http://git-wip-us.apache.org/repos/asf/incubator-trafodion/blob/91794b53/core/sqf/src/seatrans/hbase-trx/src/main/java/org/apache/hadoop/hbase/client/transactional/TransState.java
----------------------------------------------------------------------
diff --git a/core/sqf/src/seatrans/hbase-trx/src/main/java/org/apache/hadoop/hbase/client/transactional/TransState.java b/core/sqf/src/seatrans/hbase-trx/src/main/java/org/apache/hadoop/hbase/client/transactional/TransState.java
index e8a6df2..c901188 100644
--- a/core/sqf/src/seatrans/hbase-trx/src/main/java/org/apache/hadoop/hbase/client/transactional/TransState.java
+++ b/core/sqf/src/seatrans/hbase-trx/src/main/java/org/apache/hadoop/hbase/client/transactional/TransState.java
@@ -45,7 +45,11 @@ public enum TransState {
 		STATE_FORGOTTEN_HEUR(15), //S5 - HEURISTIC - Waiting Superior TM xa_forget request
 		STATE_ABORTING_PART2(16), // Internal State
 		STATE_TERMINATING(17),
-		STATE_LAST(17),
+		STATE_FORGOTTEN_COMMITTED(18), //N/A
+		STATE_FORGOTTEN_ABORT(19), //N/A
+		STATE_RECOVERY_COMMITTED(20), //N/A
+		STATE_RECOVERY_ABORT(21), //N/A
+		STATE_LAST(21),
 		STATE_BAD(-1);
     private Integer value;
     

http://git-wip-us.apache.org/repos/asf/incubator-trafodion/blob/91794b53/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 04b6360..6c51568 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
@@ -185,6 +185,8 @@ public class TransactionManager {
 
   private IdTm idServer;
   private static final int ID_TM_SERVER_TIMEOUT = 1000;
+  private static final int ABORT_EXCEPTION_DELAY = 30000;
+  private static final int ABORT_EXCEPTION_RETIRES = 30;
 
   private Map<String,Long> batchRSMetrics = new ConcurrentHashMap<String, Long>();
   private long regions = 0;
@@ -260,7 +262,7 @@ public class TransactionManager {
                throws IOException {
         transactionState = txState;
         this.location = location;
-        table = new HTable(location.getRegionInfo().getTable(), connection, cp_tpe);
+        table = new HTable(location.getRegionInfo().getTable(), (Connection) connection, cp_tpe);
         startKey = location.getRegionInfo().getStartKey();
         endKey_orig = location.getRegionInfo().getEndKey();
         if(endKey_orig == null || endKey_orig == HConstants.EMPTY_END_ROW)
@@ -294,6 +296,7 @@ public class TransactionManager {
           try {
 
             if (LOG.isDebugEnabled()) LOG.debug("doCommitX -- ENTRY txid: " + transactionId
+                    + " commitId " + commitId
                     + " participantNum " + participantNum
                     + " ignoreUnknownTransaction: " + ignoreUnknownTransaction);
             Batch.Call<TrxRegionService, CommitResponse> callable =
@@ -306,6 +309,7 @@ public class TransactionManager {
                     public CommitResponse call(TrxRegionService instance) throws IOException {
                       org.apache.hadoop.hbase.coprocessor.transactional.generated.TrxRegionProtos.CommitRequest.Builder builder = CommitRequest.newBuilder();
                       builder.setTransactionId(transactionId);
+                      builder.setCommitId(commitId);
                       builder.setParticipantNum(participantNum);
                       builder.setRegionName(ByteString.copyFromUtf8(Bytes.toString(regionName))); //ByteString.copyFromUtf8(Bytes.toString(regionName)));
                       builder.setIgnoreUnknownTransactionException(ignoreUnknownTransaction);
@@ -317,9 +321,16 @@ public class TransactionManager {
 
                Map<byte[], CommitResponse> result = null;
                try {
-                 if (LOG.isTraceEnabled()) LOG.trace("doCommitX -- before coprocessorService txid: " + transactionId +
-                        " ignoreUnknownTransaction: " + ignoreUnknownTransaction + " table: " +
-                        table.toString() + " startKey: " + new String(startKey, "UTF-8") + " endKey: " + new String(endKey, "UTF-8"));
+                 if (ignoreUnknownTransaction){
+                    if(LOG.isDebugEnabled())
+                           LOG.debug("doCommitX -- Recovery Redrive before coprocessorService txid: " + transactionId +
+                        " ignoreUnknownTransaction: " + ignoreUnknownTransaction + " table: " + table.toString()
+                        + " startKey " + ((startKey != null) ?
+                                (Bytes.equals(startKey, HConstants.EMPTY_START_ROW) ? "INFINITE" : Hex.encodeHexString(startKey)) : "NULL")
+                        + " endKey " +  ((endKey != null) ?
+                                (Bytes.equals(endKey, HConstants.EMPTY_END_ROW) ? "INFINITE" : Hex.encodeHexString(endKey)) : "NULL")
+                        + "Region :" + regionName.toString());
+                 }
                  result = table.coprocessorService(TrxRegionService.class, startKey, endKey, callable);
                } catch (ServiceException se) {
                   String msg = new String ("ERROR occurred while calling coprocessor service in doCommitX for transaction "
@@ -384,17 +395,19 @@ public class TransactionManager {
                       else if (exceptionString.contains("DUPLICATE")) {
                          throw new UnknownTransactionException(cresponse.getException());
                       }
+                      else if (exceptionString.contains("NonPendingTransactionException")) {
+                          throw new NonPendingTransactionException(cresponse.getException());
+                      }
                       else if (exceptionString.contains("org.apache.hadoop.hbase.exceptions.FailedSanityCheckException")) {
                          throw new org.apache.hadoop.hbase.exceptions.FailedSanityCheckException(cresponse.getException());
                       }
                       else {
                         throw new RetryTransactionException(cresponse.getException());
                       }
+                    }
                   }
+                  retry = false;
                }
-               retry = false;
-             }
-
           }
           catch (UnknownTransactionException ute) {
              String errMsg = new String("doCommitX UnknownTransactionException for transaction "
@@ -405,10 +418,19 @@ public class TransactionManager {
              }
              else {
                 LOG.error(errMsg, ute);
-                transactionState.logUteDetails();
+                transactionState.logExceptionDetails(true);
                 transactionState.requestPendingCountDec(null);
+//                throw ute;
              }
           }
+          catch (NonPendingTransactionException npte) {
+              String errMsg = new String("doCommitX NonPendingTransactionException for transaction "
+                               + transactionId + " participantNum " + participantNum + " Location " + location.getRegionInfo().getRegionNameAsString());
+              LOG.error(errMsg, npte);
+              transactionState.logExceptionDetails(false);
+              transactionState.requestPendingCountDec(null);
+//              throw npte;
+          }
           catch (org.apache.hadoop.hbase.exceptions.FailedSanityCheckException fsce) {
               LOG.error("doCommitX FailedSanityCheckException for transaction " + transactionId + " participantNum " + participantNum + 
                  " Location " + location.getRegionInfo().getRegionNameAsString(), fsce);
@@ -458,8 +480,11 @@ public class TransactionManager {
           refresh = false;
           try {
 
-            if (LOG.isTraceEnabled()) LOG.trace("doCommitX -- ENTRY txid: " + transactionId +
-                                                  " ignoreUnknownTransaction: " + ignoreUnknownTransaction);
+            if (LOG.isTraceEnabled()) LOG.trace("doCommitX -- ENTRY txid: " + transactionId
+                    + " participantNum " + participantNum
+                    + " commitId " + commitId
+                    + " ignoreUnknownTransaction: " + ignoreUnknownTransaction);
+
             Batch.Call<SsccRegionService, SsccCommitResponse> callable =
                new Batch.Call<SsccRegionService, SsccCommitResponse>() {
                  ServerRpcController controller = new ServerRpcController();
@@ -482,8 +507,12 @@ public class TransactionManager {
                Map<byte[], SsccCommitResponse> result = null;
                try {
                  if (LOG.isTraceEnabled()) LOG.trace("doCommitX -- before coprocessorService txid: " + transactionId +
-                        " ignoreUnknownTransaction: " + ignoreUnknownTransaction + " table: " +
-                        table.toString() + " startKey: " + new String(startKey, "UTF-8") + " endKey: " + new String(endKey, "UTF-8"));
+                        " ignoreUnknownTransaction: " + ignoreUnknownTransaction + " table: " + table.toString()
+                        + " startKey " + ((startKey != null) ?
+                                (Bytes.equals(startKey, HConstants.EMPTY_START_ROW) ? "INFINITE" : Hex.encodeHexString(startKey)) : "NULL")
+                        + " endKey " +  ((endKey != null) ?
+                                (Bytes.equals(endKey, HConstants.EMPTY_END_ROW) ? "INFINITE" : Hex.encodeHexString(endKey)) : "NULL"));
+
                  result = table.coprocessorService(SsccRegionService.class, startKey, endKey, callable);
                } catch (ServiceException se) {
                   String msg = new String("ERROR occurred while calling coprocessor service in doCommitX for transaction " 
@@ -512,6 +541,14 @@ public class TransactionManager {
                       if (exceptionString.contains("UnknownTransactionException")) {
                           throw new UnknownTransactionException(cresponse.getException());
                       }
+                      else if (exceptionString.contains("DUPLICATE")) {
+                          LOG.error("doCommitX, coprocessor UnknownTransactionException: " + cresponse.getException());
+                          throw new UnknownTransactionException(cresponse.getException());
+                      }
+                      else if (exceptionString.contains("NonPendingTransactionException")) {
+                          LOG.error("doCommitX, coprocessor NonPendingTransactionException: " + cresponse.getException());
+                          throw new NonPendingTransactionException(cresponse.getException());
+                       }
                       else {
                         throw new RetryTransactionException(cresponse.getException());
                       }
@@ -529,10 +566,19 @@ public class TransactionManager {
              }
              else {
                 LOG.error(errMsg, ute);
-                transactionState.logUteDetails();
+                transactionState.logExceptionDetails(true);
                 transactionState.requestPendingCountDec(null);
+//                throw ute;
              }
           }
+          catch (NonPendingTransactionException npte) {
+              String errMsg = new String("doCommitX, NonPendingTransactionException  for transaction "
+                 + transactionId + " participantNum " + participantNum + " Location " + location.getRegionInfo().getRegionNameAsString());
+              LOG.error(errMsg, npte);
+              transactionState.logExceptionDetails(false);
+              transactionState.requestPendingCountDec(null);
+//              throw npte;
+          }
           catch (RetryTransactionException rte) {
              if (retryCount == RETRY_ATTEMPTS) {
                 String errMsg = new String("Exceeded " + retryCount + " retry attempts in doCommitX for transaction "
@@ -644,7 +690,7 @@ public class TransactionManager {
                 retry = true;
              }
              else if(result.size() == 1){
-               if(LOG.isInfoEnabled()) LOG.info("doPrepareX(MVCC), received result size: " + result.size() + " for transaction "
+               if(LOG.isDebugEnabled()) LOG.debug("doPrepareX(MVCC), received result size: " + result.size() + " for transaction "
                    + transactionId + " participantNum " + participantNum + " Location " + location.getRegionInfo().getRegionNameAsString());
                 // size is 1
                 for (CommitRequestResponse cresponse : result.values()){
@@ -799,7 +845,7 @@ public class TransactionManager {
                 throw cue;
              }
              if(result.size() != 1)  {
-                LOG.error("doPrepareX, received incorrect result size: " + result.size());
+                LOG.error("doPrepareX (SSCC), received incorrect result size: " + result.size());
                 refresh = true;
                 retry = true;
              }
@@ -854,10 +900,11 @@ 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 " 
+       String errMsg = new String(" returned from commit request in doPrepareX for transaction "
                      + transactionId + " participantNum " + participantNum + " Location " + location.getRegionInfo().getRegionNameAsString());
        boolean canCommit = true;
        boolean readOnly = false;
+       String tmpStatus = null;
 
        switch (commitStatus) {
           case TransactionalReturn.COMMIT_OK:
@@ -868,9 +915,20 @@ public class TransactionManager {
             readOnly = true;
             break;
           case TransactionalReturn.COMMIT_UNSUCCESSFUL_FROM_COPROCESSOR:
+             tmpStatus = new String("COMMIT_UNSUCCESSFUL_FROM_COPROCESSOR");
+             if (LOG.isInfoEnabled()) LOG.info(tmpStatus + errMsg);
+             canCommit = false;
+             transactionState.addRegionToIgnore(location); // No need to re-abort.
+             break;
           case TransactionalReturn.COMMIT_CONFLICT:
+             tmpStatus = new String("COMMIT_CONFLICT");
+             if (LOG.isInfoEnabled()) LOG.info(tmpStatus + errMsg);
+             canCommit = false;
+             transactionState.addRegionToIgnore(location); // No need to re-abort.
+             break;
           case TransactionalReturn.COMMIT_UNSUCCESSFUL:
-             LOG.error(errMsg);
+             tmpStatus = new String("COMMIT_UNSUCCESSFUL");
+             if (LOG.isInfoEnabled()) LOG.info(tmpStatus + errMsg);
              canCommit = false;
              transactionState.addRegionToIgnore(location); // No need to re-abort.
              break;
@@ -909,7 +967,7 @@ public class TransactionManager {
     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());
+                        + participantNum + " region " + regionName.toString() + " dropTableRecorded " + dropTableRecorded);
         boolean retry = false;
         boolean refresh = false;
         int retryCount = 0;
@@ -935,6 +993,7 @@ public class TransactionManager {
                 builder.setParticipantNum(participantNum);
                 builder.setRegionName(ByteString.copyFromUtf8(Bytes.toString(regionName)));
                 builder.setDropTableRecorded(dropTableRecorded);
+                builder.setIgnoreUnknownTransactionException(true);
                 instance.abortTransaction(controller, builder.build(), rpcCallback);
                 return rpcCallback.get();
               }
@@ -943,8 +1002,12 @@ public class TransactionManager {
             Map<byte[], AbortTransactionResponse> result = null;
               try {
                  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"));
+                        + transactionId + " table: " + table.toString()
+                        + " startKey " + ((startKey != null) ?
+                                (Bytes.equals(startKey, HConstants.EMPTY_START_ROW) ? "INFINITE" : Hex.encodeHexString(startKey)) : "NULL")
+                        + " endKey " +  ((endKey != null) ?
+                                (Bytes.equals(endKey, HConstants.EMPTY_END_ROW) ? "INFINITE" : Hex.encodeHexString(endKey)) : "NULL"));
+
                  result = table.coprocessorService(TrxRegionService.class, startKey, endKey, callable);
               } catch (ServiceException se) {
                   String msg = "ERROR occurred while calling doAbortX coprocessor service";
@@ -967,17 +1030,19 @@ public class TransactionManager {
               }
               else {
                  for (AbortTransactionResponse cresponse : result.values()) {
-                   String exceptionString = cresponse.getException();
-                   if (exceptionString != null) {
-                     if (exceptionString.contains("UnknownTransactionException")) {
-                       throw new UnknownTransactionException(exceptionString);
-                     }
-                     throw new RetryTransactionException(cresponse.getException());
+                   if (cresponse.getHasException()) {
+                      String exceptionString = cresponse.getException();
+                      if (exceptionString.contains("UnknownTransactionException"))
+                         throw new UnknownTransactionException(exceptionString);
+                      else if (exceptionString.contains("NonPendingTransactionException"))
+                         throw new NonPendingTransactionException(exceptionString);
+                      else
+                         throw new RetryTransactionException(cresponse.getException());
                    }
                  }
                  retry = false;
               }
-           }
+          }
           catch (UnknownTransactionException ute) {
              String errMsg = new String("doAbortX UnknownTransactionException for transaction "
                               + transactionId + " participantNum " + participantNum + " Location " + location.getRegionInfo().getRegionNameAsString());
@@ -987,10 +1052,17 @@ public class TransactionManager {
              }
              else {
                 LOG.error(errMsg, ute);
-                transactionState.logUteDetails();
+                transactionState.logExceptionDetails(true);
                 transactionState.requestPendingCountDec(null);
              }
           }
+          catch (NonPendingTransactionException npte) {
+             String errMsg = new String("doAbortX NonPendingTransactionException for transaction "
+                                 + transactionId + " participantNum " + participantNum + " Location " + location.getRegionInfo().getRegionNameAsString());
+             LOG.error(errMsg, npte);
+             transactionState.logExceptionDetails(false);
+             transactionState.requestPendingCountDec(null);
+          }
           catch (RetryTransactionException rte) {
               if (retryCount == RETRY_ATTEMPTS) {
                  String errMsg = new String("Exceeded " + retryCount + " retry attempts in doAbortX for transaction " 
@@ -1003,7 +1075,7 @@ public class TransactionManager {
               else if (rte.toString().contains("Asked to commit a non-pending transaction ")) {
                  String errMsg = new String("doAbortX will not retry transaction" 
                      + transactionId + " participantNum " + participantNum + " Location " + location.getRegionInfo().getRegionNameAsString());
-                 LOG.error(errMsg,rte);
+                 LOG.warn(errMsg,rte);
                  refresh = false;
                  retry = false;
               }
@@ -1029,7 +1101,7 @@ public class TransactionManager {
                  else {
                     LOG.error("doAbortX -- table: " + table.toString() + " is disabled, ignoring table and returning");
                     transactionState.requestPendingCountDec(null);
-                    return 0;                    	
+                    return 0;
                  }
                  if (LOG.isWarnEnabled()) LOG.warn("doAbortX -- setting retry, count: " + retryCount);
                  refresh = false;
@@ -1064,8 +1136,13 @@ public class TransactionManager {
 
             Map<byte[], SsccAbortTransactionResponse> result = null;
               try {
-                  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"));
+                  if (LOG.isTraceEnabled()) LOG.trace("doAbortX -- before coprocessorService txid: " + transactionId
+                          + " table: " + table.toString()
+                          + " startKey " + ((startKey != null) ?
+                                  (Bytes.equals(startKey, HConstants.EMPTY_START_ROW) ? "INFINITE" : Hex.encodeHexString(startKey)) : "NULL")
+                          + " endKey " +  ((endKey != null) ?
+                                  (Bytes.equals(endKey, HConstants.EMPTY_END_ROW) ? "INFINITE" : Hex.encodeHexString(endKey)) : "NULL"));
+
                   result = table.coprocessorService(SsccRegionService.class, startKey, endKey, callable);
               } catch (ServiceException se) {
                   String msg = "ERROR occurred while calling doAbortX coprocessor service";
@@ -1106,7 +1183,7 @@ public class TransactionManager {
              }
              else {
                 LOG.error(errMsg, ute);
-                transactionState.logUteDetails();
+                transactionState.logExceptionDetails(true);
                 transactionState.requestPendingCountDec(null);
              }
           }
@@ -1114,7 +1191,7 @@ public class TransactionManager {
               if (retryCount == RETRY_ATTEMPTS){
                    String errMsg = new String ("Exceeded retry attempts in doAbortX: " + retryCount + " (Not ingoring)");
                    LOG.error(errMsg);
-                   RollbackUnsuccessfulException rue = new RollbackUnsuccessfulException(errMsg, rte);  
+                   RollbackUnsuccessfulException rue = new RollbackUnsuccessfulException(errMsg, rte);
                    transactionState.requestPendingCountDec(rue);
                    throw rue;
               }
@@ -1139,12 +1216,12 @@ public class TransactionManager {
                  else {
                     LOG.error("doAbortX -- table: " + table.toString() + " is disabled, ignoring table and returning");
                     transactionState.requestPendingCountDec(null);
-                    return 0;                    	
+                    return 0;
                  }
                  if (LOG.isWarnEnabled()) LOG.warn("doAbortX -- setting retry, count: " + retryCount);
                  refresh = false;
               }
-              if (retry) 
+              if (retry)
                  retrySleep = retry(retrySleep);
            } while (retry && retryCount++ <= RETRY_ATTEMPTS);
 
@@ -1200,7 +1277,7 @@ public class TransactionManager {
                  transactionState.requestPendingCountDec(cue);
                  throw cue;
             }
-		 if(!retry) {
+		 if (!retry) {
 		      List<String> exceptions = commitMultipleResponse.getExceptionList();
 
 		      checkException(transactionState, locations, exceptions);
@@ -1341,7 +1418,7 @@ 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 " 
+       String errMsg = new String( commitStatus + " returned from commit request in doPrepareX for transaction "
                      + transactionId + " participantNum " + participantNum + " Table " + location.getRegionInfo().getTable().toString());
        switch (commitStatus) {
           case 0:
@@ -1362,12 +1439,13 @@ public class TransactionManager {
                  commitErr = TransactionalReturn.COMMIT_UNSUCCESSFUL;
              canCommit = false;
              readOnly = false;
-             LOG.warn(errMsg);
+             if (LOG.isInfoEnabled()) LOG.info(errMsg);
              transactionState.addRegionToIgnore(locations.get(resultCount)); // No need to re-abort.
              break;
           default:
              CommitUnsuccessfulException cue = new CommitUnsuccessfulException(errMsg);
-             LOG.warn(errMsg, cue);
+             if (LOG.isWarnEnabled())
+                 LOG.warn(errMsg, cue);
              throw cue;
        }
        resultCount++;
@@ -1405,6 +1483,7 @@ public class TransactionManager {
           TrxRegionProtos.AbortTransactionMultipleRequest.Builder builder = AbortTransactionMultipleRequest.newBuilder();
           builder.setTransactionId(transactionId);
           builder.setParticipantNum(participantNum);
+
           for(TransactionRegionLocation location : locations) {
               builder.addRegionName(ByteString.copyFrom(location.getRegionInfo().getRegionName()));
          }
@@ -1500,7 +1579,10 @@ public class TransactionManager {
 
        Map<byte[], PushEpochResponse> result = null;
        if (LOG.isTraceEnabled()) LOG.trace("pushRegionEpochX -- before coprocessorService: startKey: "
-              + new String(startKey, "UTF-8") + " endKey: " + new String(endKey, "UTF-8"));
+               + ((startKey != null) ?
+                       (Bytes.equals(startKey, HConstants.EMPTY_START_ROW) ? "INFINITE" : Hex.encodeHexString(startKey)) : "NULL")
+               + " endKey " +  ((endKey != null) ?
+                       (Bytes.equals(endKey, HConstants.EMPTY_END_ROW) ? "INFINITE" : Hex.encodeHexString(endKey)) : "NULL"));
 
        boolean loopExit = false;
        try {
@@ -1615,7 +1697,7 @@ public class TransactionManager {
 
         threadPool = Executors.newFixedThreadPool(intThreads);
 
-    cp_tpe = Executors.newFixedThreadPool(intCpThreads);
+	cp_tpe = Executors.newFixedThreadPool(intCpThreads);
 
     /* This was in the HTable code - let's use a fixed thread pool instead
     cp_tpe = new ThreadPoolExecutor(1,
@@ -1639,8 +1721,6 @@ public class TransactionManager {
         connection = conn;
     }
 
-
-
     /**
      * Called to start a transaction.
      *
@@ -1665,7 +1745,8 @@ public class TransactionManager {
       long startIdVal = -1;
 
       // Set the startid
-      if (ts.islocalTransaction() && (TRANSACTION_ALGORITHM == AlgorithmType.SSCC)) {
+      if (ts.islocalTransaction() &&
+         ((TRANSACTION_ALGORITHM == AlgorithmType.SSCC))) {
          IdTmId startId;
          try {
             startId = new IdTmId();
@@ -1696,7 +1777,8 @@ public class TransactionManager {
      * @throws CommitUnsuccessfulException
      */
     public int prepareCommit(final TransactionState transactionState) throws CommitUnsuccessfulException, IOException {
-       if (LOG.isTraceEnabled()) LOG.trace("Enter prepareCommit, txid: " + transactionState.getTransactionId());
+       if (LOG.isTraceEnabled()) LOG.trace("Enter prepareCommit, txid: " + transactionState.getTransactionId()
+                          + " with " + transactionState.getParticipatingRegions().size() + " participants");
 
        if (batchRegionServer && (TRANSACTION_ALGORITHM == AlgorithmType.MVCC)) {
          boolean allReadOnly = true;
@@ -1728,9 +1810,10 @@ public class TransactionManager {
                 loopCount++;
                 final int lv_participant = loopCount;
                 compPool.submit(new TransactionManagerCallable(transactionState, entry.getValue().iterator().next(), connection) {
-                    public Integer call() throws CommitUnsuccessfulException, IOException {
-                        return doPrepareX(entry.getValue(), transactionState.getTransactionId(), lv_participant);
-                    }
+
+                   public Integer call() throws CommitUnsuccessfulException, IOException {
+                       return doPrepareX(entry.getValue(), transactionState.getTransactionId(), lv_participant);
+                   }
                 });
             }
 
@@ -2002,7 +2085,6 @@ public class TransactionManager {
       if(LOG.isTraceEnabled()) LOG.trace("retryCommit -- ENTRY -- txid: " + transactionState.getTransactionId());
       synchronized(transactionState.getRetryRegions()) {
           List<TransactionRegionLocation> completedList = new ArrayList<TransactionRegionLocation>();
-          final long commitIdVal = (TRANSACTION_ALGORITHM == AlgorithmType.SSCC) ? transactionState.getCommitId() : -1;
           int loopCount = 0;
           for (TransactionRegionLocation location : transactionState.getRetryRegions()) {
             loopCount++;
@@ -2015,7 +2097,7 @@ public class TransactionManager {
 
                     return doCommitX(location.getRegionInfo().getRegionName(),
                             transactionState.getTransactionId(),
-                            commitIdVal,
+                            transactionState.getCommitId(),
                             participantNum,
                             ignoreUnknownTransaction);
                 }
@@ -2028,7 +2110,7 @@ public class TransactionManager {
     }
 
     public void pushRegionEpoch (HTableDescriptor desc, final TransactionState ts) throws IOException {
-       LOG.info("pushRegionEpoch start; transId: " + ts.getTransactionId());
+       if (LOG.isTraceEnabled()) LOG.trace("pushRegionEpoch start; transId: " + ts.getTransactionId());
 
        TransactionalTable ttable1 = new TransactionalTable(Bytes.toBytes(desc.getNameAsString()), connection);
        long lvTransid = ts.getTransactionId();
@@ -2110,6 +2192,7 @@ public class TransactionManager {
         throws CommitUnsuccessfulException, UnsuccessfulDDLException, IOException {
        if (LOG.isTraceEnabled()) LOG.trace("doCommit [" + transactionState.getTransactionId() +
                       "] ignoreUnknownTransaction not supplied");
+
        doCommit(transactionState, false);
     }
 
@@ -2125,9 +2208,7 @@ public class TransactionManager {
         int loopCount = 0;
         if (batchRegionServer && (TRANSACTION_ALGORITHM == AlgorithmType.MVCC)) {
              if (LOG.isTraceEnabled()) LOG.trace("Committing [" + transactionState.getTransactionId() +
-                      "] ignoreUnknownTransaction: " + ignoreUnknownTransaction);
-             // Set the commitId
-             transactionState.setCommitId(-1); // Dummy for MVCC
+                      "] with commitId: " + transactionState.getCommitId() + ", ignoreUnknownTransaction: " + ignoreUnknownTransaction);
 
              ServerName servername;
              List<TransactionRegionLocation> regionList;
@@ -2154,11 +2235,15 @@ public class TransactionManager {
                  final int lv_participant = loopCount;
 
                  threadPool.submit(new TransactionManagerCallable(transactionState, entry.getValue().iterator().next(), connection) {
-                     public Integer call() throws CommitUnsuccessfulException, IOException {
-                        if (LOG.isTraceEnabled()) LOG.trace("before doCommit() [" + transactionState.getTransactionId() + "]" +
-                                                            " ignoreUnknownTransaction: " + ignoreUnknownTransaction);
-                        return doCommitX(entry.getValue(), transactionState.getTransactionId(),
-                                      transactionState.getCommitId(), lv_participant, ignoreUnknownTransaction);
+                    public Integer call() throws CommitUnsuccessfulException, IOException {
+                        if (LOG.isTraceEnabled()) LOG.trace("before doCommit() ["
+                                        + transactionState.getTransactionId()
+                                        + "] ignoreUnknownTransaction: " + ignoreUnknownTransaction);
+                        return doCommitX(entry.getValue(),
+                                           transactionState.getTransactionId(),
+                                           transactionState.getCommitId(),
+                                           lv_participant,
+                                           ignoreUnknownTransaction);
                      }
                   });
              }
@@ -2171,7 +2256,17 @@ public class TransactionManager {
         if (LOG.isTraceEnabled()) LOG.trace("Committing [" + transactionState.getTransactionId() +
                       "] ignoreUnknownTransactionn: " + ignoreUnknownTransaction);
 
-        if (LOG.isTraceEnabled()) LOG.trace("sending commits for ts: " + transactionState);
+        if (LOG.isDebugEnabled()) {
+           LOG.debug("sending commits for ts: " + transactionState + ", with commitId: "
+                    + transactionState.getCommitId() + " and " + transactionState.getParticipatingRegions().size() + " participants" );
+           for (TransactionRegionLocation location : transactionState.getParticipatingRegions()) {
+               LOG.debug("TransactionRegionLocation Name: "
+                    + location.getRegionInfo().getRegionNameAsString()
+                    + "\n Start key    : " + Hex.encodeHexString(location.getRegionInfo().getStartKey())
+                    + "\n End key    : " + Hex.encodeHexString(location.getRegionInfo().getEndKey()));
+           }
+        }
+
            int participants = transactionState.participatingRegions.size() - transactionState.regionsToIgnore.size();
            if (LOG.isTraceEnabled()) LOG.trace("Committing [" + transactionState.getTransactionId() + "] with " + participants + " participants" );
            // (Asynchronously send commit
@@ -2190,7 +2285,8 @@ public class TransactionManager {
               threadPool.submit(new TransactionManagerCallable(transactionState, location, connection) {
                  public Integer call() throws CommitUnsuccessfulException, IOException {
                     if (LOG.isDebugEnabled()) LOG.debug("before doCommit() [" + transactionState.getTransactionId()
-                              + "] participantNum " + participantNum + " ignoreUnknownTransaction: " + ignoreUnknownTransaction);
+                              + "], commitId : " + transactionState.getCommitId() + " participantNum " + participantNum
+                              + " ignoreUnknownTransaction: " + ignoreUnknownTransaction);
                     return doCommitX(regionName,
                             transactionState.getTransactionId(),
                             transactionState.getCommitId(),
@@ -2253,7 +2349,7 @@ public class TransactionManager {
                 retry = false;
             }
             catch(IOException e){
-                LOG.info("Exception in doCommitDDL, Step: getRow. txID: " + transactionState.getTransactionId() + "Exception: " , e);
+                if(LOG.isInfoEnabled()) LOG.info("Exception in doCommitDDL, Step: getRow. txID: " + transactionState.getTransactionId() + "Exception: " , e);
 
                 if (retryCount == RETRY_ATTEMPTS)
                 {
@@ -2288,11 +2384,11 @@ public class TransactionManager {
                         //Check for TableNotFoundException, if that is the case, no further
                         //processing needed. This is not an error. Possible we are retrying the entire set of DDL changes
                         //because tis transaction was pinned for some reason.
-			LOG.info(" Exception for " + tblName + ", but continuing txID: " + transactionState.getTransactionId(), t); 
+                        if(LOG.isInfoEnabled()) LOG.info(" Exception for " + tblName + ", but continuing txID: " + transactionState.getTransactionId(), t);
                         retry = false;
                     }
                     catch (IOException e) {
-                        LOG.info("Fatal exception in doCommitDDL, Step : DeleteTable: TxID:" + transactionState.getTransactionId() + "Exception: " , e);
+                        if(LOG.isInfoEnabled()) LOG.info("Fatal exception in doCommitDDL, Step : DeleteTable: TxID:" + transactionState.getTransactionId() + "Exception: " , e);
 
                         if(retryCount == RETRY_ATTEMPTS)
                         {
@@ -2321,7 +2417,7 @@ public class TransactionManager {
         }
         catch (IOException e)
         {
-            LOG.info("Fatal Exception in doCommitDDL, Step: deleteRow. txID: " + transactionState.getTransactionId() + "Exception: " , e);
+            if(LOG.isInfoEnabled()) LOG.info("Fatal Exception in doCommitDDL, Step: deleteRow. txID: " + transactionState.getTransactionId() + "Exception: " , e);
 
             if(retryCount == RETRY_ATTEMPTS)
             {
@@ -2348,51 +2444,52 @@ public class TransactionManager {
      * @throws IOException
      */
     public void abort(final TransactionState transactionState) throws IOException, UnsuccessfulDDLException {
+
       if(LOG.isTraceEnabled()) LOG.trace("Abort -- ENTRY txID: " + transactionState.getTransactionId());
-        int loopCount = 0;
+      int loopCount = 0;
 
-          transactionState.setStatus(TransState.STATE_ABORTED);
-          // (Asynchronously send aborts
-          if (batchRegionServer && (TRANSACTION_ALGORITHM == AlgorithmType.MVCC)) {
+      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;
-            }
-            servername = location.getServerName();
+            for (TransactionRegionLocation location : transactionState.getParticipatingRegions()) {
+               if (transactionState.getRegionsToIgnore().contains(location)) {
+                  continue;
+               }
+               servername = location.getServerName();
 
-            if(!locations.containsKey(servername)) {
-                regionList = new ArrayList<TransactionRegionLocation>();
-                locations.put(servername, regionList);
-            }
-            else {
-                regionList = locations.get(servername);
+                if(!locations.containsKey(servername)) {
+                    regionList = new ArrayList<TransactionRegionLocation>();
+                    locations.put(servername, regionList);
+                }
+                else {
+                    regionList = locations.get(servername);
+                }
+                regionList.add(location);
             }
-            regionList.add(location);
-        }
-        for(final Map.Entry<ServerName, List<TransactionRegionLocation>> entry : locations.entrySet()) {
-            loopCount++;
-            final int participantNum = loopCount;
+            for(final Map.Entry<ServerName, List<TransactionRegionLocation>> entry : locations.entrySet()) {
+                loopCount++;
+                final int lv_participant = loopCount;
 
             threadPool.submit(new TransactionManagerCallable(transactionState, entry.getValue().iterator().next(), connection) {
-                public Integer call() throws IOException {
-                   if (LOG.isTraceEnabled()) LOG.trace("before abort() [" + transactionState.getTransactionId() + "]");
+                    public Integer call() throws IOException {
+                       if (LOG.isTraceEnabled()) LOG.trace("before abort() [" + transactionState.getTransactionId() + "]");
 
-                   return doAbortX(entry.getValue(), transactionState.getTransactionId(), participantNum);
-                }
-             });
-        }
-        transactionState.completeSendInvoke(loopCount);
-    }
-    else {
-      loopCount = 0;
-      for (TransactionRegionLocation location : transactionState.getParticipatingRegions()) {
-          if (transactionState.getRegionsToIgnore().contains(location)) {
-              continue;
-          }
+                       return doAbortX(entry.getValue(), transactionState.getTransactionId(), lv_participant);
+                    }
+                 });
+            }
+            transactionState.completeSendInvoke(loopCount);
+      }
+      else {
+        loopCount = 0;
+        for (TransactionRegionLocation location : transactionState.getParticipatingRegions()) {
+            if (transactionState.getRegionsToIgnore().contains(location)) {
+               continue;
+            }
             loopCount++;
             final int participantNum = loopCount;
             final byte[] regionName = location.getRegionInfo().getRegionName();
@@ -2406,10 +2503,9 @@ public class TransactionManager {
               }
             });
         }
-
         // all requests sent at this point, can record the count
         transactionState.completeSendInvoke(loopCount);
-    }
+      }
          
        IOException savedException = null;
 
@@ -2460,7 +2556,7 @@ public class TransactionManager {
                 retry = false;
             }
             catch (IOException e){
-                LOG.info("Fatal Exception in abortDDL, Step: getRow. txID: " + transactionState.getTransactionId() + "Exception: " , e);
+                if(LOG.isInfoEnabled()) LOG.info("Fatal Exception in abortDDL, Step: getRow. txID: " + transactionState.getTransactionId() + "Exception: " , e);
 
                 if(retryCount == RETRY_ATTEMPTS)
                 {
@@ -2496,7 +2592,7 @@ public class TransactionManager {
                         retry = false;
                     }
                     catch (IOException e){
-                        LOG.info("Fatal exception in abortDDL, Step : truncateTable: TxID:" + transactionState.getTransactionId() + "Exception: ", e);
+                        if(LOG.isInfoEnabled()) LOG.info("Fatal exception in abortDDL, Step : truncateTable: TxID:" + transactionState.getTransactionId() + "Exception: ", e);
 
                         if(retryCount == RETRY_ATTEMPTS)
                         {
@@ -2532,11 +2628,11 @@ public class TransactionManager {
                         //Check for TableNotFoundException, if that is the case, no further
                         //processing needed. This is not an error. Possible we are retrying the entire set of DDL changes
                         //because this transaction is being redriven for some reason.
-			LOG.info(" Exception for " + tblName + ", but continuing txID: " + transactionState.getTransactionId(), t); 
+                        if(LOG.isInfoEnabled()) LOG.info(" Exception for " + tblName + ", but continuing txID: " + transactionState.getTransactionId(), t);
                         retry = false;
                     }
                     catch (IOException e) {
-                        LOG.info("Fatal exception in abortDDL, Step : DeleteTable: TxID:" + transactionState.getTransactionId() + "Exception: " , e);
+                        if(LOG.isInfoEnabled()) LOG.info("Fatal exception in abortDDL, Step : DeleteTable: TxID:" + transactionState.getTransactionId() + "Exception: " , e);
 
                         if(retryCount == RETRY_ATTEMPTS)
                         {
@@ -2575,11 +2671,11 @@ public class TransactionManager {
                     catch (TableNotFoundException t) {
                         //Check for TableNotFoundException, if that is the case, no further
                         //processing needed. This would happen if the table is created and dropped in the same transaction 
-                        LOG.info(" Exception for " + tblName + ", but continuing txID: " + transactionState.getTransactionId(), t); 
+                        if(LOG.isInfoEnabled()) LOG.info(" Exception for " + tblName + ", but continuing txID: " + transactionState.getTransactionId(), t);
                         retry = false;
                     }
                     catch (IOException e) {
-                        LOG.info("Fatal exception in abortDDL, Step : enableTable: TxID:" + transactionState.getTransactionId() + "Exception: ", e);
+                        if(LOG.isInfoEnabled()) LOG.info("Fatal exception in abortDDL, Step : enableTable: TxID:" + transactionState.getTransactionId() + "Exception: ", e);
                         if(retryCount == RETRY_ATTEMPTS)
                         {
                             LOG.error("Fatal Exception in abortDDL, Step: enableTable. Raising UnsuccessfulDDLException TxID:" + transactionState.getTransactionId() );
@@ -2609,7 +2705,7 @@ public class TransactionManager {
             }
             catch (IOException e)
             {
-                LOG.info("Fatal Exception in abortDDL, Step: deleteRow. txID: " + transactionState.getTransactionId() + "Exception: ", e);
+                if(LOG.isInfoEnabled()) LOG.info("Fatal Exception in abortDDL, Step: deleteRow. txID: " + transactionState.getTransactionId() + "Exception: ", e);
                 if(retryCount == RETRY_ATTEMPTS)
                 {
                     LOG.error("Fatal Exception in abortDDL, Step: deleteRow. Raising UnsuccessfulDDLException. txID: " + transactionState.getTransactionId());
@@ -2631,7 +2727,10 @@ public class TransactionManager {
     }
 
     public void registerRegion(final TransactionState transactionState, TransactionRegionLocation location)throws IOException{
-        if (LOG.isTraceEnabled()) LOG.trace("registerRegion ENTRY, transactioState:" + transactionState);
+
+        if (LOG.isTraceEnabled()) LOG.trace("registerRegion ENTRY, transactioState:" + transactionState
+                     + " location: " + location);
+
         if(transactionState.addRegion(location)){
            if (LOG.isTraceEnabled()) LOG.trace("registerRegion -- added region: " + location.getRegionInfo().getRegionNameAsString() + " with endKey: "
                       + Hex.encodeHexString(location.getRegionInfo().getEndKey()) + " to tx " + transactionState.getTransactionId());
@@ -2659,19 +2758,37 @@ public class TransactionManager {
             else {
               admin.createTable(desc);
             }
+            // make sure the table is enabled
+            boolean keepPolling = true;
+            TableName tableName = desc.getTableName();
+            while (keepPolling) {
+
+               try {
+                 Thread.sleep(1000); // sleep one second or until interrupted
+               }
+               catch (InterruptedException e) {
+                  // ignore the interruption and keep going
+               }
+
+               // To be changed in 2.0 for truncate table
+               if (admin.isTableEnabled(tableName))
+                  keepPolling = false;
+            }
+
          } finally {
             admin.close();
          }
-            // Set transaction state object as participating in ddl transaction
-            transactionState.setDDLTx(true);
 
-            //record this create in TmDDL.
-            tmDDL.putRow( transactionState.getTransactionId(), "CREATE", desc.getNameAsString());
+        // Set transaction state object as participating in ddl transaction
+        transactionState.setDDLTx(true);
+
+        //record this create in TmDDL.
+        tmDDL.putRow( transactionState.getTransactionId(), "CREATE", desc.getNameAsString());
 
-            if (LOG.isTraceEnabled()) LOG.trace("createTable: epoch pushed into regions for : " + desc.getNameAsString());
-            pushRegionEpoch(desc, transactionState);
+        if (LOG.isTraceEnabled()) LOG.trace("createTable: epoch pushed into regions for : " + desc.getNameAsString());
+        pushRegionEpoch(desc, transactionState);
 
-          if (LOG.isTraceEnabled()) LOG.trace("createTable EXIT, transactionState: " + transactionState.getTransactionId());
+        if (LOG.isTraceEnabled()) LOG.trace("createTable EXIT, transactionState: " + transactionState.getTransactionId());
 
     }
 
@@ -3074,7 +3191,7 @@ public class TransactionManager {
             byte[] endKey = regionInfo.getEndKey();
 
             if(endKey != HConstants.EMPTY_END_ROW)
-                endKey = TransactionManager.binaryIncrementPos(endKey, -1);
+               endKey = TransactionManager.binaryIncrementPos(endKey, -1);
 
             table = new HTable(regionInfo.getTable(), connection, cp_tpe);
 
@@ -3097,6 +3214,8 @@ public class TransactionManager {
 
         if(resultArray.length == 0) {
             table.close();
+            LOG.error("Problem with calling recoveryRequest, no regions returned result.  TMID " + tmID
+                    + " region: " + regionName);
             throw new IOException("Problem with calling recoveryRequest, no regions returned result \n"
                                    + " TMid: " + tmid + " region: " + regionName);
         }