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:36 UTC

[01/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

Repository: incubator-trafodion
Updated Branches:
  refs/heads/master c25694d9e -> 815d03329


http://git-wip-us.apache.org/repos/asf/incubator-trafodion/blob/91794b53/core/sql/src/main/java/org/trafodion/sql/HBaseClient.java
----------------------------------------------------------------------
diff --git a/core/sql/src/main/java/org/trafodion/sql/HBaseClient.java b/core/sql/src/main/java/org/trafodion/sql/HBaseClient.java
index db540aa..26edfcd 100644
--- a/core/sql/src/main/java/org/trafodion/sql/HBaseClient.java
+++ b/core/sql/src/main/java/org/trafodion/sql/HBaseClient.java
@@ -949,10 +949,11 @@ public class HBaseClient {
 
     public void releaseHTableClient(HTableClient htable) 
                     throws IOException {
+        if (logger.isDebugEnabled()) logger.debug("HBaseClient.releaseHTableClient("
+            + (htable == null ? " htable is null " : htable.getTableName()) + "). ");
         if (htable == null)
             return;
 	                
-        if (logger.isDebugEnabled()) logger.debug("HBaseClient.releaseHTableClient(" + htable.getTableName() + ").");
         boolean cleanJniObject = false;
         htable.release(cleanJniObject);
     }

http://git-wip-us.apache.org/repos/asf/incubator-trafodion/blob/91794b53/core/sql/src/main/java/org/trafodion/sql/HTableClient.java
----------------------------------------------------------------------
diff --git a/core/sql/src/main/java/org/trafodion/sql/HTableClient.java b/core/sql/src/main/java/org/trafodion/sql/HTableClient.java
index eb100dc..0b4e475 100644
--- a/core/sql/src/main/java/org/trafodion/sql/HTableClient.java
+++ b/core/sql/src/main/java/org/trafodion/sql/HTableClient.java
@@ -30,6 +30,7 @@ import java.util.HashMap;
 import java.util.Map;
 import java.util.NavigableSet;
 
+import org.apache.commons.codec.binary.Hex;
 import java.util.concurrent.Callable;
 import java.util.concurrent.Future;
 import java.util.concurrent.Executors;
@@ -42,6 +43,7 @@ import java.nio.LongBuffer;
 import java.nio.ByteOrder;
 
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.client.Delete;
@@ -51,10 +53,12 @@ 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.Connection;
+import org.apache.hadoop.hbase.client.Table;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.client.coprocessor.AggregationClient;
 import org.apache.hadoop.hbase.client.transactional.RMInterface;
 import org.apache.hadoop.hbase.client.transactional.TransactionalAggregationClient;
+import org.apache.hadoop.hbase.client.transactional.TransactionalTable;
 import org.apache.hadoop.hbase.client.transactional.TransactionState;
 
 import org.apache.log4j.Logger;
@@ -262,6 +266,7 @@ public class HTableClient {
 
 	   public void release() throws IOException
 	   {
+         if (logger.isTraceEnabled()) logger.trace("HTableClient.release(" + (tableName == null ? " tableName is null " : tableName) + ") called.");
 	     if (admin != null)
 	     {
 	       admin.close();
@@ -793,10 +798,14 @@ public class HTableClient {
                                  int espNum,
                                  int versions)
 	        throws IOException {
-	  if (logger.isTraceEnabled()) logger.trace("Enter startScan() " + tableName + " txid: " + transID+ " CacheBlocks: " + cacheBlocks + " numCacheRows: " + numCacheRows + " Bulkread: " + useSnapshotScan);
 
 	  Scan scan;
 
+	  if (logger.isTraceEnabled()) logger.trace("Enter startScan() " + tableName + " txid: " + transID + " startRow="
+			    + ((startRow != null) ? (Bytes.equals(startRow, HConstants.EMPTY_START_ROW) ? "INFINITE" : Hex.encodeHexString(startRow)) : "NULL")
+			    + " stopRow=" + ((stopRow != null) ? (Bytes.equals(stopRow, HConstants.EMPTY_START_ROW) ? "INFINITE" : Hex.encodeHexString(stopRow)) : "NULL")
+		        + " CacheBlocks: " + cacheBlocks + " numCacheRows: " + numCacheRows + " Bulkread: " + useSnapshotScan);
+
 	  if (startRow != null && startRow.toString() == "")
 	    startRow = null;
 	  if (stopRow != null && stopRow.toString() == "")
@@ -1351,8 +1360,8 @@ public class HTableClient {
                                  boolean asyncOperation,
                                  final boolean useRegionXn) throws IOException {
 
-            if (logger.isTraceEnabled()) logger.trace("Enter deleteRow(" + new String(rowID) + ", "
-                                                      + timestamp + ") " + tableName);
+            if (logger.isTraceEnabled()) logger.trace("Enter deleteRow transID " + transID
+                  + " (" + new String(rowID) + ", " + timestamp + ") " + tableName);
             
             final Delete del;
             if (timestamp == -1)
@@ -1402,7 +1411,8 @@ public class HTableClient {
 		      long timestamp,
                       boolean asyncOperation) throws IOException {
 
-	        if (logger.isTraceEnabled()) logger.trace("Enter deleteRows() " + tableName);
+	        if (logger.isTraceEnabled()) logger.trace("Enter deleteRowsInt() transID "
+	              + transID + " " + tableName);
 
 		final List<Delete> listOfDeletes = new ArrayList<Delete>();
 		listOfDeletes.clear();
@@ -1463,9 +1473,8 @@ public class HTableClient {
 					 byte[] columnToCheck, byte[] colValToCheck,
 					 long timestamp, final boolean useRegionXn) throws IOException {
 
-            if (logger.isTraceEnabled()) logger.trace("Enter checkAndDeleteRow(" + new String(rowID) + ", "
-                                                      + new String(columnToCheck) + ", " + new String(colValToCheck) + ", " + timestamp + ") " + tableName);
-            
+            if (logger.isTraceEnabled()) logger.trace("Enter checkAndDeleteRow transID " + transID
+                    + " (" + new String(rowID) + ", "  + new String(columnToCheck) + ", " + new String(colValToCheck) + ", " + timestamp + ") " + tableName);
             Delete del;
             if (timestamp == -1)
                 del = new Delete(rowID);
@@ -1503,7 +1512,10 @@ public class HTableClient {
                               final boolean useRegionXn) throws IOException, InterruptedException, 
                           ExecutionException 
 	{
-		if (logger.isTraceEnabled()) logger.trace("Enter putRow() " + tableName);
+		if (logger.isTraceEnabled()) logger.trace("Enter putRow() " + tableName + 
+							  " transID: " + transID +
+							  " useTRex: " + useTRex +
+							  " useRegionXn: " + useRegionXn);
 
 	 	final Put put;
 		ByteBuffer bb;
@@ -1618,8 +1630,9 @@ public class HTableClient {
                        long timestamp, boolean asyncOperation)
 			throws IOException, InterruptedException, ExecutionException  {
 
-		if (logger.isTraceEnabled()) logger.trace("Enter putRows() " + tableName);
-
+		if (logger.isTraceEnabled()) logger.trace("Enter putRows() " + tableName +
+							  " transID: " + transID + 
+							  " useTRex: " + useTRex);
 		Put put;
 		ByteBuffer bbRows, bbRowIDs;
 		short numCols, numRows;
@@ -1725,16 +1738,19 @@ public class HTableClient {
 		    Configuration customConf = connection.getConfiguration();
                     long rowCount = 0;
 
-                    if (transID > 0) {
-		      TransactionalAggregationClient aggregationClient = 
+       if (transID > 0) {
+          TransactionalAggregationClient aggregationClient = 
                           new TransactionalAggregationClient(customConf, connection);
 		      Scan scan = new Scan();
 		      scan.addFamily(colFamily);
 		      scan.setCacheBlocks(false);
 		      final ColumnInterpreter<Long, Long, EmptyMsg, LongMsg, LongMsg> ci =
-			new LongColumnInterpreter();
+              new LongColumnInterpreter();
 		      byte[] tname = getTableName().getBytes();
-		      rowCount = aggregationClient.rowCount(transID, 
+              TransactionalTable lv_ttable = new TransactionalTable(getTableName(), connection);
+              TransactionState ts = table.registerTransaction(lv_ttable, transID, startRowID);
+
+		      rowCount = aggregationClient.rowCount(transID, ts.getStartId(),
                         org.apache.hadoop.hbase.TableName.valueOf(getTableName()),
                         ci,
                         scan);


[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

Posted by sa...@apache.org.
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);
         }


[06/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

Posted by sa...@apache.org.
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/coprocessor/transactional/generated/SsccRegionProtos.java
----------------------------------------------------------------------
diff --git a/core/sqf/src/seatrans/hbase-trx/src/main/java/org/apache/hadoop/hbase/coprocessor/transactional/generated/SsccRegionProtos.java b/core/sqf/src/seatrans/hbase-trx/src/main/java/org/apache/hadoop/hbase/coprocessor/transactional/generated/SsccRegionProtos.java
index 5e6e90d..d2782a0 100644
--- a/core/sqf/src/seatrans/hbase-trx/src/main/java/org/apache/hadoop/hbase/coprocessor/transactional/generated/SsccRegionProtos.java
+++ b/core/sqf/src/seatrans/hbase-trx/src/main/java/org/apache/hadoop/hbase/coprocessor/transactional/generated/SsccRegionProtos.java
@@ -1,24 +1,3 @@
-// @@@ START COPYRIGHT @@@
-//
-// Licensed to the Apache Software Foundation (ASF) under one
-// or more contributor license agreements.  See the NOTICE file
-// distributed with this work for additional information
-// regarding copyright ownership.  The ASF licenses this file
-// to you under the Apache License, Version 2.0 (the
-// "License"); you may not use this file except in compliance
-// with the License.  You may obtain a copy of the License at
-//
-//   http://www.apache.org/licenses/LICENSE-2.0
-//
-// Unless required by applicable law or agreed to in writing,
-// software distributed under the License is distributed on an
-// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
-// KIND, either express or implied.  See the License for the
-// specific language governing permissions and limitations
-// under the License.
-//
-// @@@ END COPYRIGHT @@@
-
 // Generated by the protocol buffer compiler.  DO NOT EDIT!
 // source: SsccRegion.proto
 


[08/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

Posted by sa...@apache.org.
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/TransactionRegionLocation.java
----------------------------------------------------------------------
diff --git a/core/sqf/src/seatrans/hbase-trx/src/main/java/org/apache/hadoop/hbase/client/transactional/TransactionRegionLocation.java b/core/sqf/src/seatrans/hbase-trx/src/main/java/org/apache/hadoop/hbase/client/transactional/TransactionRegionLocation.java
index d3118db..38f598b 100644
--- a/core/sqf/src/seatrans/hbase-trx/src/main/java/org/apache/hadoop/hbase/client/transactional/TransactionRegionLocation.java
+++ b/core/sqf/src/seatrans/hbase-trx/src/main/java/org/apache/hadoop/hbase/client/transactional/TransactionRegionLocation.java
@@ -28,7 +28,7 @@ import org.apache.commons.logging.LogFactory;
 import org.apache.commons.codec.binary.Hex;
 
 import org.apache.hadoop.hbase.util.Bytes;
-
+import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.HRegionLocation;
 import org.apache.hadoop.hbase.ServerName;
@@ -63,62 +63,79 @@ public class TransactionRegionLocation extends HRegionLocation {
 
    @Override
    public int compareTo(HRegionLocation o) {
-      if (LOG.isTraceEnabled()) LOG.trace("compareTo ENTRY: " + o);
-      int result = super.getHostname().compareTo(o.getHostname());
+      if (o == null) {
+        if (LOG.isDebugEnabled()) LOG.debug("CompareTo TransactionRegionLocation object is null");
+        return 1;
+      }
+
+      if (LOG.isDebugEnabled()) LOG.debug("CompareTo TransactionRegionLocation Entry:  TableNames :\n      mine: "
+              + this.getRegionInfo().getTable().getNameAsString() + "\n object's : " + o.getRegionInfo().getTable().getNameAsString());
+
+      // Make sure this is the same table
+      int result = this.getRegionInfo().getTable().compareTo(o.getRegionInfo().getTable());
       if (result != 0){
-         if (LOG.isTraceEnabled()) LOG.trace("compareTo hostnames differ: mine: " + super.getHostname() + " hex: " +
-              Hex.encodeHexString(Bytes.toBytes(super.getHostname())) + " object's: " + o.getHostname() + " hex: " + Hex.encodeHexString(Bytes.toBytes(o.getHostname())));
+         if (LOG.isDebugEnabled()) LOG.debug("compareTo TransactionRegionLocation TableNames are different: result is " + result);
          return result;
       }
-      result = super.getPort() - o.getPort();
-      if (result != 0) {
-         if (LOG.isTraceEnabled()) LOG.trace("compareTo ports differ: mine: " + super.getPort() + " object's: " + o.getPort());
-         return result;
+
+      if (LOG.isDebugEnabled()) LOG.debug("Tables match - comparing keys for "
+              + this.getRegionInfo().getTable().getNameAsString()
+              + "\n This start key    : " + Hex.encodeHexString(this.getRegionInfo().getStartKey())
+              + "\n Object's start key: " + Hex.encodeHexString(o.getRegionInfo().getStartKey())
+              + "\n This end key    : " + Hex.encodeHexString(this.getRegionInfo().getEndKey())
+              + "\n Object's end key: " + Hex.encodeHexString(o.getRegionInfo().getEndKey()));
+
+      // Here we are going to compare the keys as a range we can return 0
+      // For these comparisons it's important to remember that 'this' is the object that is being added
+      // and that 'object' is an object already added in the participationRegions set.
+      //
+      // We are trying to limit the registration of daughter regions after a region split.
+      // So if a location is already added whose startKey is less than ours and whose end
+      // key is greater than ours we will return 0 so that 'this' does not get added into
+      // the participatingRegions list.
+
+      // firstKeyInRange will be true if object's startKey is less than ours.
+      int startKeyResult = Bytes.compareTo(this.getRegionInfo().getStartKey(), o.getRegionInfo().getStartKey());
+      boolean firstKeyInRange = startKeyResult >= 0;
+      boolean objLastKeyInfinite = Bytes.equals(o.getRegionInfo().getEndKey(), HConstants.EMPTY_END_ROW);
+      boolean thisLastKeyInfinite = Bytes.equals(this.getRegionInfo().getEndKey(), HConstants.EMPTY_END_ROW);
+      int endKeyResult = Bytes.compareTo(this.getRegionInfo().getEndKey(), o.getRegionInfo().getEndKey());
+
+      // lastKey is in range if the existing object has an infinite end key, no matter what this end key is.
+      boolean lastKeyInRange =  objLastKeyInfinite || ( ! thisLastKeyInfinite && endKeyResult <= 0);
+      if (LOG.isDebugEnabled()) LOG.debug("firstKeyInRange " + firstKeyInRange + " lastKeyInRange " + lastKeyInRange);
+
+      if (firstKeyInRange && lastKeyInRange) {
+         if (LOG.isDebugEnabled()) LOG.debug("Object's region contains this region's start and end keys.  Regions match for "
+                                   + o.getRegionInfo().getTable().getNameAsString());
+         return 0;
       }
-      result = super.getRegionInfo().compareTo(o.getRegionInfo());
-      if (result != 0) {
-         if (LOG.isTraceEnabled()) LOG.trace("compareTo regionInfo differs: mine: " + super.getRegionInfo() + " hex: " +
-             Hex.encodeHexString(super.getRegionInfo().getRegionName()) + " object's: " + o.getRegionInfo() + "hex: " +
-             Hex.encodeHexString(o.getRegionInfo().getRegionName()));
+
+      if (startKeyResult != 0){
+         if (LOG.isDebugEnabled()) LOG.debug("compareTo TransactionRegionLocation startKeys don't match: result is " + startKeyResult);
+         return startKeyResult;
       }
-      if (LOG.isTraceEnabled()) LOG.trace("compareTo HRegionLocation result is: " + result );
-      return result;
-   }
 
-   public int compareTo(TransactionRegionLocation o) {
-      if (LOG.isTraceEnabled()) LOG.trace("compareTo TransactionRegionLocation ENTRY: " + o.getRegionInfo().getRegionNameAsString());
-      int result = super.getHostname().compareTo(o.getHostname());
-      if (result != 0){
-         if (LOG.isTraceEnabled()) LOG.trace("compareTo hostnames differ: mine: " + super.getHostname() + " hex: " +
-              Hex.encodeHexString(Bytes.toBytes(super.getHostname())) + " object's: " + o.getHostname() + " hex: " + Hex.encodeHexString(Bytes.toBytes(o.getHostname())));
-         return result;
+      if (objLastKeyInfinite) {
+         if (LOG.isInfoEnabled()) LOG.info("Object's region contains this region's end keys for "
+                  + o.getRegionInfo().getTable().getNameAsString()
+                  + "\n This start key    : " + Hex.encodeHexString(this.getRegionInfo().getStartKey())
+                  + "\n Object's start key: " + Hex.encodeHexString(o.getRegionInfo().getStartKey())
+                  + "\n This end key    : " + Hex.encodeHexString(this.getRegionInfo().getEndKey())
+                  + "\n Object's end key: " + Hex.encodeHexString(o.getRegionInfo().getEndKey()));
       }
-      result = super.getPort() - o.getPort();
-      if (result != 0) {
-         if (LOG.isTraceEnabled()) LOG.trace("compareTo ports differ: mine: " + super.getPort() + " object's: " + o.getPort());
-         return result;
+
+      if (this.getRegionInfo().getStartKey().length != 0 && this.getRegionInfo().getEndKey().length == 0) {
+         if (LOG.isDebugEnabled()) LOG.debug("compareTo TransactionRegionLocation \"this\" is the last region: result is 1");
+         return 1; // this is last region
       }
-      result = super.getRegionInfo().compareTo(o.getRegionInfo());
-      if (result != 0) {
-         if (super.getRegionInfo().getEncodedName().compareTo(o.getRegionInfo().getEncodedName()) == 0){
-            if (LOG.isTraceEnabled()) LOG.trace("compareTo regionInfo RegionNames match " + o.getRegionInfo().getEncodedName());
-            if (super.getRegionInfo().containsRange(o.getRegionInfo().getStartKey(), o.getRegionInfo().getEndKey())) {
-               if (LOG.isTraceEnabled()) LOG.trace("This region contains object's start and end keys.  Regions match " + o.getRegionInfo().getEncodedName());
-               result = 0;
-               return result;
-            }
-            if (LOG.isTraceEnabled()) LOG.trace("compareTo TransactionRegionLocation regionInfo RegionNames match, but object end keys differ:\n  this hex name: "
-                        + Hex.encodeHexString(super.getRegionInfo().getRegionName()) + "\n   obj hex name: "
-                        + Hex.encodeHexString(o.getRegionInfo().getRegionName())
-                        + "\n This end key    : " + Hex.encodeHexString(super.getRegionInfo().getEndKey())
-                        + "\n Object's end key: " + Hex.encodeHexString(o.getRegionInfo().getEndKey())  + "\n result " + result);
-         }
-         else {
-            if (LOG.isTraceEnabled()) LOG.trace("compareTo TransactionRegionLocation regionInfo.getEncodedName differs:\n      mine: "
-                    + super.getRegionInfo().getEncodedName() + "\n object's : " + o.getRegionInfo().getEncodedName() + " result is " + result);
-         }
+      if (o.getRegionInfo().getStartKey().length != 0 && o.getRegionInfo().getEndKey().length == 0) {
+         if (LOG.isDebugEnabled()) LOG.debug("compareTo TransactionRegionLocation \"object\" is the last region: result is -1");
+         return -1; // o is the last region
       }
-      return result;
+      if (LOG.isDebugEnabled()) LOG.debug("compareTo TransactionRegionLocation endKeys comparison: result is " + endKeyResult);
+      return endKeyResult;
+
    }
 
   /**
@@ -126,20 +143,32 @@ public class TransactionRegionLocation extends HRegionLocation {
    */
   @Override
   public boolean equals(Object o) {
-    if (LOG.isTraceEnabled()) LOG.trace("equals ENTRY: this: " + this + " o: " + o);
+    if (LOG.isDebugEnabled()) LOG.debug("equals ENTRY: this: " + this + " o: " + o);
     if (this == o) {
-      if (LOG.isTraceEnabled()) LOG.trace("equals same object: " + o);
+      if (LOG.isDebugEnabled()) LOG.debug("equals same object: " + o);
       return true;
     }
     if (o == null) {
-      if (LOG.isTraceEnabled()) LOG.trace("equals o is null");
+      if (LOG.isDebugEnabled()) LOG.debug("equals o is null");
       return false;
     }
-    if (!(o instanceof TransactionRegionLocation)) {
-      if (LOG.isTraceEnabled()) LOG.trace("equals o is not an instance of: " + o);
+    if (!(o instanceof HRegionLocation)) {
+      if (LOG.isDebugEnabled()) LOG.debug("equals o is not an instance of: " + o);
       return false;
     }
-    return this.compareTo((TransactionRegionLocation)o) == 0;
+    return this.compareTo((HRegionLocation)o) == 0;
   }
 
+  /**
+   * toString
+   * @return String this
+   *
+   */
+  @Override
+  public String toString() {
+    return super.toString() + "encodedName " + super.getRegionInfo().getEncodedName()
+            + " start key: " + Hex.encodeHexString(super.getRegionInfo().getStartKey())
+            + " end key: " + Hex.encodeHexString(super.getRegionInfo().getEndKey())
+            + " tableRecodedDropped " + isTableRecodedDropped();
+  }
 }

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/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 fee5d54..4b5fdf9 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
@@ -33,6 +33,7 @@ import org.apache.commons.codec.binary.Hex;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.HRegionLocation;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.HConstants;
@@ -45,11 +46,16 @@ public class TransactionState {
 
     static final Log LOG = LogFactory.getLog(TransactionState.class);
 
+    // Current transactionId has the following composition:
+    //  int   sequenceId
+    //  short nodeId
+    //  short clusterId
     private final long transactionId;
     private TransState status;
     private long startEpoch;
     private long startId;
     private long commitId;
+    private long recoveryASN;
 
     /**
      * 
@@ -71,9 +77,11 @@ public class TransactionState {
     private boolean ddlTrans;
     private static boolean useConcurrentHM = false;
     private boolean hasRetried = false;
-    private boolean uteLogged = false;
+    private boolean exceptionLogged = false;
+    private long nodeId;
+    private long clusterId;
     private String recordException;
-
+    private static long TM_MAX_REGIONSERVER_STRING = 3072;
     public Set<String> tableNames = Collections.synchronizedSet(new HashSet<String>());
     public Set<TransactionRegionLocation> participatingRegions;
     /**
@@ -82,7 +90,7 @@ public class TransactionState {
     public Set<TransactionRegionLocation> regionsToIgnore = Collections.synchronizedSet(new HashSet<TransactionRegionLocation>());
     private Set<TransactionRegionLocation> retryRegions = Collections.synchronizedSet(new HashSet<TransactionRegionLocation>());
 
-    private native void registerRegion(long transid, long startId, int port, byte[] hostname, long startcode, byte[] regionInfo);
+    private native void registerRegion(long transid, long startid, int port, byte[] hostname, long startcode, byte[] regionInfo);
 
     public boolean islocalTransaction() {
       return localTransaction;
@@ -115,6 +123,9 @@ public class TransactionState {
         commitSendDone = false;
         hasError = null;
         ddlTrans = false;
+        recoveryASN = -1;
+        setNodeId();
+        setClusterId();
         recordException = null;
 
         if(useConcurrentHM) {
@@ -279,15 +290,13 @@ public class TransactionState {
       recordException = null;
     }
 
-      public void registerLocation(final TransactionRegionLocation location) throws IOException {
+    // Used at the client end - the one performing the mutation - e.g. the SQL process
+    public void registerLocation(final TransactionRegionLocation location) throws IOException {
         byte [] lv_hostname = location.getHostname().getBytes();
         int lv_port = location.getPort();
         long lv_startcode = location.getServerName().getStartcode();
+        HRegionInfo regionInfo = location.getRegionInfo();
 
-        /*        ByteArrayOutputStream lv_bos = new ByteArrayOutputStream();
-        DataOutputStream lv_dos = new DataOutputStream(lv_bos);
-        location.getRegionInfo().write(lv_dos);
-        lv_dos.flush(); */
         byte [] lv_byte_region_info = location.getRegionInfo().toByteArray();
         if (LOG.isTraceEnabled()) LOG.trace("TransactionState.registerLocation: [" + location.getRegionInfo().getEncodedName() +
           "], endKey: " + Hex.encodeHexString(location.getRegionInfo().getEndKey()) + " transaction [" + transactionId + "]");
@@ -296,27 +305,50 @@ public class TransactionState {
           if(LOG.isTraceEnabled()) LOG.trace("TransactionState.registerLocation local transaction not sending registerRegion.");
         }
         else {
-          if (LOG.isTraceEnabled()) LOG.trace("TransactionState.registerLocation global transaction registering region for ts: " + transactionId + " and startId: " + startId);
+          if (lv_byte_region_info.length > TM_MAX_REGIONSERVER_STRING){
+             String skey = (Bytes.equals(location.getRegionInfo().getStartKey(), HConstants.EMPTY_START_ROW)) ? "skey=null" : ("skey=" + Hex.encodeHexString(location.getRegionInfo().getStartKey()));
+             String ekey = (Bytes.equals(location.getRegionInfo().getEndKey(), HConstants.EMPTY_END_ROW)) ? "ekey=null" : ("ekey=" + Hex.encodeHexString(location.getRegionInfo().getEndKey()));
+             IOException ioe = new IOException("RegionInfo is too large (" + lv_byte_region_info.length
+                     + "), try reducing table keys for "
+                     + location.getRegionInfo().getTable().getNameAsString()
+                     + " skey: " + skey + " ekey: " + ekey);
+             LOG.error("RegionInfo is too large (" + lv_byte_region_info.length
+                     + "), try reducing table keys for "
+                     + location.getRegionInfo().getTable().getNameAsString()
+                     + " skey: " + skey + " ekey: " + ekey, ioe);
+             throw ioe;
+          }
+          if (LOG.isDebugEnabled()){
+              String skey = (Bytes.equals(location.getRegionInfo().getStartKey(), HConstants.EMPTY_START_ROW)) ? "skey=null" : ("skey=" + Hex.encodeHexString(location.getRegionInfo().getStartKey()));
+              String ekey = (Bytes.equals(location.getRegionInfo().getEndKey(), HConstants.EMPTY_END_ROW)) ? "ekey=null" : ("ekey=" + Hex.encodeHexString(location.getRegionInfo().getEndKey()));
+              LOG.debug("TransactionState.registerLocation global transaction registering region "
+                   + location.getRegionInfo().getTable().getNameAsString()
+                   + " using byte array size: " + lv_byte_region_info.length + " for ts: "
+                   + transactionId + " and startId: " + startId + " skey: " + skey + " ekey: " + ekey + " ");
+          }
           registerRegion(transactionId, startId, lv_port, lv_hostname, lv_startcode, lv_byte_region_info);
         }
       }
 
     public boolean addRegion(final TransactionRegionLocation trRegion) {
-        if (LOG.isTraceEnabled()) LOG.trace("addRegion ENTRY with trRegion [" + trRegion.getRegionInfo().getEncodedName() + "], endKey: "
+        if (LOG.isDebugEnabled()) LOG.debug("addRegion ENTRY with trRegion [" + trRegion.getRegionInfo().getRegionNameAsString()
+                + "], startKey: " + Hex.encodeHexString(trRegion.getRegionInfo().getStartKey()) + "], endKey: "
                   + Hex.encodeHexString(trRegion.getRegionInfo().getEndKey()) + " and transaction [" + transactionId + "]");
 
         boolean added = participatingRegions.add(trRegion);
 
         if (added) {
-           if (LOG.isTraceEnabled()) LOG.trace("Added new trRegion (#" + participatingRegions.size()
+           if (LOG.isDebugEnabled()) LOG.debug("Added new trRegion (#" + participatingRegions.size()
                         + ") to participatingRegions ["        + trRegion.getRegionInfo().getRegionNameAsString()
-                        + "], endKey: "        + Hex.encodeHexString(trRegion.getRegionInfo().getEndKey()) 
+                        + "], startKey: "      + Hex.encodeHexString(trRegion.getRegionInfo().getStartKey())
+                        + "], endKey: "        + Hex.encodeHexString(trRegion.getRegionInfo().getEndKey())
                         + " and transaction [" + transactionId + "]");
         }
         else {
-           if (LOG.isTraceEnabled()) LOG.trace("trRegion already present in (" + participatingRegions.size()
-                       + ") participatinRegions ["    + trRegion.getRegionInfo().getEncodedName()
-                       + "], endKey: "        + Hex.encodeHexString(trRegion.getRegionInfo().getEndKey()) 
+           if (LOG.isDebugEnabled()) LOG.debug("trRegion already present in (" + participatingRegions.size()
+                       + ") participatinRegions ["    + trRegion.getRegionInfo().getRegionNameAsString()
+                       + "], startKey: "      + Hex.encodeHexString(trRegion.getRegionInfo().getStartKey())
+                       + "], endKey: "        + Hex.encodeHexString(trRegion.getRegionInfo().getEndKey())
                        + " and transaction [" + transactionId + "]");
         }
 
@@ -325,10 +357,10 @@ public class TransactionState {
 
     public boolean addRegion(final HRegionLocation hregion) {
 
-        if (LOG.isTraceEnabled()) LOG.trace("Creating new TransactionRegionLocation from HRegionLocation [" + hregion.getRegionInfo().getRegionNameAsString() +
+        if (LOG.isDebugEnabled()) LOG.debug("Creating new TransactionRegionLocation from HRegionLocation [" + hregion.getRegionInfo().getRegionNameAsString() +
                               " endKey: " + Hex.encodeHexString(hregion.getRegionInfo().getEndKey()) + " for transaction [" + transactionId + "]");
         TransactionRegionLocation trRegion = new TransactionRegionLocation(hregion.getRegionInfo(),
-                                                                             hregion.getServerName());
+									   hregion.getServerName());
 // Save hregion for now
         boolean added = participatingRegions.add(trRegion);
 
@@ -409,8 +441,16 @@ public class TransactionState {
      * @return Return the nodeId.
      */
     public long getNodeId() {
+       return nodeId;
+    }
+
+    /**
+     * Set the originating node of the transaction.
+     *
+     */
+    private void setNodeId() {
 
-       return ((transactionId >> 32) & 0xFFL);
+       nodeId = ((transactionId >> 32) & 0xFFL);
     }
 
     /**
@@ -424,6 +464,37 @@ public class TransactionState {
     }
 
     /**
+     * Get the originating clusterId of the transaction.
+     *
+     * @return Return the clusterId.
+     */
+    public long getClusterId() {
+
+        return clusterId;
+
+    }
+
+
+    /**
+     * Get the originating clusterId of the passed in transaction.
+     *
+     * @return Return the clusterId.
+     */
+    public static long getClusterId(long transId) {
+
+        return (transId >> 48);
+    }
+
+    /**
+     * Set the originating clusterId of the passed in transaction.
+     *
+     */
+    private void setClusterId() {
+
+        clusterId = (transactionId >> 48);
+    }
+
+    /**
      * Set the startEpoc.
      *
      */
@@ -475,6 +546,23 @@ public class TransactionState {
     }
 
     /**
+     * Set the recoveryASN.
+     *
+     */
+    public void setRecoveryASN(final long value) {
+        this.recoveryASN = value;
+    }
+
+    /**
+     * Get the recoveryASN.
+     *
+     * @return Return the recoveryASN.
+     */
+    public long getRecoveryASN() {
+        return recoveryASN;
+    }
+
+    /**
      * @see java.lang.Object#toString()
      */
     @Override
@@ -482,7 +570,7 @@ public class TransactionState {
         return "transactionId: " + transactionId + ", startId: " + startId + ", commitId: " + commitId +
                ", startEpoch: " + startEpoch + ", participants: " + participatingRegions.size()
                + ", ignoring: " + regionsToIgnore.size() + ", hasDDL: " + hasDDLTx()
-               + ", state: " + status.toString();
+               + ", recoveryASN: " + getRecoveryASN() + ", state: " + status.toString();
     }
 
     public int getParticipantCount() {
@@ -521,18 +609,26 @@ public class TransactionState {
       return this.hasRetried;
     }
 
-    public void logUteDetails()
+    public boolean hasPlaceHolder() {
+       return false;
+    }
+
+    public synchronized void logExceptionDetails(final boolean ute)
     {
-       if (uteLogged)
+       if (exceptionLogged)
           return;
        int participantNum = 0;
        byte[] startKey;
        byte[] endKey_orig;
        byte[] endKey;
+       boolean isIgnoredRegion = false;
 
+       LOG.error("Starting " + (ute == true ? "UTE " : "NPTE ") + "for trans: " + this.toString());
        for (TransactionRegionLocation location : getParticipatingRegions()) {
-          participantNum++;
-          final byte[] regionName = location.getRegionInfo().getRegionName();
+          isIgnoredRegion = getRegionsToIgnore().contains(location);
+          if (! isIgnoredRegion) {
+             participantNum++;
+          }
 
           startKey = location.getRegionInfo().getStartKey();
           endKey_orig = location.getRegionInfo().getEndKey();
@@ -541,13 +637,16 @@ public class TransactionState {
           else
               endKey = TransactionManager.binaryIncrementPos(endKey_orig, -1);
 
-          LOG.warn("UTE for transId: " + getTransactionId()
-                    + " participantNum " + participantNum
+          LOG.error((ute == true ? "UTE " : "NPTE ") + "for transId: " + getTransactionId()
+                    + " participantNum " + (isIgnoredRegion ? " Ignored region " : 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"));
+                    + " 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")
+                    + " RegionEndKey " + ((endKey_orig != null) ?
+                             (Bytes.equals(endKey_orig, HConstants.EMPTY_END_ROW) ? "INFINITE" : Hex.encodeHexString(endKey_orig)) : "NULL"));
        }
-       uteLogged = true;
+       exceptionLogged = true;
     }
 }

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/TransactionalAggregationClient.java
----------------------------------------------------------------------
diff --git a/core/sqf/src/seatrans/hbase-trx/src/main/java/org/apache/hadoop/hbase/client/transactional/TransactionalAggregationClient.java b/core/sqf/src/seatrans/hbase-trx/src/main/java/org/apache/hadoop/hbase/client/transactional/TransactionalAggregationClient.java
index 8d54246..52edcd9 100644
--- a/core/sqf/src/seatrans/hbase-trx/src/main/java/org/apache/hadoop/hbase/client/transactional/TransactionalAggregationClient.java
+++ b/core/sqf/src/seatrans/hbase-trx/src/main/java/org/apache/hadoop/hbase/client/transactional/TransactionalAggregationClient.java
@@ -98,6 +98,7 @@ public class TransactionalAggregationClient {
    * given range. In case qualifier is null, a max of all values for the given
    * family is returned.
    * @param transactionId
+   * @param startId
    * @param tableName
    * @param ci
    * @param scan
@@ -107,12 +108,12 @@ public class TransactionalAggregationClient {
    *           & propagated to it.
    */
   public <R, S, P extends Message, Q extends Message, T extends Message> R max(
-       final long transactionId, final TableName tableName, final ColumnInterpreter<R, S, P, Q, T> ci, final Scan scan)
+       final long transactionId, final long startId, final TableName tableName, final ColumnInterpreter<R, S, P, Q, T> ci, final Scan scan)
       throws Throwable {
     TransactionalTable table = null;
     try {
       table = new TransactionalTable(tableName.getName(), connection);
-      return max(transactionId, table, ci, scan);
+      return max(transactionId, startId, table, ci, scan);
     } finally {
       if (table != null) {
         table.close();
@@ -133,12 +134,12 @@ public class TransactionalAggregationClient {
    *           & propagated to it.
    */
   public <R, S, P extends Message, Q extends Message, T extends Message> 
-  R max(final long transactionId, final TransactionalTable table, final ColumnInterpreter<R, S, P, Q, T> ci,
+  R max(final long transactionId, final long startId, final TransactionalTable table, final ColumnInterpreter<R, S, P, Q, T> ci,
       final Scan scan) throws Throwable {
       byte [] currentBeginKey = scan.getStartRow();	
       HRegionInfo currentRegion = table.getRegionLocation(currentBeginKey).getRegionInfo();
       com.google.protobuf.ByteString regionName = ByteString.copyFromUtf8(currentRegion.getRegionNameAsString());
-    final TransactionalAggregateRequest requestArg = validateArgAndGetPB(regionName, transactionId, scan, ci, false);
+    final TransactionalAggregateRequest requestArg = validateArgAndGetPB(regionName, transactionId, startId, scan, ci, false);
     class MaxCallBack implements Batch.Callback<R> {
       R max = null;
 
@@ -199,6 +200,7 @@ public class TransactionalAggregationClient {
    * given range. In case qualifier is null, a min of all values for the given
    * family is returned.
    * @param transactionId
+   * @param startId
    * @param tableName
    * @param ci
    * @param scan
@@ -206,12 +208,12 @@ public class TransactionalAggregationClient {
    * @throws Throwable
    */
   public <R, S, P extends Message, Q extends Message, T extends Message> R min(
-       final long transactionId, final TableName tableName, final ColumnInterpreter<R, S, P, Q, T> ci, final Scan scan)
+       final long transactionId, final long startId, final TableName tableName, final ColumnInterpreter<R, S, P, Q, T> ci, final Scan scan)
       throws Throwable {
     TransactionalTable table = null;
     try {
       table = new TransactionalTable(tableName.getName(), connection);
-      return min(transactionId, table, ci, scan);
+      return min(transactionId, startId, table, ci, scan);
     } finally {
       if (table != null) {
         table.close();
@@ -230,12 +232,12 @@ public class TransactionalAggregationClient {
    * @throws Throwable
    */
   public <R, S, P extends Message, Q extends Message, T extends Message> 
-  R min(final long transactionId, final TransactionalTable table, final ColumnInterpreter<R, S, P, Q, T> ci,
+  R min(final long transactionId, final long startId, final TransactionalTable table, final ColumnInterpreter<R, S, P, Q, T> ci,
       final Scan scan) throws Throwable {
       byte [] currentBeginKey = scan.getStartRow();	
       HRegionInfo currentRegion = table.getRegionLocation(currentBeginKey).getRegionInfo();
       com.google.protobuf.ByteString regionName = ByteString.copyFromUtf8(currentRegion.getRegionNameAsString());
-    final TransactionalAggregateRequest requestArg = validateArgAndGetPB(regionName, transactionId, scan, ci, false);
+    final TransactionalAggregateRequest requestArg = validateArgAndGetPB(regionName, transactionId, startId, scan, ci, false);
     class MinCallBack implements Batch.Callback<R> {
 
       private R min = null;
@@ -283,6 +285,7 @@ public class TransactionalAggregationClient {
    * not of the given filter: in this case, this particular row will not be
    * counted ==> an error.
    * @param transactionId
+   * @param startId
    * @param tableName
    * @param ci
    * @param scan
@@ -291,6 +294,7 @@ public class TransactionalAggregationClient {
    */
   public <R, S, P extends Message, Q extends Message, T extends Message> long rowCount(
       final long transactionId,
+      final long startId,
       final TableName tableName, 
       final ColumnInterpreter<R, S, P, Q, T> ci, 
       final Scan scan)
@@ -298,7 +302,7 @@ public class TransactionalAggregationClient {
     TransactionalTable table = null;
     try {
       table = new TransactionalTable(tableName.getName(), connection);
-      return rowCount(transactionId, table, ci, scan);
+      return rowCount(transactionId, startId, table, ci, scan);
     } finally {
       if (table != null) {
         table.close();
@@ -320,12 +324,12 @@ public class TransactionalAggregationClient {
    * @throws Throwable
    */
   public <R, S, P extends Message, Q extends Message, T extends Message> 
-  long rowCount(final long transactionId, final TransactionalTable table,
+  long rowCount(final long transactionId, final long startId, final TransactionalTable table,
       final ColumnInterpreter<R, S, P, Q, T> ci, final Scan scan) throws Throwable {
       byte [] currentBeginKey = scan.getStartRow();	
       HRegionInfo currentRegion = table.getRegionLocation(currentBeginKey).getRegionInfo();
       com.google.protobuf.ByteString regionName = ByteString.copyFromUtf8(currentRegion.getRegionNameAsString());
-    final TransactionalAggregateRequest requestArg = validateArgAndGetPB(regionName, transactionId, scan, ci, true);
+    final TransactionalAggregateRequest requestArg = validateArgAndGetPB(regionName, transactionId, startId, scan, ci, true);
     class RowNumCallback implements Batch.Callback<Long> {
       private final AtomicLong rowCountL = new AtomicLong(0);
 
@@ -364,6 +368,7 @@ public class TransactionalAggregationClient {
    * It sums up the value returned from various regions. In case qualifier is
    * null, summation of all the column qualifiers in the given family is done.
    * @param transactionId
+   * @param startId
    * @param tableName
    * @param ci
    * @param scan
@@ -371,12 +376,12 @@ public class TransactionalAggregationClient {
    * @throws Throwable
    */
   public <R, S, P extends Message, Q extends Message, T extends Message> S sum(
-       final long transactionId, final TableName tableName, final ColumnInterpreter<R, S, P, Q, T> ci, final Scan scan)
+       final long transactionId, final long startId, final TableName tableName, final ColumnInterpreter<R, S, P, Q, T> ci, final Scan scan)
       throws Throwable {
     TransactionalTable table = null;
     try {
       table = new TransactionalTable(tableName.getName(), connection);
-      return sum(transactionId, table, ci, scan);
+      return sum(transactionId, startId, table, ci, scan);
     } finally {
       if (table != null) {
         table.close();
@@ -394,12 +399,12 @@ public class TransactionalAggregationClient {
    * @throws Throwable
    */
   public <R, S, P extends Message, Q extends Message, T extends Message> 
-  S sum(final long transactionId, final TransactionalTable table, final ColumnInterpreter<R, S, P, Q, T> ci,
+  S sum(final long transactionId, final long startId, final TransactionalTable table, final ColumnInterpreter<R, S, P, Q, T> ci,
       final Scan scan) throws Throwable {
       byte [] currentBeginKey = scan.getStartRow();	
       HRegionInfo currentRegion = table.getRegionLocation(currentBeginKey).getRegionInfo();
       com.google.protobuf.ByteString regionName = ByteString.copyFromUtf8(currentRegion.getRegionNameAsString());
-    final TransactionalAggregateRequest requestArg = validateArgAndGetPB(regionName, transactionId, scan, ci, false);
+    final TransactionalAggregateRequest requestArg = validateArgAndGetPB(regionName, transactionId, startId, scan, ci, false);
     
     class SumCallBack implements Batch.Callback<S> {
       S sumVal = null;
@@ -443,17 +448,18 @@ public class TransactionalAggregationClient {
    * corresponding regions. Approach is to compute a global sum of region level
    * sum and rowcount and then compute the average.
    * @param transactionId
+   * @param startId
    * @param tableName
    * @param scan
    * @throws Throwable
    */
   private <R, S, P extends Message, Q extends Message, T extends Message> Pair<S, Long> getAvgArgs(
-       final long transactionId, final TableName tableName, final ColumnInterpreter<R, S, P, Q, T> ci, final Scan scan)
+       final long transactionId, final long startId, final TableName tableName, final ColumnInterpreter<R, S, P, Q, T> ci, final Scan scan)
       throws Throwable {
     TransactionalTable table = null;
     try {
       table = new TransactionalTable(tableName.getName(), connection);
-      return getAvgArgs(transactionId, table, ci, scan);
+      return getAvgArgs(transactionId, startId, table, ci, scan);
     } finally {
       if (table != null) {
         table.close();
@@ -470,12 +476,12 @@ public class TransactionalAggregationClient {
    * @throws Throwable
    */
   private <R, S, P extends Message, Q extends Message, T extends Message>
-  Pair<S, Long> getAvgArgs(final long transactionId, final TransactionalTable table,
+  Pair<S, Long> getAvgArgs(final long transactionId, final long startId, final TransactionalTable table,
       final ColumnInterpreter<R, S, P, Q, T> ci, final Scan scan) throws Throwable {
       byte [] currentBeginKey = scan.getStartRow();	
       HRegionInfo currentRegion = table.getRegionLocation(currentBeginKey).getRegionInfo();
       com.google.protobuf.ByteString regionName = ByteString.copyFromUtf8(currentRegion.getRegionNameAsString());
-    final TransactionalAggregateRequest requestArg = validateArgAndGetPB(regionName, transactionId, scan, ci, false);
+    final TransactionalAggregateRequest requestArg = validateArgAndGetPB(regionName, transactionId, startId, scan, ci, false);
     class AvgCallBack implements Batch.Callback<Pair<S, Long>> {
       S sum = null;
       Long rowCount = 0l;
@@ -528,6 +534,7 @@ public class TransactionalAggregationClient {
    * columninterpreter says. So, this methods collects the necessary parameters
    * to compute the average and returs the double value.
    * @param transactionId
+   * @param startId
    * @param tableName
    * @param ci
    * @param scan
@@ -535,9 +542,9 @@ public class TransactionalAggregationClient {
    * @throws Throwable
    */
   public <R, S, P extends Message, Q extends Message, T extends Message>
-  double avg( final long transactionId, final TableName tableName,
+  double avg( final long transactionId, final long startId, final TableName tableName,
       final ColumnInterpreter<R, S, P, Q, T> ci, Scan scan) throws Throwable {
-    Pair<S, Long> p = getAvgArgs(transactionId, tableName, ci, scan);
+    Pair<S, Long> p = getAvgArgs(transactionId, startId, tableName, ci, scan);
     return ci.divideForAvg(p.getFirst(), p.getSecond());
   }
 
@@ -554,8 +561,8 @@ public class TransactionalAggregationClient {
    * @throws Throwable
    */
   public <R, S, P extends Message, Q extends Message, T extends Message> double avg(
-      final long transactionId, final TransactionalTable table, final ColumnInterpreter<R, S, P, Q, T> ci, Scan scan) throws Throwable {
-    Pair<S, Long> p = getAvgArgs(transactionId, table, ci, scan);
+      final long transactionId, final long startId, final TransactionalTable table, final ColumnInterpreter<R, S, P, Q, T> ci, Scan scan) throws Throwable {
+    Pair<S, Long> p = getAvgArgs(transactionId, startId, table, ci, scan);
     return ci.divideForAvg(p.getFirst(), p.getSecond());
   }
 
@@ -571,12 +578,12 @@ public class TransactionalAggregationClient {
    * @throws Throwable
    */
   private <R, S, P extends Message, Q extends Message, T extends Message>
-  Pair<List<S>, Long> getStdArgs(final long transactionId, final TransactionalTable table,
+  Pair<List<S>, Long> getStdArgs(final long transactionId,  final long startId, final TransactionalTable table,
       final ColumnInterpreter<R, S, P, Q, T> ci, final Scan scan) throws Throwable {
       byte [] currentBeginKey = scan.getStartRow();	
       HRegionInfo currentRegion = table.getRegionLocation(currentBeginKey).getRegionInfo();
       com.google.protobuf.ByteString regionName = ByteString.copyFromUtf8(currentRegion.getRegionNameAsString());
-    final TransactionalAggregateRequest requestArg = validateArgAndGetPB(regionName, transactionId, scan, ci, false);
+    final TransactionalAggregateRequest requestArg = validateArgAndGetPB(regionName, transactionId, startId, scan, ci, false);
     class StdCallback implements Batch.Callback<Pair<List<S>, Long>> {
       long rowCountVal = 0l;
       S sumVal = null, sumSqVal = null;
@@ -640,6 +647,7 @@ public class TransactionalAggregationClient {
    * columninterpreter says. So, this methods collects the necessary parameters
    * to compute the std and returns the double value.
    * @param transactionId
+   * @param startId
    * @param tableName
    * @param ci
    * @param scan
@@ -647,12 +655,12 @@ public class TransactionalAggregationClient {
    * @throws Throwable
    */
   public <R, S, P extends Message, Q extends Message, T extends Message>
-  double std( final long transactionId, final TableName tableName, ColumnInterpreter<R, S, P, Q, T> ci,
+  double std( final long transactionId, final long startId, final TableName tableName, ColumnInterpreter<R, S, P, Q, T> ci,
       Scan scan) throws Throwable {
     TransactionalTable table = null;
     try {
       table = new TransactionalTable(tableName.getName(), connection);
-      return std(transactionId, table, ci, scan);
+      return std(transactionId, startId, table, ci, scan);
     } finally {
       if (table != null) {
         table.close();
@@ -673,8 +681,8 @@ public class TransactionalAggregationClient {
    * @throws Throwable
    */
   public <R, S, P extends Message, Q extends Message, T extends Message> double std(
-      final long transactionId, final TransactionalTable table, ColumnInterpreter<R, S, P, Q, T> ci, Scan scan) throws Throwable {
-    Pair<List<S>, Long> p = getStdArgs(transactionId, table, ci, scan);
+      final long transactionId, final long startId, final TransactionalTable table, ColumnInterpreter<R, S, P, Q, T> ci, Scan scan) throws Throwable {
+    Pair<List<S>, Long> p = getStdArgs(transactionId, startId, table, ci, scan);
     double res = 0d;
     double avg = ci.divideForAvg(p.getFirst().get(0), p.getSecond());
     double avgOfSumSq = ci.divideForAvg(p.getFirst().get(1), p.getSecond());
@@ -697,12 +705,12 @@ public class TransactionalAggregationClient {
    */
   private <R, S, P extends Message, Q extends Message, T extends Message>
   Pair<NavigableMap<byte[], List<S>>, List<S>>
-  getMedianArgs(final long transactionId, final TransactionalTable table,
+  getMedianArgs(final long transactionId, final long startId, final TransactionalTable table,
       final ColumnInterpreter<R, S, P, Q, T> ci, final Scan scan) throws Throwable {
       byte [] currentBeginKey = scan.getStartRow();	
       HRegionInfo currentRegion = table.getRegionLocation(currentBeginKey).getRegionInfo();
       com.google.protobuf.ByteString regionName = ByteString.copyFromUtf8(currentRegion.getRegionNameAsString());
-    final TransactionalAggregateRequest requestArg = validateArgAndGetPB(regionName, transactionId, scan, ci, false);
+    final TransactionalAggregateRequest requestArg = validateArgAndGetPB(regionName, transactionId, startId, scan, ci, false);
     final NavigableMap<byte[], List<S>> map =
       new TreeMap<byte[], List<S>>(Bytes.BYTES_COMPARATOR);
     class StdCallback implements Batch.Callback<List<S>> {
@@ -757,6 +765,7 @@ public class TransactionalAggregationClient {
    * given cf-cq combination. This method collects the necessary parameters
    * to compute the median and returns the median.
    * @param transactionId
+   * @param startId
    * @param tableName
    * @param ci
    * @param scan
@@ -764,12 +773,12 @@ public class TransactionalAggregationClient {
    * @throws Throwable
    */
   public <R, S, P extends Message, Q extends Message, T extends Message>
-  R median( final long transactionId, final TableName tableName, ColumnInterpreter<R, S, P, Q, T> ci,
+  R median( final long transactionId, final long startId, final TableName tableName, ColumnInterpreter<R, S, P, Q, T> ci,
       Scan scan) throws Throwable {
     TransactionalTable table = null;
     try {
       table = new TransactionalTable(tableName.getName(), connection);
-      return median(transactionId, table, ci, scan);
+      return median(transactionId, startId, table, ci, scan);
     } finally {
       if (table != null) {
         table.close();
@@ -788,9 +797,9 @@ public class TransactionalAggregationClient {
    * @throws Throwable
    */
   public <R, S, P extends Message, Q extends Message, T extends Message>
-  R median(final long transactionId, final TransactionalTable table, ColumnInterpreter<R, S, P, Q, T> ci,
+  R median(final long transactionId, final long startId, final TransactionalTable table, ColumnInterpreter<R, S, P, Q, T> ci,
       Scan scan) throws Throwable {
-    Pair<NavigableMap<byte[], List<S>>, List<S>> p = getMedianArgs(transactionId, table, ci, scan);
+    Pair<NavigableMap<byte[], List<S>>, List<S>> p = getMedianArgs(transactionId, startId, table, ci, scan);
     byte[] startRow = null;
     byte[] colFamily = scan.getFamilies()[0];
     NavigableSet<byte[]> quals = scan.getFamilyMap().get(colFamily);
@@ -859,19 +868,21 @@ public class TransactionalAggregationClient {
   }
 
   <R, S, P extends Message, Q extends Message, T extends Message> TransactionalAggregateRequest 
-  validateArgAndGetPB(com.google.protobuf.ByteString regionName, long transactionId, Scan scan, ColumnInterpreter<R,S,P,Q,T> ci, boolean canFamilyBeAbsent)
+  validateArgAndGetPB(com.google.protobuf.ByteString regionName, long transactionId, final long startId, Scan scan, ColumnInterpreter<R,S,P,Q,T> ci, boolean canFamilyBeAbsent)
       throws IOException {
     validateParameters(scan, canFamilyBeAbsent);
     final TransactionalAggregateRequest.Builder requestBuilder = 
         TransactionalAggregateRequest.newBuilder();
     requestBuilder.setInterpreterClassName(ci.getClass().getCanonicalName());
     P columnInterpreterSpecificData = null;
+    TransactionState ts = null;
     if ((columnInterpreterSpecificData = ci.getRequestData()) 
        != null) {
       requestBuilder.setInterpreterSpecificBytes(columnInterpreterSpecificData.toByteString());
     }
     requestBuilder.setScan(ProtobufUtil.toScan(scan));
     requestBuilder.setTransactionId(transactionId);
+    requestBuilder.setStartId(startId);
     requestBuilder.setRegionName(regionName);    
     return requestBuilder.build();
   }

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/TransactionalScanner.java.tmpl
----------------------------------------------------------------------
diff --git a/core/sqf/src/seatrans/hbase-trx/src/main/java/org/apache/hadoop/hbase/client/transactional/TransactionalScanner.java.tmpl b/core/sqf/src/seatrans/hbase-trx/src/main/java/org/apache/hadoop/hbase/client/transactional/TransactionalScanner.java.tmpl
index af7ed70..35dc84a 100755
--- a/core/sqf/src/seatrans/hbase-trx/src/main/java/org/apache/hadoop/hbase/client/transactional/TransactionalScanner.java.tmpl
+++ b/core/sqf/src/seatrans/hbase-trx/src/main/java/org/apache/hadoop/hbase/client/transactional/TransactionalScanner.java.tmpl
@@ -64,7 +64,7 @@ public class TransactionalScanner extends AbstractClientScanner {
     public TransactionalTable ttable;
     protected boolean closed = false;
     // Experiment with this parameter, may be faster without having to send the final close()
-    protected boolean doNotCloseOnLast = true;
+    protected boolean regionShouldNotCloseOnLast = true;
     protected int nbRows = 100;
     protected long nextCallSeq = 0;
     private boolean hasMore = true;
@@ -141,21 +141,26 @@ public class TransactionalScanner extends AbstractClientScanner {
     protected boolean nextScanner(final boolean done) throws IOException{
         if(LOG.isTraceEnabled()) LOG.trace("nextScanner() -- ENTRY txID: " + ts.getTransactionId());
         if(this.currentBeginKey != null) {
-            if(LOG.isTraceEnabled()) LOG.trace("nextScanner() currentBeginKey != null txID: " + ts.getTransactionId());
-            if (doNotCloseOnLast)
+            if(LOG.isTraceEnabled()) LOG.trace("nextScanner() currentBeginKey != null txID: " + ts.getTransactionId()
+                    + " currentBeginKey=" + (Bytes.equals(this.currentBeginKey, HConstants.EMPTY_START_ROW) ?
+                      "INFINITE" : Hex.encodeHexString(this.currentBeginKey)));
+            if (regionShouldNotCloseOnLast){ // if the region won't automatically we need to tell it to close
+              if(LOG.isTraceEnabled()) LOG.trace("nextScanner() regionShouldNotCloseOnLast is true; closing scanner");
               close();
+            }
             if((this.currentEndKey == HConstants.EMPTY_END_ROW) || 
                 Bytes.equals(this.currentEndKey, HConstants.EMPTY_BYTE_ARRAY) ||
                 checkScanStopRow(this.currentEndKey) || 
                 done) {
-                if(LOG.isTraceEnabled()) LOG.trace("endKey: " + Bytes.toString(this.currentEndKey));
+                if(LOG.isTraceEnabled()) LOG.trace("endKey: " + (Bytes.equals(this.currentEndKey, HConstants.EMPTY_END_ROW) ?
+                      "INFINITE" : Hex.encodeHexString(this.currentEndKey)));
                 if(LOG.isTraceEnabled()) LOG.trace("nextScanner() -- EXIT -- returning false txID: " + ts.getTransactionId());
                 this.moreScanners = false;
                 return false;
             }
             else
-                //this.currentBeginKey = TransactionManager.binaryIncrementPos(this.currentEndKey,1);
-                  this.currentBeginKey = this.currentEndKey;
+                this.currentBeginKey = TransactionManager.binaryIncrementPos(this.currentEndKey,1);
+                //this.currentBeginKey = this.currentEndKey;
         }
         else {
             // First call to nextScanner
@@ -165,15 +170,25 @@ public class TransactionalScanner extends AbstractClientScanner {
         this.currentRegion = ttable.getRegionLocation(this.currentBeginKey, false).getRegionInfo();
         this.currentEndKey = this.currentRegion.getEndKey();
 
-        if(LOG.isTraceEnabled()) LOG.trace("nextScanner() txID: " + ts.getTransactionId() + " Region Info: " + currentRegion.getRegionNameAsString()+ 
-                         "currentBeginKey: " + Hex.encodeHexString(this.currentBeginKey) + "currentEndKey: " + Hex.encodeHexString(this.currentEndKey));
-        //if(this.currentEndKey != HConstants.EMPTY_END_ROW)
-        //   this.currentEndKey = TransactionManager.binaryIncrementPos(currentRegion.getEndKey(), -1);
+        if(LOG.isTraceEnabled()) LOG.trace("nextScanner() txID: " + ts.getTransactionId() + " Region Info: " + currentRegion.getRegionNameAsString()
+              + "currentBeginKey: " + (Bytes.equals(this.currentBeginKey, HConstants.EMPTY_START_ROW) ?
+                      "INFINITE" : Hex.encodeHexString(this.currentBeginKey))
+              + " currentEndKey: " + (Bytes.equals(this.currentEndKey, HConstants.EMPTY_END_ROW) ?
+                      "INFINITE" : Hex.encodeHexString(this.currentEndKey)));
+        if ( ! (Bytes.equals(this.currentEndKey, HConstants.EMPTY_END_ROW)))
+           this.currentEndKey = TransactionManager.binaryIncrementPos(currentRegion.getEndKey(), -1);
 
         this.closed = false;
 
+        // Need to add the next region as a participant to the transaction so it gets phase 1 notification
+        // and does not orphan the TransactionState object in the region that gets created during the scan
+        if (LOG.isTraceEnabled()) LOG.trace("nextScanner() calling RMInterface.registerTransaction for startKey "
+                  +  Hex.encodeHexString(this.currentBeginKey));
+        RMInterface.registerTransaction(ttable, ts, this.currentBeginKey);
+
       TrxRegionProtos.OpenScannerRequest.Builder requestBuilder = OpenScannerRequest.newBuilder();
       requestBuilder.setTransactionId(ts.getTransactionId());
+      requestBuilder.setStartId(ts.getStartId());
       requestBuilder.setRegionName(ByteString.copyFromUtf8(currentRegion.getRegionNameAsString()));
       requestBuilder.setScan(ProtobufUtil.toScan(scan));
       TrxRegionProtos.OpenScannerRequest openRequest = requestBuilder.build();
@@ -212,10 +227,11 @@ public class TransactionalScanner extends AbstractClientScanner {
                 final long nextCallSeqInput = this.nextCallSeq;
                 TrxRegionProtos.PerformScanRequest.Builder requestBuilder = PerformScanRequest.newBuilder();
                 requestBuilder.setTransactionId(ts.getTransactionId());
+                requestBuilder.setStartId(ts.getStartId());
                 requestBuilder.setRegionName(ByteString.copyFromUtf8(currentRegion.getRegionNameAsString()));
                 requestBuilder.setScannerId(scannerID);
                 requestBuilder.setNumberOfRows(nbRows);
-                if (doNotCloseOnLast)
+                if (regionShouldNotCloseOnLast)
                     requestBuilder.setCloseScanner(false);
                 else
                     requestBuilder.setCloseScanner(true);

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/TransactionalTable.java
----------------------------------------------------------------------
diff --git a/core/sqf/src/seatrans/hbase-trx/src/main/java/org/apache/hadoop/hbase/client/transactional/TransactionalTable.java b/core/sqf/src/seatrans/hbase-trx/src/main/java/org/apache/hadoop/hbase/client/transactional/TransactionalTable.java
index a7d99c0..322af24 100644
--- a/core/sqf/src/seatrans/hbase-trx/src/main/java/org/apache/hadoop/hbase/client/transactional/TransactionalTable.java
+++ b/core/sqf/src/seatrans/hbase-trx/src/main/java/org/apache/hadoop/hbase/client/transactional/TransactionalTable.java
@@ -38,16 +38,15 @@ import java.util.List;
 import java.util.Map;
 import java.util.concurrent.ExecutorService;
 import java.util.concurrent.Executors;
-import org.apache.hadoop.hbase.client.RetriesExhaustedWithDetailsException;
 
 import org.apache.commons.codec.binary.Hex;
-
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.HBaseConfiguration;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.Cell;
+import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.HRegionLocation;
 import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.KeyValueUtil;
@@ -58,6 +57,7 @@ import org.apache.hadoop.hbase.client.HTable;
 import org.apache.hadoop.hbase.client.Put;
 import org.apache.hadoop.hbase.client.Result;
 import org.apache.hadoop.hbase.client.ResultScanner;
+import org.apache.hadoop.hbase.client.RetriesExhaustedWithDetailsException;
 import org.apache.hadoop.hbase.client.Scan;
 import org.apache.hadoop.hbase.client.TrafParallelClientScanner;
 import org.apache.hadoop.hbase.client.coprocessor.Batch;
@@ -107,12 +107,15 @@ import java.util.concurrent.ThreadPoolExecutor;
  * Table with transactional support.
  */
 public class TransactionalTable extends HTable implements TransactionalTableClient {
-    static final Log LOG = LogFactory.getLog(RMInterface.class);
+    static final Log LOG = LogFactory.getLog(TransactionalTable.class);
+    static Configuration       config;
     static private Connection connection = null;
     static ThreadPoolExecutor threadPool = null;
     static int                 retries = 15;
     static int                 delay = 1000;
-    private String retryErrMsg = "Coprocessor result is null, retries exhausted";
+    static int                 regionNotReadyDelay = 30000;
+
+    private String retryErrMsg = "Coprocessor result is null, retries exhausted for table " + this.getName().getNameAsString();
 
     static {
        Configuration config = HBaseConfiguration.create();
@@ -176,6 +179,7 @@ public class TransactionalTable extends HTable implements TransactionalTableClie
         org.apache.hadoop.hbase.coprocessor.transactional.generated.TrxRegionProtos.GetTransactionalRequest.Builder builder = GetTransactionalRequest.newBuilder();            
         builder.setGet(ProtobufUtil.toGet(get));
         builder.setTransactionId(transactionState.getTransactionId());
+        builder.setStartId(transactionState.getStartId());
         builder.setRegionName(ByteString.copyFromUtf8(regionName));
    
         instance.get(controller, builder.build(), rpcCallback);
@@ -198,8 +202,17 @@ public class TransactionalTable extends HTable implements TransactionalTableClie
             retry = false;
           } 
 
-          if(result == null || result.getException().contains("closing region")) {
-            Thread.sleep(TransactionalTable.delay);
+          if(result == null || result.getException().contains("closing region")
+                            || result.getException().contains("NewTransactionStartedBefore")) {
+            if (result != null && result.getException().contains("NewTransactionStartedBefore")) {
+               if (LOG.isTraceEnabled()) LOG.trace("Get retrying because region is recovering,"
+                      + " transaction [" + transactionState.getTransactionId() + "]");
+
+               Thread.sleep(TransactionalTable.regionNotReadyDelay);
+            }
+            else{
+               Thread.sleep(TransactionalTable.delay);
+            }
             retry = true;
             transactionState.setRetried(true);
             retryCount++;
@@ -242,6 +255,7 @@ public class TransactionalTable extends HTable implements TransactionalTableClie
           public DeleteTransactionalResponse call(TrxRegionService instance) throws IOException {
             org.apache.hadoop.hbase.coprocessor.transactional.generated.TrxRegionProtos.DeleteTransactionalRequest.Builder builder = DeleteTransactionalRequest.newBuilder();      
             builder.setTransactionId(transactionState.getTransactionId());
+            builder.setStartId(transactionState.getStartId());
             builder.setRegionName(ByteString.copyFromUtf8(regionName));
             
             MutationProto m1 = ProtobufUtil.toMutation(MutationType.DELETE, delete);
@@ -268,8 +282,17 @@ public class TransactionalTable extends HTable implements TransactionalTableClie
               retry = false;
             }
 
-            if(result == null || result.getException().contains("closing region")) {
-              Thread.sleep(TransactionalTable.delay);
+            if(result == null || result.getException().contains("closing region")
+                              || result.getException().contains("NewTransactionStartedBefore")) {
+              if (result != null && result.getException().contains("NewTransactionStartedBefore")) {
+                 if (LOG.isTraceEnabled()) LOG.trace("Delete retrying because region is recovering,"
+                          + " transaction [" + transactionState.getTransactionId() + "]");
+
+                 Thread.sleep(TransactionalTable.regionNotReadyDelay);
+              }
+              else{
+                 Thread.sleep(TransactionalTable.delay);
+              }
               retry = true;
               transactionState.setRetried(true);
               retryCount++;
@@ -286,7 +309,8 @@ public class TransactionalTable extends HTable implements TransactionalTableClie
     }
 
     public void deleteRegionTx(final long tid, final Delete delete, final boolean autoCommit) throws IOException {
-  	if (LOG.isTraceEnabled()) LOG.trace("TransactionalTable.deleteRegionTx ENTRY, autoCommit: " + autoCommit);
+        if (LOG.isTraceEnabled()) LOG.trace("TransactionalTable.deleteRegionTx ENTRY, autoCommit: "
+                + autoCommit);
         SingleVersionDeleteNotSupported.validateDelete(delete);
         final String regionName = super.getRegionLocation(delete.getRow()).getRegionInfo().getRegionNameAsString();
 
@@ -300,6 +324,7 @@ public class TransactionalTable extends HTable implements TransactionalTableClie
           public DeleteRegionTxResponse call(TrxRegionService instance) throws IOException {
             org.apache.hadoop.hbase.coprocessor.transactional.generated.TrxRegionProtos.DeleteRegionTxRequest.Builder builder = DeleteRegionTxRequest.newBuilder();
             builder.setTid(tid);
+            builder.setCommitId(-1);
             builder.setAutoCommit(autoCommit);
             builder.setRegionName(ByteString.copyFromUtf8(regionName));
 
@@ -358,7 +383,6 @@ public class TransactionalTable extends HTable implements TransactionalTableClie
     }
 
     public synchronized void put(final TransactionState transactionState, final Put put, final boolean bool_addLocation) throws IOException {
-      validatePut(put);
     	if (LOG.isTraceEnabled()) LOG.trace("TransactionalTable.put ENTRY");
     
       if (bool_addLocation) addLocation(transactionState, super.getRegionLocation(put.getRow()));
@@ -373,6 +397,7 @@ public class TransactionalTable extends HTable implements TransactionalTableClie
       public PutTransactionalResponse call(TrxRegionService instance) throws IOException {
         org.apache.hadoop.hbase.coprocessor.transactional.generated.TrxRegionProtos.PutTransactionalRequest.Builder builder = PutTransactionalRequest.newBuilder();
         builder.setTransactionId(transactionState.getTransactionId());
+        builder.setStartId(transactionState.getStartId());
         builder.setRegionName(ByteString.copyFromUtf8(regionName));
   
         
@@ -398,8 +423,17 @@ public class TransactionalTable extends HTable implements TransactionalTableClie
           retry = false;
         }
 
-        if(result == null || result.getException().contains("closing region")) {
-          Thread.sleep(TransactionalTable.delay);
+        if(result == null || result.getException().contains("closing region")
+                          || result.getException().contains("NewTransactionStartedBefore")) {
+          if (result != null && result.getException().contains("NewTransactionStartedBefore")) {
+             if (LOG.isTraceEnabled()) LOG.trace("Put retrying because region is recovering,"
+                      + " transaction [" + transactionState.getTransactionId() + "]");
+
+             Thread.sleep(TransactionalTable.regionNotReadyDelay);
+          }
+          else{
+             Thread.sleep(TransactionalTable.delay);
+          }
           retry = true;
           transactionState.setRetried(true);
           retryCount++;
@@ -420,9 +454,9 @@ public class TransactionalTable extends HTable implements TransactionalTableClie
   }
 
    public synchronized void putRegionTx(final long tid, final Put put, final boolean autoCommit) throws IOException{
-        if (LOG.isTraceEnabled()) LOG.trace("TransactionalTable.putRegionTx ENTRY, autoCommit: " + autoCommit);
+        if (LOG.isTraceEnabled()) LOG.trace("TransactionalTable.putRegionTx ENTRY, autoCommit: "
+               + autoCommit);
 
-        validatePut(put);
         final String regionName = super.getRegionLocation(put.getRow()).getRegionInfo().getRegionNameAsString();
 
         Batch.Call<TrxRegionService, PutRegionTxResponse> callable =
@@ -434,6 +468,7 @@ public class TransactionalTable extends HTable implements TransactionalTableClie
         public PutRegionTxResponse call(TrxRegionService instance) throws IOException {
           org.apache.hadoop.hbase.coprocessor.transactional.generated.TrxRegionProtos.PutRegionTxRequest.Builder builder = PutRegionTxRequest.newBuilder();
           builder.setTid(tid);
+          builder.setCommitId(-1);
           builder.setAutoCommit(autoCommit);
           builder.setRegionName(ByteString.copyFromUtf8(regionName));
           MutationProto m1 = ProtobufUtil.toMutation(MutationType.PUT, put);
@@ -480,13 +515,21 @@ public class TransactionalTable extends HTable implements TransactionalTableClie
   }
 
   public synchronized ResultScanner getScanner(final TransactionState transactionState, final Scan scan) throws IOException {
-    if (LOG.isTraceEnabled()) LOG.trace("Enter TransactionalTable.getScanner");
+    if (LOG.isTraceEnabled()) LOG.trace("Enter TransactionalTable.getScanner for transaction " + transactionState.getTransactionId() + " scan ");
     if (scan.getCaching() <= 0) {
         scan.setCaching(getScannerCaching());
     }
     
     Long value = (long) 0;
     TransactionalScanner scanner = new TransactionalScanner(this, transactionState, scan, value);
+    if (LOG.isTraceEnabled()){
+        String startRow = (Bytes.equals(scan.getStartRow(), HConstants.EMPTY_START_ROW) ?
+                "INFINITE" : Hex.encodeHexString(scan.getStartRow()));
+        String stopRow = (Bytes.equals(scan.getStopRow(), HConstants.EMPTY_END_ROW) ?
+                "INFINITE" : Hex.encodeHexString(scan.getStopRow()));
+       LOG.trace("Exit TransactionalTable.getScanner for transaction "
+            + transactionState.getTransactionId() + " scan startRow=" + startRow + ", stopRow=" + stopRow);
+    }
     
     return scanner;         
   }
@@ -495,7 +538,8 @@ public class TransactionalTable extends HTable implements TransactionalTableClie
 		  final byte[] family, final byte[] qualifier, final byte[] value,
           final Delete delete, final boolean autoCommit) throws IOException {
     if (LOG.isTraceEnabled()) LOG.trace("Enter TransactionalTable.checkAndDeleteRegionTx row: " + row
-    		+ " family: " + family + " qualifier: " + qualifier + " value: " + value);
+                + " family: " + family + " qualifier: " + qualifier + " value: " + value
+                + " autoCommit: " + autoCommit);
     if (!Bytes.equals(row, delete.getRow())) {
        throw new IOException("checkAndDeleteRegionTx action's getRow must match the passed row");
     }
@@ -512,6 +556,7 @@ public class TransactionalTable extends HTable implements TransactionalTableClie
     public CheckAndDeleteRegionTxResponse call(TrxRegionService instance) throws IOException {
         org.apache.hadoop.hbase.coprocessor.transactional.generated.TrxRegionProtos.CheckAndDeleteRegionTxRequest.Builder builder = CheckAndDeleteRegionTxRequest.newBuilder();
         builder.setTid(tid);
+        builder.setCommitId(-1);
         builder.setRegionName(ByteString.copyFromUtf8(regionName));
         builder.setRow(HBaseZeroCopyByteString.wrap(row));
         builder.setAutoCommit(autoCommit);
@@ -589,6 +634,7 @@ public class TransactionalTable extends HTable implements TransactionalTableClie
       public CheckAndDeleteResponse call(TrxRegionService instance) throws IOException {
         org.apache.hadoop.hbase.coprocessor.transactional.generated.TrxRegionProtos.CheckAndDeleteRequest.Builder builder = CheckAndDeleteRequest.newBuilder();
         builder.setTransactionId(transactionState.getTransactionId());
+        builder.setStartId(transactionState.getStartId());
         builder.setRegionName(ByteString.copyFromUtf8(regionName));
         builder.setRow(HBaseZeroCopyByteString.wrap(row));
         if(family != null)
@@ -629,8 +675,17 @@ public class TransactionalTable extends HTable implements TransactionalTableClie
             retry = false;
           }
 
-          if(result == null || result.getException().contains("closing region")) {
-            Thread.sleep(TransactionalTable.delay);
+          if(result == null || result.getException().contains("closing region")
+                            || result.getException().contains("NewTransactionStartedBefore")) {
+            if (result != null && result.getException().contains("NewTransactionStartedBefore")) {
+               if (LOG.isTraceEnabled()) LOG.trace("CheckAndDelete retrying because region is recovering, "
+                             + " transaction [" + transactionState.getTransactionId() + "]");
+
+               Thread.sleep(TransactionalTable.regionNotReadyDelay);
+            }
+            else{
+               Thread.sleep(TransactionalTable.delay);
+            }
             retry = true;
             transactionState.setRetried(true);
             retryCount++;
@@ -651,8 +706,8 @@ public class TransactionalTable extends HTable implements TransactionalTableClie
 			final byte[] row, final byte[] family, final byte[] qualifier,
 			final byte[] value, final Put put) throws IOException {
 
-		if (LOG.isTraceEnabled()) LOG.trace("Enter TransactionalTable.checkAndPut row: " + row
-				+ " family: " + family + " qualifier: " + qualifier
+		if (LOG.isTraceEnabled()) LOG.trace("Enter TransactionalTable.checkAndPut row: " + Hex.encodeHexString(row)
+				+ " put.row " + Hex.encodeHexString(put.getRow()) + " family: " + family + " qualifier: " + qualifier
 				+ " value: " + value);
 		if (!Bytes.equals(row, put.getRow())) {
 			throw new IOException("Action's getRow must match the passed row");
@@ -672,6 +727,8 @@ public class TransactionalTable extends HTable implements TransactionalTableClie
       public CheckAndPutResponse call(TrxRegionService instance) throws IOException {
         org.apache.hadoop.hbase.coprocessor.transactional.generated.TrxRegionProtos.CheckAndPutRequest.Builder builder = CheckAndPutRequest.newBuilder();
         builder.setTransactionId(transactionState.getTransactionId());
+        if (LOG.isTraceEnabled()) LOG.trace("checkAndPut, seting request startid: " + transactionState.getStartId());
+        builder.setStartId(transactionState.getStartId());
         builder.setRegionName(ByteString.copyFromUtf8(regionName));
         builder.setRow(HBaseZeroCopyByteString.wrap(row));
         if (family != null)
@@ -711,8 +768,17 @@ public class TransactionalTable extends HTable implements TransactionalTableClie
             retry = false;
           }
 
-          if(result == null || result.getException().contains("closing region")) {
-            Thread.sleep(TransactionalTable.delay);
+          if(result == null || result.getException().contains("closing region")
+                            || result.getException().contains("NewTransactionStartedBefore")) {
+            if (result != null && result.getException().contains("NewTransactionStartedBefore")) {
+               if (LOG.isTraceEnabled()) LOG.trace("CheckAndPut retrying because region is recovering ,"
+                       + " transaction [" + transactionState.getTransactionId() + "]");
+
+               Thread.sleep(TransactionalTable.regionNotReadyDelay);
+            }
+            else{
+               Thread.sleep(TransactionalTable.delay);
+            }
             retry = true;
             transactionState.setRetried(true);
             retryCount++;
@@ -775,6 +841,7 @@ public class TransactionalTable extends HTable implements TransactionalTableClie
 	      public CheckAndPutRegionTxResponse call(TrxRegionService instance) throws IOException {
 	        org.apache.hadoop.hbase.coprocessor.transactional.generated.TrxRegionProtos.CheckAndPutRegionTxRequest.Builder builder = CheckAndPutRegionTxRequest.newBuilder();
 	        builder.setTid(tid);
+            builder.setCommitId(-1);
 	        builder.setRegionName(ByteString.copyFromUtf8(regionName));
 	        builder.setRow(HBaseZeroCopyByteString.wrap(row));
 	        if (family != null)
@@ -881,6 +948,7 @@ public class TransactionalTable extends HTable implements TransactionalTableClie
    	      public DeleteMultipleTransactionalResponse call(TrxRegionService instance) throws IOException {
    	        org.apache.hadoop.hbase.coprocessor.transactional.generated.TrxRegionProtos.DeleteMultipleTransactionalRequest.Builder builder = DeleteMultipleTransactionalRequest.newBuilder();
    	        builder.setTransactionId(transactionState.getTransactionId());
+            builder.setStartId(transactionState.getStartId());
    	        builder.setRegionName(ByteString.copyFromUtf8(regionName));
 
    	        for(Delete delete : rowsInSameRegion) {
@@ -908,8 +976,19 @@ public class TransactionalTable extends HTable implements TransactionalTableClie
  	            retry = false;
  	          }
 
- 	          if(result == null || result.getException().contains("closing region")) {
- 	            Thread.sleep(TransactionalTable.delay);
+                if(result == null || result.getException().contains("closing region")
+                     || result.getException().contains("NewTransactionStartedBefore")) {
+                if (result != null && result.getException().contains("NewTransactionStartedBefore")) {
+                   if (LOG.isTraceEnabled()) LOG.trace("delete <List> retrying because region is recovering trRegion ["
+                           + location.getRegionInfo().getEncodedName() + "], endKey: "
+                           + Hex.encodeHexString(location.getRegionInfo().getEndKey())
+                           + " and transaction [" + transactionId + "]");
+
+                   Thread.sleep(TransactionalTable.regionNotReadyDelay);
+                }
+                else{
+                   Thread.sleep(TransactionalTable.delay);
+                }
  	            retry = true;
  	            transactionState.setRetried(true);
  	            retryCount++;
@@ -946,7 +1025,6 @@ public class TransactionalTable extends HTable implements TransactionalTableClie
 		List<Put> list = null;
                 int size = 0;
 		for (Put put : puts) {
-			validatePut(put);
 			hlocation = this.getRegionLocation(put.getRow(), false);
                         location = new TransactionRegionLocation(hlocation.getRegionInfo(), hlocation.getServerName());
                 if (LOG.isTraceEnabled()) LOG.trace("put <List> with trRegion [" + location.getRegionInfo().getEncodedName() + "], endKey: "
@@ -976,6 +1054,7 @@ public class TransactionalTable extends HTable implements TransactionalTableClie
 	      public PutMultipleTransactionalResponse call(TrxRegionService instance) throws IOException {
 	        org.apache.hadoop.hbase.coprocessor.transactional.generated.TrxRegionProtos.PutMultipleTransactionalRequest.Builder builder = PutMultipleTransactionalRequest.newBuilder();
 	        builder.setTransactionId(transactionState.getTransactionId());
+            builder.setStartId(transactionState.getStartId());
 	        builder.setRegionName(ByteString.copyFromUtf8(regionName));
 
 	        for (Put put : rowsInSameRegion){
@@ -1002,8 +1081,19 @@ public class TransactionalTable extends HTable implements TransactionalTableClie
             retry = false;
           }
 
-          if(result == null || result.getException().contains("closing region")) {
-            Thread.sleep(TransactionalTable.delay);
+          if(result == null || result.getException().contains("closing region")
+                       || result.getException().contains("NewTransactionStartedBefore")) {
+            if (result != null && result.getException().contains("NewTransactionStartedBefore")) {
+               if (LOG.isTraceEnabled()) LOG.trace("put <List> retrying because region is recovering trRegion ["
+                      + location.getRegionInfo().getEncodedName() + "], endKey: "
+                      + Hex.encodeHexString(location.getRegionInfo().getEndKey())
+                      + " and transaction [" + transactionId + "]");
+
+               Thread.sleep(TransactionalTable.regionNotReadyDelay);
+            }
+            else{
+                Thread.sleep(TransactionalTable.delay);
+            }
             retry = true;
             transactionState.setRetried(true);
             retryCount++;
@@ -1020,24 +1110,6 @@ public class TransactionalTable extends HTable implements TransactionalTableClie
      }
 		}
 	
-    // validate for well-formedness
-    public void validatePut(final Put put) throws IllegalArgumentException {
-        if (put.isEmpty()) {
-            throw new IllegalArgumentException("No columns to insert");
-        }
-        if (maxKeyValueSize > 0) {
-            for (List<Cell> list : put.getFamilyCellMap().values()) {
-                for (Cell c : list) {
-                    if (KeyValueUtil.length(c) > maxKeyValueSize) {
-                        throw new IllegalArgumentException("KeyValue size too large");
-                    }
-                }
-            }
-        }
-    }
-
-    private int maxKeyValueSize;
-
     public HRegionLocation getRegionLocation(byte[] row, boolean f)
                                   throws IOException {
         return super.getRegionLocation(row,f);


[21/22] incubator-trafodion git commit: Merged seabase/EXPECTED022

Posted by sa...@apache.org.
Merged seabase/EXPECTED022


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

Branch: refs/heads/master
Commit: 74b206373f7bfd5dc23a2e763463077a8616d587
Parents: 7224c05
Author: Sean Broeder <sb...@edev05.esgyn.local>
Authored: Fri May 12 16:11:44 2017 +0000
Committer: Sean Broeder <sb...@edev05.esgyn.local>
Committed: Fri May 12 16:11:44 2017 +0000

----------------------------------------------------------------------
 core/sql/regress/seabase/EXPECTED022 | 32 +++++++++++++++++++++++++++----
 1 file changed, 28 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-trafodion/blob/74b20637/core/sql/regress/seabase/EXPECTED022
----------------------------------------------------------------------
diff --git a/core/sql/regress/seabase/EXPECTED022 b/core/sql/regress/seabase/EXPECTED022
index 64a6e30..37e50e3 100644
--- a/core/sql/regress/seabase/EXPECTED022
+++ b/core/sql/regress/seabase/EXPECTED022
@@ -635,8 +635,8 @@ ROW_ID (EXPR)
 --- SQL operation complete.
 >>invoke t022hbm1;
 
--- Definition of Trafodion table TRAFODION.SCH.T022HBM1
--- Definition current  Thu May 11 22:43:35 2017
+-- Definition of table #CAT.#SCH.T022HBM1
+-- Definition current
 
  (
  SYSKEY LARGEINT NO DEFAULT NOT NULL NOT DROPPABLE
@@ -720,8 +720,8 @@ a2 ?
 --- SQL operation complete.
 >>invoke t022hbm1_like;
 
--- Definition of Trafodion table TRAFODION.SCH.T022HBM1_LIKE
--- Definition current  Thu May 11 22:43:57 2017
+-- Definition of table #CAT.#SCH.T022HBM1_LIKE
+-- Definition current
 
  (
  "cf".A VARCHAR(4) CHARACTER SET ISO88591 COLLATE
@@ -1031,6 +1031,30 @@ A B C
 -- Definition current
 
  (
+ "cf".A VARCHAR(4) CHARACTER SET ISO88591 COLLATE
+ DEFAULT NO @eof@ NULL NOT DROPPABLE
+ , "cf".B INT DEFAULT NULL
+ )
+ PRIMARY KEY NOT SERIALIZED (A ASC)
+
+--- SQL operation complete.
+>>
+>>drop external table t022hbm1;
+
+--- SQL operation complete.
+>>create external table t022hbm1 (a char(4) not null, b int)
++> primary key serialized (a)
++> attribute default column family 'cf'
++> map to hbase table t022hbm1
++> data format native;
+
+--- SQL operation complete.
+>>invoke t022hbm1;
+
+-- Definition of TRAFODION HBase mapped table T022HBM1
+-- Definition current
+
+ (
  "cf".A CHAR(4) CHARACTER SET ISO88591 COLLATE
  DEFAULT NO @eof@ NULL NOT DROPPABLE
  , "cf".B INT DEFAULT NULL


[10/22] incubator-trafodion git commit: Added copyright to the generated protobufs

Posted by sa...@apache.org.
Added copyright to the generated protobufs


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

Branch: refs/heads/master
Commit: 19edeb4ba35c2725d415b947314d25e5f5f6a687
Parents: 91794b5
Author: Sean Broeder <sb...@edev05.novalocal>
Authored: Mon May 1 17:26:38 2017 +0000
Committer: Sean Broeder <sb...@edev05.novalocal>
Committed: Mon May 1 17:26:38 2017 +0000

----------------------------------------------------------------------
 .../generated/SsccRegionProtos.java             | 23 ++++++++++++++++++++
 .../generated/TrxRegionProtos.java              | 23 ++++++++++++++++++++
 2 files changed, 46 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-trafodion/blob/19edeb4b/core/sqf/src/seatrans/hbase-trx/src/main/java/org/apache/hadoop/hbase/coprocessor/transactional/generated/SsccRegionProtos.java
----------------------------------------------------------------------
diff --git a/core/sqf/src/seatrans/hbase-trx/src/main/java/org/apache/hadoop/hbase/coprocessor/transactional/generated/SsccRegionProtos.java b/core/sqf/src/seatrans/hbase-trx/src/main/java/org/apache/hadoop/hbase/coprocessor/transactional/generated/SsccRegionProtos.java
index d2782a0..dbbbba7 100644
--- a/core/sqf/src/seatrans/hbase-trx/src/main/java/org/apache/hadoop/hbase/coprocessor/transactional/generated/SsccRegionProtos.java
+++ b/core/sqf/src/seatrans/hbase-trx/src/main/java/org/apache/hadoop/hbase/coprocessor/transactional/generated/SsccRegionProtos.java
@@ -1,3 +1,26 @@
+/**
+ * * @@@ START COPYRIGHT @@@
+ * *
+ * * Licensed to the Apache Software Foundation (ASF) under one
+ * * or more contributor license agreements.  See the NOTICE file
+ * * distributed with this work for additional information
+ * * regarding copyright ownership.  The ASF licenses this file
+ * * to you under the Apache License, Version 2.0 (the
+ * * "License"); you may not use this file except in compliance
+ * * with the License.  You may obtain a copy of the License at
+ * *
+ * *   http://www.apache.org/licenses/LICENSE-2.0
+ * *
+ * * Unless required by applicable law or agreed to in writing,
+ * * software distributed under the License is distributed on an
+ * * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * * KIND, either express or implied.  See the License for the
+ * * specific language governing permissions and limitations
+ * * under the License.
+ * *
+ * * @@@ END COPYRIGHT @@@
+ * **/
+
 // Generated by the protocol buffer compiler.  DO NOT EDIT!
 // source: SsccRegion.proto
 

http://git-wip-us.apache.org/repos/asf/incubator-trafodion/blob/19edeb4b/core/sqf/src/seatrans/hbase-trx/src/main/java/org/apache/hadoop/hbase/coprocessor/transactional/generated/TrxRegionProtos.java
----------------------------------------------------------------------
diff --git a/core/sqf/src/seatrans/hbase-trx/src/main/java/org/apache/hadoop/hbase/coprocessor/transactional/generated/TrxRegionProtos.java b/core/sqf/src/seatrans/hbase-trx/src/main/java/org/apache/hadoop/hbase/coprocessor/transactional/generated/TrxRegionProtos.java
index 019297b..bace23f 100755
--- a/core/sqf/src/seatrans/hbase-trx/src/main/java/org/apache/hadoop/hbase/coprocessor/transactional/generated/TrxRegionProtos.java
+++ b/core/sqf/src/seatrans/hbase-trx/src/main/java/org/apache/hadoop/hbase/coprocessor/transactional/generated/TrxRegionProtos.java
@@ -1,3 +1,26 @@
+/**
+ * * @@@ START COPYRIGHT @@@
+ * *
+ * * Licensed to the Apache Software Foundation (ASF) under one
+ * * or more contributor license agreements.  See the NOTICE file
+ * * distributed with this work for additional information
+ * * regarding copyright ownership.  The ASF licenses this file
+ * * to you under the Apache License, Version 2.0 (the
+ * * "License"); you may not use this file except in compliance
+ * * with the License.  You may obtain a copy of the License at
+ * *
+ * *   http://www.apache.org/licenses/LICENSE-2.0
+ * *
+ * * Unless required by applicable law or agreed to in writing,
+ * * software distributed under the License is distributed on an
+ * * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * * KIND, either express or implied.  See the License for the
+ * * specific language governing permissions and limitations
+ * * under the License.
+ * *
+ * * @@@ END COPYRIGHT @@@
+ * **/
+
 // Generated by the protocol buffer compiler.  DO NOT EDIT!
 // source: TrxRegion.proto
 


[11/22] incubator-trafodion git commit: Merge fixes

Posted by sa...@apache.org.
Merge fixes


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

Branch: refs/heads/master
Commit: 4d378a6c365f27415dc06f6cf62904bb3be7b47f
Parents: 19edeb4
Author: Sean Broeder <sb...@edev05.novalocal>
Authored: Mon May 1 19:20:47 2017 +0000
Committer: Sean Broeder <sb...@edev05.novalocal>
Committed: Mon May 1 19:20:47 2017 +0000

----------------------------------------------------------------------
 .../NonPendingTransactionException.java         | 47 ++++++++++++++
 .../transactional/TrxRegionEndpoint.java.tmpl   |  2 +-
 .../TransactionalRegionScannerHolder.java       | 64 --------------------
 3 files changed, 48 insertions(+), 65 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-trafodion/blob/4d378a6c/core/sqf/src/seatrans/hbase-trx/src/main/java/org/apache/hadoop/hbase/client/transactional/NonPendingTransactionException.java
----------------------------------------------------------------------
diff --git a/core/sqf/src/seatrans/hbase-trx/src/main/java/org/apache/hadoop/hbase/client/transactional/NonPendingTransactionException.java b/core/sqf/src/seatrans/hbase-trx/src/main/java/org/apache/hadoop/hbase/client/transactional/NonPendingTransactionException.java
new file mode 100644
index 0000000..643f594
--- /dev/null
+++ b/core/sqf/src/seatrans/hbase-trx/src/main/java/org/apache/hadoop/hbase/client/transactional/NonPendingTransactionException.java
@@ -0,0 +1,47 @@
+/**
+* @@@ START COPYRIGHT @@@
+*
+* Licensed to the Apache Software Foundation (ASF) under one
+* or more contributor license agreements.  See the NOTICE file
+* distributed with this work for additional information
+* regarding copyright ownership.  The ASF licenses this file
+* to you under the Apache License, Version 2.0 (the
+* "License"); you may not use this file except in compliance
+* with the License.  You may obtain a copy of the License at
+*
+*   http://www.apache.org/licenses/LICENSE-2.0
+*
+* Unless required by applicable law or agreed to in writing,
+* software distributed under the License is distributed on an
+* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+* KIND, either express or implied.  See the License for the
+* specific language governing permissions and limitations
+* under the License.
+*
+* @@@ END COPYRIGHT @@@
+**/
+
+package org.apache.hadoop.hbase.client.transactional;
+
+import org.apache.hadoop.hbase.DoNotRetryIOException;
+
+/**
+ * Thrown if a region server receives a commit before a commit request
+ */
+public class NonPendingTransactionException extends DoNotRetryIOException {
+
+  private static final long serialVersionUID = 698575374929591099L;
+
+  /** constructor */
+  public NonPendingTransactionException() {
+    super();
+  }
+
+  /**
+   * Constructor
+   * @param s message
+   */
+  public NonPendingTransactionException(String s) {
+    super(s);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-trafodion/blob/4d378a6c/core/sqf/src/seatrans/hbase-trx/src/main/java/org/apache/hadoop/hbase/coprocessor/transactional/TrxRegionEndpoint.java.tmpl
----------------------------------------------------------------------
diff --git a/core/sqf/src/seatrans/hbase-trx/src/main/java/org/apache/hadoop/hbase/coprocessor/transactional/TrxRegionEndpoint.java.tmpl b/core/sqf/src/seatrans/hbase-trx/src/main/java/org/apache/hadoop/hbase/coprocessor/transactional/TrxRegionEndpoint.java.tmpl
index 78ed97e..b0f9d24 100644
--- a/core/sqf/src/seatrans/hbase-trx/src/main/java/org/apache/hadoop/hbase/coprocessor/transactional/TrxRegionEndpoint.java.tmpl
+++ b/core/sqf/src/seatrans/hbase-trx/src/main/java/org/apache/hadoop/hbase/coprocessor/transactional/TrxRegionEndpoint.java.tmpl
@@ -120,7 +120,7 @@ import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.CellUtil;
-#ifdef HDP2.3 HDP2.4 CDH5.7 APACHE1.2
+#ifdef HDP2.3 HDP2.4 CDH5.5 CDH5.7 APACHE1.2
 import org.apache.hadoop.hbase.ChoreService;
 import org.apache.hadoop.hbase.ScheduledChore;
 #endif

http://git-wip-us.apache.org/repos/asf/incubator-trafodion/blob/4d378a6c/core/sqf/src/seatrans/hbase-trx/src/main/java/org/apache/hadoop/hbase/regionserver/transactional/TransactionalRegionScannerHolder.java
----------------------------------------------------------------------
diff --git a/core/sqf/src/seatrans/hbase-trx/src/main/java/org/apache/hadoop/hbase/regionserver/transactional/TransactionalRegionScannerHolder.java b/core/sqf/src/seatrans/hbase-trx/src/main/java/org/apache/hadoop/hbase/regionserver/transactional/TransactionalRegionScannerHolder.java
deleted file mode 100644
index b4096f0..0000000
--- a/core/sqf/src/seatrans/hbase-trx/src/main/java/org/apache/hadoop/hbase/regionserver/transactional/TransactionalRegionScannerHolder.java
+++ /dev/null
@@ -1,64 +0,0 @@
-/**
-* @@@ START COPYRIGHT @@@
-*
-* Licensed to the Apache Software Foundation (ASF) under one
-* or more contributor license agreements.  See the NOTICE file
-* distributed with this work for additional information
-* regarding copyright ownership.  The ASF licenses this file
-* to you under the Apache License, Version 2.0 (the
-* "License"); you may not use this file except in compliance
-* with the License.  You may obtain a copy of the License at
-*
-*   http://www.apache.org/licenses/LICENSE-2.0
-*
-* Unless required by applicable law or agreed to in writing,
-* software distributed under the License is distributed on an
-* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
-* KIND, either express or implied.  See the License for the
-* specific language governing permissions and limitations
-* under the License.
-*
-* @@@ END COPYRIGHT @@@
-**/
-
-package org.apache.hadoop.hbase.regionserver.transactional;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.hbase.regionserver.Region;
-import org.apache.hadoop.hbase.regionserver.RegionScanner;
-
-/**
- * Holds a RegionScanner
- */
- public class TransactionalRegionScannerHolder {
-    public RegionScanner s;
-    public Region r;
-    public long nextCallSeq;
-    public long numberOfRows;
-    public long rowsRemaining;
-    public long transId;
-    public long scannerId;
-    public boolean hasMore;
-
-
-    public TransactionalRegionScannerHolder(long transId,
-                                            long scannerId,
-                                            RegionScanner s, 
-                                            Region r) {
-      this.transId = transId;
-      this.scannerId = scannerId;
-      this.s = s;
-      this.r = r;
-      this.nextCallSeq = 0L;
-      this.numberOfRows = 0L;
-      this.rowsRemaining = 0L;
-      this.hasMore = false;
-    }
-
-    public void cleanHolder() {
-      this.r = null;
-      this.s = null;
-    }
-  }
-


[20/22] incubator-trafodion git commit: Merge branch 'master' of github.com:apache/incubator-trafodion into sean_traf

Posted by sa...@apache.org.
Merge branch 'master' of github.com:apache/incubator-trafodion into sean_traf

Conflicts:
	core/sql/regress/seabase/EXPECTED022


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

Branch: refs/heads/master
Commit: 7224c058f6ea313585982212061ba9a6ced00577
Parents: 85115b0 c25694d
Author: Sean Broeder <sb...@edev05.esgyn.local>
Authored: Fri May 12 13:30:50 2017 +0000
Committer: Sean Broeder <sb...@edev05.esgyn.local>
Committed: Fri May 12 13:30:50 2017 +0000

----------------------------------------------------------------------
 NOTICE                                          |   2 +-
 core/conn/jdbcT4/pom.xml                        |  93 +++++++++++++
 .../odbc/odbcclient/unixcli/cli/sqltocconv.cpp  |   4 +
 core/sqf/build-scripts/build.branch             |   4 +-
 core/sqf/sqenvcom.sh                            |  32 ++---
 core/sqf/src/seabed/src/ms.cpp                  |   2 +
 core/sql/cli/CliExtern.cpp                      |  23 +++-
 core/sql/regress/seabase/EXPECTED022            | 129 +++----------------
 core/sql/regress/seabase/TEST022                |  33 ++---
 .../src/asciidoc/_chapters/ambari_install.adoc  |  51 ++++++--
 docs/provisioning_guide/src/images/amb1.PNG     | Bin 0 -> 46318 bytes
 docs/provisioning_guide/src/images/amb2.PNG     | Bin 0 -> 12063 bytes
 docs/provisioning_guide/src/images/amb3.PNG     | Bin 0 -> 13764 bytes
 docs/provisioning_guide/src/images/amb4.PNG     | Bin 0 -> 44413 bytes
 docs/provisioning_guide/src/images/amb5.PNG     | Bin 0 -> 10552 bytes
 .../src/asciidoc/_chapters/olap_functions.adoc  |   2 +-
 .../sql_functions_and_expressions.adoc          |   4 +-
 install/ambari-installer/Makefile               |  11 +-
 .../2.1/configuration/traf-cluster-env.xml      |   1 +
 .../2.1/configuration/trafodion-env.xml         |   4 +
 .../TRAFODION/2.1/package/scripts/params.py     |   2 +
 .../2.1/package/scripts/trafodionmaster.py      |   3 +-
 install/python-installer/db_install.py          |  11 +-
 install/python-installer/scripts/traf_check.py  |   5 +
 win-odbc64/odbcclient/drvr35/sqltocconv.cpp     |   6 +
 25 files changed, 245 insertions(+), 177 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-trafodion/blob/7224c058/core/sql/regress/seabase/EXPECTED022
----------------------------------------------------------------------
diff --cc core/sql/regress/seabase/EXPECTED022
index 399deac,80784c7..64a6e30
--- a/core/sql/regress/seabase/EXPECTED022
+++ b/core/sql/regress/seabase/EXPECTED022
@@@ -635,13 -635,13 +635,13 @@@ ROW_ID (EXPR
  --- SQL operation complete.
  >>invoke t022hbm1;
  
- -- Definition of table #CAT.#SCH.T022HBM1
- -- Definition current
+ -- Definition of Trafodion table TRAFODION.SCH.T022HBM1
+ -- Definition current  Thu May 11 22:43:35 2017
  
 -  (
 -    SYSKEY                           LARGEINT NO DEFAULT NOT NULL NOT DROPPABLE
 -  , A                                INT DEFAULT NULL
 -  )
 + (
 + SYSKEY LARGEINT NO DEFAULT NOT NULL NOT DROPPABLE
 + , A INT DEFAULT NULL
 + )
  
  --- SQL operation complete.
  >>drop table t022hbm1;
@@@ -720,17 -720,17 +720,17 @@@ a2 
  --- SQL operation complete.
  >>invoke t022hbm1_like;
  
- -- Definition of table #CAT.#SCH.T022HBM1_LIKE
- -- Definition current
+ -- Definition of Trafodion table TRAFODION.SCH.T022HBM1_LIKE
+ -- Definition current  Thu May 11 22:43:57 2017
  
 -  (
 -    "cf".A                           VARCHAR(4) CHARACTER SET ISO88591 COLLATE
 -      DEFAULT NO DEFAULT NOT NULL NOT DROPPABLE
 -  , "cf".B                           CHAR(4) CHARACTER SET ISO88591 COLLATE
 -      DEFAULT DEFAULT NULL
 -  , "cf".C                           INT DEFAULT NULL
 -  )
 -  PRIMARY KEY (A ASC)
 + (
 + "cf".A VARCHAR(4) CHARACTER SET ISO88591 COLLATE
 + DEFAULT NO @eof@ NULL NOT DROPPABLE
 + , "cf".B CHAR(4) CHARACTER SET ISO88591 COLLATE
 + DEFAULT DEFAULT NULL
 + , "cf".C INT DEFAULT NULL
 + )
 + PRIMARY KEY (A ASC)
  
  --- SQL operation complete.
  >>
@@@ -840,93 -840,30 +840,30 @@@ a2 ? 
  >>drop external table t022hbm1;
  
  --- SQL operation complete.
 ->>create external table t022hbm1 (a varchar(4) not null, b int) 
 -+>        primary key (a)
 -+>        attribute default column family 'cf'
 -+>        map to hbase table t022hbm1
 -+>        data format native;
 +>>create external table t022hbm1 (a varchar(4) not null, b int)
 ++> primary key (a)
 ++> attribute default column family 'cf'
 ++> map to hbase table t022hbm1
 ++> data format native;
  
  --- SQL operation complete.
- >>
- >>insert into t022hbm1 values ('a', 1);
- 
- --- 1 row(s) inserted.
- >>select * from t022hbm1;
- 
- A B
- ---- -----------
- 
- a 1
- 
- --- 1 row(s) selected.
- >>update t022hbm1 set b = b + 1;
- 
- --- 1 row(s) updated.
- >>select * from t022hbm1;
- 
- A B
- ---- -----------
- 
- a 2
- 
- --- 1 row(s) selected.
- >>insert into t022hbm1 values ('a', 1);
- 
- *** ERROR[8102] The operation is prevented by a unique constraint.
- 
- --- 0 row(s) inserted.
- >> -- should fail
- >>insert into t022hbm1 values ('b', null);
- 
- --- 1 row(s) inserted.
- >>select * from t022hbm1;
- 
- A B
- ---- -----------
- 
- a 2
- b ?
- 
- --- 2 row(s) selected.
- >>delete from t022hbm1 where a = 'a';
- 
- --- 1 row(s) deleted.
- >>select * from t022hbm1;
- 
- A B
- ---- -----------
- 
- b ?
- 
- --- 1 row(s) selected.
- >>update t022hbm1 set b = 10;
- 
- --- 1 row(s) updated.
- >>select * from t022hbm1;
- 
- A B
- ---- -----------
- 
- b 10
- 
- --- 1 row(s) selected.
- >>update t022hbm1 set b = null;
- 
- --- 1 row(s) updated.
- >>select * from t022hbm1;
- 
- A B
- ---- -----------
- 
- b ?
- 
- --- 1 row(s) selected.
- >>delete from t022hbm1;
- 
- --- 1 row(s) deleted.
- >>select * from t022hbm1;
- 
- --- 0 row(s) selected.
+ >>--This part of the test is being commented out due to non deterministic results
+ >>--It needs to be renabled after this JIRA is fixed : TRAFODION-2613
+ >>--insert into t022hbm1 values ('a', 1);
+ >>--select * from t022hbm1;
+ >>--update t022hbm1 set b = b + 1;
+ >>--select * from t022hbm1;
+ >>--insert into t022hbm1 values ('a', 1); -- should fail
+ >>--insert into t022hbm1 values ('b', null);
+ >>--select * from t022hbm1;
+ >>--delete from t022hbm1 where a = 'a';
+ >>--select * from t022hbm1;
+ >>--update t022hbm1 set b = 10;
+ >>--select * from t022hbm1;
+ >>--update t022hbm1 set b = null;
+ >>--select * from t022hbm1;
+ >>--delete from t022hbm1;
+ >>--select * from t022hbm1;
  >>
  >>drop external table t022hbm1;
  
@@@ -1090,39 -1027,15 +1027,15 @@@ A B 
  --- SQL operation complete.
  >>invoke t022hbm1;
  
 --- Definition of Trafodion HBase mapped table T022HBM1
 --- Definition current  Thu May 11 22:45:13 2017
 +-- Definition of TRAFODION HBase mapped table T022HBM1
- -- Definition current
- 
-  (
-  "cf".A VARCHAR(4) CHARACTER SET ISO88591 COLLATE
-  DEFAULT NO @eof@ NULL NOT DROPPABLE
-  , "cf".B INT DEFAULT NULL
-  )
-  PRIMARY KEY NOT SERIALIZED (A ASC)
- 
- --- SQL operation complete.
- >>
- >>drop external table t022hbm1;
- 
- --- SQL operation complete.
- >>create external table t022hbm1 (a char(4) not null, b int)
- +> primary key serialized (a)
- +> attribute default column family 'cf'
- +> map to hbase table t022hbm1
- +> data format native;
- 
- --- SQL operation complete.
- >>invoke t022hbm1;
- 
- -- Definition of TRAFODION HBase mapped table T022HBM1
 +-- Definition current
  
 -  (
 -    "cf".A                           VARCHAR(4) CHARACTER SET ISO88591 COLLATE
 -      DEFAULT NO DEFAULT NOT NULL NOT DROPPABLE
 -  , "cf".B                           INT DEFAULT NULL
 -  )
 -  PRIMARY KEY NOT SERIALIZED (A ASC)
 + (
 + "cf".A CHAR(4) CHARACTER SET ISO88591 COLLATE
 + DEFAULT NO @eof@ NULL NOT DROPPABLE
 + , "cf".B INT DEFAULT NULL
 + )
 + PRIMARY KEY (A ASC)
  
  --- SQL operation complete.
  >>

http://git-wip-us.apache.org/repos/asf/incubator-trafodion/blob/7224c058/core/sql/regress/seabase/TEST022
----------------------------------------------------------------------


[13/22] incubator-trafodion git commit: Updated .gitignore file

Posted by sa...@apache.org.
Updated .gitignore file


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

Branch: refs/heads/master
Commit: cd75b425b4ef99b6074f2083477c4bf9511fad32
Parents: d9c2ec5
Author: Sean Broeder <sb...@edev05.novalocal>
Authored: Tue May 2 00:00:31 2017 +0000
Committer: Sean Broeder <sb...@edev05.novalocal>
Committed: Tue May 2 00:00:31 2017 +0000

----------------------------------------------------------------------
 core/sqf/src/seatrans/.gitignore | 1 +
 1 file changed, 1 insertion(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-trafodion/blob/cd75b425/core/sqf/src/seatrans/.gitignore
----------------------------------------------------------------------
diff --git a/core/sqf/src/seatrans/.gitignore b/core/sqf/src/seatrans/.gitignore
index 734bad8..e1dcf48 100644
--- a/core/sqf/src/seatrans/.gitignore
+++ b/core/sqf/src/seatrans/.gitignore
@@ -11,6 +11,7 @@ hbase-trx/src/main/java/org/apache/hadoop/hbase/coprocessor/transactional/TrxReg
 hbase-trx/src/main/java/org/apache/hadoop/hbase/regionserver/transactional/CleanOldTransactionsChore.java
 hbase-trx/src/main/java/org/apache/hadoop/hbase/regionserver/transactional/KeyValueListScanner.java
 hbase-trx/src/main/java/org/apache/hadoop/hbase/regionserver/transactional/MemoryUsageChore.java
+hbase-trx/src/main/java/org/apache/hadoop/hbase/regionserver/transactional/TransactionalRegionScannerHolder.java
 hbase-trx/src/main/java/org/apache/hadoop/hbase/regionserver/transactional/TrxTransactionState.java
 hbase-trx/src/main/java/org/apache/hadoop/hbase/client/ClientScanner98.java
 hbase-trx/src/main/java/org/apache/hadoop/hbase/client/TrafParallelClientScanner.java


[02/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

Posted by sa...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-trafodion/blob/91794b53/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 a66d880..d0fbb06 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
@@ -36,30 +36,36 @@ import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.CellUtil;
 import org.apache.hadoop.hbase.client.Connection;
+import org.apache.hadoop.hbase.client.coprocessor.Batch;
 import org.apache.hadoop.hbase.client.Admin;
-import org.apache.hadoop.hbase.client.HTable;
-import org.apache.hadoop.hbase.client.HTable;
-import org.apache.hadoop.hbase.client.Table;
+import org.apache.hadoop.hbase.client.BufferedMutator;
+import org.apache.hadoop.hbase.client.Durability;
 import org.apache.hadoop.hbase.client.Get;
 import org.apache.hadoop.hbase.client.Put;
 import org.apache.hadoop.hbase.client.Delete;
+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.RetriesExhaustedWithDetailsException;
 import org.apache.hadoop.hbase.client.Scan;
-import org.apache.hadoop.hbase.client.transactional.TransactionManager;
-import org.apache.hadoop.hbase.client.transactional.TransactionState;
+import org.apache.hadoop.hbase.client.Table;
 import org.apache.hadoop.hbase.client.transactional.CommitUnsuccessfulException;
-import org.apache.hadoop.hbase.client.transactional.UnknownTransactionException;
 import org.apache.hadoop.hbase.client.transactional.HBaseBackedTransactionLogger;
+import org.apache.hadoop.hbase.client.transactional.TransactionManager;
 import org.apache.hadoop.hbase.client.transactional.TransactionRegionLocation;
+import org.apache.hadoop.hbase.client.transactional.TransactionState;
 import org.apache.hadoop.hbase.client.transactional.TransState;
 import org.apache.hadoop.hbase.client.transactional.UnknownTransactionException;
 import org.apache.hadoop.hbase.client.coprocessor.Batch;
 import org.apache.hadoop.hbase.coprocessor.transactional.generated.TrxRegionProtos.TlogDeleteRequest;
 import org.apache.hadoop.hbase.coprocessor.transactional.generated.TrxRegionProtos.TlogDeleteResponse;
+import org.apache.hadoop.hbase.coprocessor.transactional.generated.TrxRegionProtos.TlogTransactionStatesFromIntervalRequest;
+import org.apache.hadoop.hbase.coprocessor.transactional.generated.TrxRegionProtos.TlogTransactionStatesFromIntervalResponse;
+import org.apache.hadoop.hbase.coprocessor.transactional.generated.TrxRegionProtos.TlogWriteRequest;
+import org.apache.hadoop.hbase.coprocessor.transactional.generated.TrxRegionProtos.TlogWriteResponse;
 import org.apache.hadoop.hbase.coprocessor.transactional.generated.TrxRegionProtos.TrxRegionService;
 import org.apache.hadoop.hbase.HColumnDescriptor;
+import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.HRegionLocation;
 import org.apache.hadoop.hbase.HTableDescriptor;
@@ -69,15 +75,22 @@ import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.TableExistsException;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.util.Bytes;
-
 import org.apache.hadoop.hbase.ZooKeeperConnectionException;
 
 import org.apache.hadoop.hbase.ipc.BlockingRpcCallback;
+import org.apache.hadoop.hbase.ipc.FailedServerException;
 import org.apache.hadoop.hbase.ipc.ServerRpcController;
 import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
 import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto;
 import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.MutationType;
 
+import org.apache.hadoop.hbase.ZooKeeperConnectionException;
+
+import org.apache.hadoop.hbase.ipc.BlockingRpcCallback;
+import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
+import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto;
+import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.MutationType;
+
 import org.apache.hadoop.hbase.regionserver.RegionSplitPolicy;
 
 import com.google.protobuf.ByteString;
@@ -109,6 +122,8 @@ import java.util.concurrent.ExecutorService;
 import java.util.concurrent.Future;
 import java.util.concurrent.RejectedExecutionException;
 
+import org.trafodion.dtm.HBaseAuditControlPoint;
+
 public class TmAuditTlog {
 
    static final Log LOG = LogFactory.getLog(TmAuditTlog.class);
@@ -117,10 +132,10 @@ public class TmAuditTlog {
    private static final byte[] TLOG_FAMILY = Bytes.toBytes("tf");
    private static final byte[] ASN_STATE = Bytes.toBytes("as");
    private static final byte[] QUAL_TX_STATE = Bytes.toBytes("tx");
-   private static HTable[] table;
+   private Table table;
    private static Connection connection;
-   private static HBaseAuditControlPoint tLogControlPoint;
-   private static long tLogControlPointNum;
+   private HBaseAuditControlPoint tLogControlPoint;
+   private long tLogControlPointNum;
    private static long tLogHashKey;
    private static int  tLogHashShiftFactor;
    private int dtmid;
@@ -153,7 +168,6 @@ public class TmAuditTlog {
 
    private static int     versions;
    private static int     tlogNumLogs;
-   private boolean useAutoFlush;
    private static boolean ageCommitted;
    private static boolean forceControlPoint;
    private boolean disableBlockCache;
@@ -163,32 +177,33 @@ public class TmAuditTlog {
 
    private static AtomicLong asn;  // Audit sequence number is the monotonic increasing value of the tLog write
 
-   private static Object tlogAuditLock[];        // Lock for synchronizing access via regions.
-
-   private static Object tablePutLock;            // Lock for synchronizing table.put operations
-                                                  // to avoid ArrayIndexOutOfBoundsException
    private static byte filler[];
    public static final int TLOG_SLEEP = 1000;      // One second
    public static final int TLOG_SLEEP_INCR = 5000; // Five seconds
    public static final int TLOG_RETRY_ATTEMPTS = 5;
 
+   private static int myClusterId = 0;
+
    /**
     * tlogThreadPool - pool of thread for asynchronous requests
     */
    ExecutorService tlogThreadPool;
 
    private abstract class TlogCallable implements Callable<Integer>{
-      TransactionState transactionState;
       HRegionLocation  location;
-      HTable table;
+      Table table;
       byte[] startKey;
       byte[] endKey_orig;
       byte[] endKey;
 
-     TlogCallable(TransactionState txState, HRegionLocation location, Connection connection) throws IOException {
-        transactionState = txState;
+     TlogCallable(HRegionLocation location, Connection connection) throws IOException {
         this.location = location;
-        table = new HTable(location.getRegionInfo().getTable(), connection, tlogThreadPool);
+        try {
+           this.table = connection.getTable(location.getRegionInfo().getTable());
+        } catch(IOException e) {
+           LOG.error("Error obtaining Table instance ", e);
+           this.table = null;
+        }
         startKey = location.getRegionInfo().getStartKey();
         endKey_orig = location.getRegionInfo().getEndKey();
         endKey = TransactionManager.binaryIncrementPos(endKey_orig, -1);
@@ -201,37 +216,38 @@ public class TmAuditTlog {
        boolean retry = false;
        boolean refresh = false;
        final Scan scan = new Scan(startKey, endKey);
+       scan.setCacheBlocks(false);
 
        int retryCount = 0;
        int retrySleep = TLOG_SLEEP;
        do {
+          try {
              if (LOG.isTraceEnabled()) LOG.trace("deleteEntriesOlderThanASNX -- ENTRY ASN: " + auditSeqNum);
-             Batch.Call<TrxRegionService, TlogDeleteResponse> callable =
-                new Batch.Call<TrxRegionService, TlogDeleteResponse>() {
-                  ServerRpcController controller = new ServerRpcController();
-                  BlockingRpcCallback<TlogDeleteResponse> rpcCallback =
-                      new BlockingRpcCallback<TlogDeleteResponse>();
-
-                     @Override
-                     public TlogDeleteResponse call(TrxRegionService instance) throws IOException {
-                        org.apache.hadoop.hbase.coprocessor.transactional.generated.TrxRegionProtos.TlogDeleteRequest.Builder
-                        builder = TlogDeleteRequest.newBuilder();
-                        builder.setAuditSeqNum(auditSeqNum);
-                        builder.setTransactionId(transactionState.getTransactionId());
-                        builder.setScan(ProtobufUtil.toScan(scan));
-                        builder.setRegionName(ByteString.copyFromUtf8(Bytes.toString(regionName))); //ByteString.copyFromUtf8(Bytes.toString(regionName)));
-                        builder.setAgeCommitted(pv_ageCommitted); 
-
-                        instance.deleteTlogEntries(controller, builder.build(), rpcCallback);
-                        return rpcCallback.get();
-                    }
-                 };
-
                  Map<byte[], TlogDeleteResponse> result = null;
                  try {
                    if (LOG.isTraceEnabled()) LOG.trace("deleteEntriesOlderThanASNX -- before coprocessorService ASN: " + auditSeqNum
                          + " startKey: " + new String(startKey, "UTF-8") + " endKey: " + new String(endKey, "UTF-8"));
-                   result = table.coprocessorService(TrxRegionService.class, startKey, endKey, callable);
+                   result = table.coprocessorService(TrxRegionService.class,
+                      startKey,
+                      endKey,
+                      new Batch.Call<TrxRegionService, TlogDeleteResponse>() {
+
+                      @Override
+                      public TlogDeleteResponse call(TrxRegionService instance) throws IOException {
+                         BlockingRpcCallback<TlogDeleteResponse> rpcCallback =
+                         new BlockingRpcCallback<TlogDeleteResponse>();
+
+                         org.apache.hadoop.hbase.coprocessor.transactional.generated.TrxRegionProtos.TlogDeleteRequest.Builder
+                         builder = TlogDeleteRequest.newBuilder();
+                         builder.setAuditSeqNum(auditSeqNum);
+                         builder.setScan(ProtobufUtil.toScan(scan));
+                         builder.setRegionName(ByteString.copyFromUtf8(Bytes.toString(regionName))); //ByteString.copyFromUtf8(Bytes.toString(regionName)));
+                         builder.setAgeCommitted(pv_ageCommitted);
+
+                         instance.deleteTlogEntries(null, builder.build(), rpcCallback);
+                         return rpcCallback.get();
+                      }
+                   });
                  } catch (Throwable e) {
                    String msg = new String("ERROR occurred while calling deleteTlogEntries coprocessor service in deleteEntriesOlderThanASNX: " + e);
                    LOG.error(msg, e);
@@ -257,31 +273,24 @@ public class TmAuditTlog {
                     }
                     retry = false;
                  }
+              } catch (Exception e) {
+                 LOG.error("deleteEntriesOlderThanASNX retrying due to Exception: ", e);
+                 refresh = true;
+                 retry = true;
+              }
               if (refresh) {
 
-               HRegionLocation lv_hrl = table.getRegionLocation(startKey);
-               HRegionInfo     lv_hri = lv_hrl.getRegionInfo();
-               String          lv_node = lv_hrl.getHostname();
-               int             lv_length = lv_node.indexOf('.');
+               RegionLocator   rl = connection.getRegionLocator(table.getName());
+               location = rl.getRegionLocation(startKey, true);
 
-               if (LOG.isTraceEnabled()) LOG.trace("deleteEntriesOlderThanASNX -- location being refreshed : "
+               if (LOG.isTraceEnabled()) LOG.trace("deleteEntriesOlderThanASNX -- location refreshed : "
                     + location.getRegionInfo().getRegionNameAsString() + "endKey: "
                     + Hex.encodeHexString(location.getRegionInfo().getEndKey()) + " for ASN: " + auditSeqNum);
                if(retryCount == TLOG_RETRY_ATTEMPTS) {
                   LOG.error("Exceeded retry attempts (" + retryCount + ") in deleteEntriesOlderThanASNX for ASN: " + auditSeqNum);
-                  // We have received our reply in the form of an exception,
-                  // so decrement outstanding count and wake up waiters to avoid
-                  // getting hung forever
                   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");
-               if (LOG.isWarnEnabled()) LOG.warn("deleteEntriesOlderThanASNX -- lv_hri: " + lv_hri);
-               if (LOG.isWarnEnabled()) LOG.warn("deleteEntriesOlderThanASNX -- location.getRegionInfo(): " + location.getRegionInfo());
-               table.getRegionLocation(startKey, true);
-
                if (LOG.isTraceEnabled()) LOG.trace("deleteEntriesOlderThanASNX -- setting retry, count: " + retryCount);
                refresh = false;
             }
@@ -297,14 +306,365 @@ public class TmAuditTlog {
                retrySleep += TLOG_SLEEP_INCR;
             }
        } while (retryCount < TLOG_RETRY_ATTEMPTS && retry == true);
-       // We have received our reply so decrement outstanding count
-       transactionState.requestPendingCountDec(null);
 
        if (LOG.isTraceEnabled()) LOG.trace("deleteEntriesOlderThanASNX -- EXIT ASN: " + auditSeqNum);
        return 0;
-     } //getTransactionStatesFromIntervalX
+     } //deleteEntriesOlderThanASNX
    } // TlogCallable
 
+   /**
+    * TlogCallable1  :  inner class for creating asynchronous requests
+    */
+   private abstract class TlogCallable1 implements Callable<Integer>{
+      TransactionState transactionState;
+      HRegionLocation  location;
+      Table table;
+      byte[] startKey;
+      byte[] endKey_orig;
+      byte[] endKey;
+
+      TlogCallable1(TransactionState txState, HRegionLocation location, Connection connection) {
+         transactionState = txState;
+         this.location = location;
+         try {
+             this.table = connection.getTable(location.getRegionInfo().getTable());
+         } catch(IOException e) {
+            LOG.error("Error obtaining Table instance ", e);
+            this.table = null;
+         }
+         startKey = location.getRegionInfo().getStartKey();
+         endKey_orig = location.getRegionInfo().getEndKey();
+         endKey = TransactionManager.binaryIncrementPos(endKey_orig, -1);
+      }
+
+     /**
+      * Method  : doTlogWriteX
+      * Params  : regionName - name of Region
+      *           transactionId - transaction identifier
+      * Return  : Always 0, can ignore
+      * Purpose : write commit/abort state record for a given transaction
+      */
+      public Integer doTlogWriteX(final byte[] regionName, final long transactionId, final long commitId, final Put put) throws IOException {
+         long threadId = Thread.currentThread().getId();
+         if (LOG.isTraceEnabled()) LOG.trace("doTlogWriteX -- ENTRY txid: " + transactionId + ", clusterId: " + myClusterId + ", thread " + threadId
+        		             + ", put: " + put.toString());
+         boolean retry = false;
+         boolean refresh = false;
+
+         int retryCount = 0;
+         int retrySleep = TLOG_SLEEP;
+
+         do {
+            try {
+              if (LOG.isTraceEnabled()) LOG.trace("doTlogWriteX -- try txid: " + transactionId + " in thread " + threadId);
+              Batch.Call<TrxRegionService, TlogWriteResponse> callable =
+                 new Batch.Call<TrxRegionService, TlogWriteResponse>() {
+                    ServerRpcController controller = new ServerRpcController();
+                    BlockingRpcCallback<TlogWriteResponse> rpcCallback = new BlockingRpcCallback<TlogWriteResponse>();
+
+                    @Override
+                    public TlogWriteResponse call(TrxRegionService instance) throws IOException {
+                       org.apache.hadoop.hbase.coprocessor.transactional.generated.TrxRegionProtos.TlogWriteRequest.Builder builder = TlogWriteRequest.newBuilder();
+                       builder.setTransactionId(transactionId);
+                       builder.setCommitId(commitId);
+                       builder.setRegionName(ByteString.copyFromUtf8(Bytes.toString(regionName))); //ByteString.copyFromUtf8(Bytes.toString(regionName)));
+                       builder.setFamily(HBaseZeroCopyByteString.wrap(TLOG_FAMILY));
+                       builder.setQualifier(HBaseZeroCopyByteString.wrap(ASN_STATE));
+                       MutationProto m1 = ProtobufUtil.toMutation(MutationType.PUT, put);
+                       builder.setPut(m1);
+
+                       instance.putTlog(controller, builder.build(), rpcCallback);
+                       long threadId = Thread.currentThread().getId();
+                       if (LOG.isTraceEnabled()) LOG.trace("TlogWrite -- sent for txid: " + transactionId + " in thread " + threadId);
+                       TlogWriteResponse response = rpcCallback.get();
+                       if (LOG.isTraceEnabled()) LOG.trace("TlogWrite -- response received (" + response + ") for txid: "
+                               + transactionId + " in thread " + threadId );
+                       return response;
+                    }
+                 };
+
+              Map<byte[], TlogWriteResponse> result = null;
+              try {
+                 if (LOG.isTraceEnabled()) LOG.trace("doTlogWriteX -- before coprocessorService txid: " + transactionId + " table: "
+                             + table.toString() + " startKey: " + new String(startKey, "UTF-8") + " endKey: " + new String(endKey, "UTF-8"));
+                 result = table.coprocessorService(TrxRegionService.class, startKey, endKey, callable);
+                 if (LOG.isTraceEnabled()) LOG.trace("doTlogWriteX -- after coprocessorService txid: " + transactionId);
+              } catch (Throwable e) {
+                 String msg = "ERROR occurred while calling doTlogWriteX coprocessor service in doTlogWriteX";
+                 LOG.error(msg + ":" , e);
+                 throw new IOException(e);
+              }
+              if(result.size() != 1) {
+                 LOG.error("doTlogWriteX, received incorrect result size: " + result.size() + " txid: " + transactionId);
+                 throw new IOException("Wrong result size in doWriteTlogX");
+              }
+              else {
+                 // size is 1
+                 for (TlogWriteResponse tlw_response : result.values()){
+                    if(tlw_response.getHasException()) {
+                       String exceptionString = new String (tlw_response.getException().toString());
+                       if (LOG.isTraceEnabled()) LOG.trace("doTlogWriteX coprocessor exception: " + tlw_response.getException());
+                       throw new Exception(tlw_response.getException());
+                    }
+                 }
+                 retry = false;
+              }
+            }
+            catch (Exception e) {
+              LOG.error("doTlogWriteX retrying due to Exception: " + e);
+              refresh = true;
+              retry = true;
+            }
+            if (refresh) {
+
+               RegionLocator   rl = connection.getRegionLocator(table.getName());
+               location = rl.getRegionLocation(startKey, true);
+
+               if (LOG.isTraceEnabled()) LOG.trace("doTlogWriteX -- location refreshed : "
+            		   + location.getRegionInfo().getRegionNameAsString() + "endKey: "
+                       + Hex.encodeHexString(location.getRegionInfo().getEndKey()) + " for transaction: " + transactionId);
+               if(retryCount == TLOG_RETRY_ATTEMPTS) {
+                  LOG.error("Exceeded retry attempts (" + retryCount + ") in doTlogWriteX for transaction: " + transactionId);
+                  // We have received our reply in the form of an exception,
+                  // so decrement outstanding count and wake up waiters to avoid
+                  // getting hung forever
+                  IOException ie = new IOException("Exceeded retry attempts (" + retryCount + ") in doTlogWriteX for transaction: " + transactionId);
+                  transactionState.requestPendingCountDec(ie);
+                  throw ie;
+               }
+
+               if (LOG.isTraceEnabled()) LOG.trace("doTlogWriteX -- setting retry, count: " + retryCount);
+               refresh = false;
+            }
+
+            retryCount++;
+            if (retryCount < TLOG_RETRY_ATTEMPTS && retry == true) {
+               try {
+                  Thread.sleep(retrySleep);
+               } catch(InterruptedException ex) {
+                  Thread.currentThread().interrupt();
+               }
+
+               retrySleep += TLOG_SLEEP_INCR;
+            }
+         } while (retryCount < TLOG_RETRY_ATTEMPTS && retry == true);
+
+         // We have received our reply so decrement outstanding count
+         transactionState.requestPendingCountDec(null);
+
+         if (LOG.isTraceEnabled()) LOG.trace("doTlogWriteX -- EXIT txid: " + transactionId);
+         return 0;
+      }//doTlogWriteX
+   }//TlogCallable1
+
+   private abstract class TlogCallable2 implements Callable<ArrayList<TransactionState>>{
+      Table table;
+      byte[] startKey = HConstants.EMPTY_BYTE_ARRAY;
+      byte[] endKey = HConstants.EMPTY_BYTE_ARRAY;
+
+      TlogCallable2(Table targetTable, Connection connection) {
+        this.table = targetTable;
+      }
+
+      public ArrayList<TransactionState> getTransactionStatesFromIntervalX(final long clusterId, final long auditSeqNum) throws IOException {
+         boolean retry = false;
+         boolean refresh = false;
+         final Scan scan = new Scan(startKey, endKey); // Null start and end keys covers all regions
+         scan.setCaching(100);
+         scan.setCacheBlocks(false);
+
+         int retryCount = 0;
+         int retrySleep = TLOG_SLEEP;
+         ArrayList<TransactionState> transList = new ArrayList<TransactionState>();
+         do {
+           try {
+              if (LOG.isTraceEnabled()) LOG.trace("getTransactionStatesFromIntervalX -- ENTRY ASN: " + auditSeqNum);
+              Batch.Call<TrxRegionService, TlogTransactionStatesFromIntervalResponse> request =
+                 new Batch.Call<TrxRegionService, TlogTransactionStatesFromIntervalResponse>() {
+                   BlockingRpcCallback<TlogTransactionStatesFromIntervalResponse> rpcCallback =
+                      new BlockingRpcCallback<TlogTransactionStatesFromIntervalResponse>();
+
+                      @Override
+                      public TlogTransactionStatesFromIntervalResponse call(TrxRegionService instance) throws IOException {
+                        org.apache.hadoop.hbase.coprocessor.transactional.generated.TrxRegionProtos.TlogTransactionStatesFromIntervalRequest.Builder builder =
+                                TlogTransactionStatesFromIntervalRequest.newBuilder();
+                        builder.setClusterId(clusterId);
+                        builder.setAuditSeqNum(auditSeqNum);
+                        builder.setScan(ProtobufUtil.toScan(scan));
+                        instance.getTransactionStatesPriorToAsn(null, builder.build(), rpcCallback);
+                        return rpcCallback.get();
+                    }
+                 };
+
+                 Map<byte[], TlogTransactionStatesFromIntervalResponse> result = null;
+                 try {
+                   if (LOG.isTraceEnabled()) LOG.trace("getTransactionStatesFromIntervalX -- before coprocessorService ASN: " + auditSeqNum
+                                       + " startKey: " + new String(startKey, "UTF-8") + " endKey: " + new String(endKey, "UTF-8"));
+                   result = table.coprocessorService(TrxRegionService.class, startKey, endKey, request); // null keys cover all regions
+                 } catch (Throwable e) {
+                    String msg = "ERROR occurred while calling getTransactionStatesFromIntervalX coprocessor service in getTransactionStatesFromIntervalX";
+                    LOG.error(msg + ":" + e);
+                    throw new Exception(msg);
+                 }
+                 if (LOG.isTraceEnabled()) LOG.trace("getTransactionStatesFromIntervalX -- after coprocessorService ASN: " + auditSeqNum
+                         + " startKey: " + new String(startKey, "UTF-8") + " result size: " + result.size());
+
+                 if(result.size() >= 1) {
+                    LOG.info("getTransactionStatesFromInterval: result size is " + result.size());
+                    org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Result row = null;
+                    for (TlogTransactionStatesFromIntervalResponse TSFI_response : result.values()){
+
+                       if(TSFI_response.getHasException()) {
+                          if (LOG.isTraceEnabled()) LOG.trace("getTransactionStatesFromIntervalX coprocessor exception: "
+                               + TSFI_response.getException());
+                          throw new Exception(TSFI_response.getException());
+                       }
+
+                       long count = TSFI_response.getCount();
+                       LOG.info("getTransactionStatesFromInterval: count is " + count);
+                       for (int i = 0; i < count; i++){
+
+                          // Here we get the transaction records returned and create new TransactionState objects
+                          // Each 'row' is a return result from a region
+                          row = TSFI_response.getResult(i);
+                          Result rowResult = ProtobufUtil.toResult(row);
+                          if (!rowResult.isEmpty()) {
+                             byte [] value = rowResult.getValue(TLOG_FAMILY, ASN_STATE);
+                             if (value == null) {
+                                if (LOG.isTraceEnabled()) LOG.trace("getTransactionStatesFromInterval: tLog value is null, continuing");
+                                continue;
+                             }
+                             if (value.length == 0) {
+                                if (LOG.isTraceEnabled()) LOG.trace("getTransactionStatesFromInterval: tLog value.length is 0, continuing");
+                                continue;
+                             }
+                             TransactionState ts;
+                             TransState lvTxState = TransState.STATE_NOTX;
+                             StringTokenizer st = new StringTokenizer(Bytes.toString(value), ",");
+                             String stateString = new String("NOTX");
+                             String transidToken;
+                             if (! st.hasMoreElements()) {
+                                continue;
+                             }
+                             String asnToken = st.nextElement().toString();
+                             transidToken = st.nextElement().toString();
+                             stateString = st.nextElement().toString();
+                             long lvTransid = Long.parseLong(transidToken, 10);
+                             ts =  new TransactionState(lvTransid);
+                             ts.setRecoveryASN(Long.parseLong(asnToken, 10));
+                             ts.clearParticipatingRegions();
+
+                             if (LOG.isTraceEnabled()) LOG.trace("getTransactionStatesFromInterval: transaction: "
+                                                 + transidToken + " stateString is: " + stateString);
+
+                             if (stateString.equals(TransState.STATE_COMMITTED.toString())){
+                                lvTxState = TransState.STATE_COMMITTED;
+                             }
+                             else if (stateString.equals(TransState.STATE_ABORTED.toString())){
+                                lvTxState = TransState.STATE_ABORTED;
+                             }
+                             else if (stateString.equals(TransState.STATE_ACTIVE.toString())){
+                                lvTxState = TransState.STATE_ACTIVE;
+                             }
+                             else if (stateString.equals(TransState.STATE_PREPARED.toString())){
+                                lvTxState = TransState.STATE_PREPARED;
+                             }
+                             else if (stateString.equals(TransState.STATE_FORGOTTEN.toString())){
+                                lvTxState = TransState.STATE_FORGOTTEN;
+                             }
+                             else if (stateString.equals(TransState.STATE_FORGOTTEN_COMMITTED.toString())){
+                                lvTxState = TransState.STATE_FORGOTTEN_COMMITTED;
+                             }
+                             else if (stateString.equals(TransState.STATE_FORGOTTEN_ABORT.toString())){
+                                lvTxState = TransState.STATE_FORGOTTEN_ABORT;
+                             }
+                             else if (stateString.equals(TransState.STATE_RECOVERY_COMMITTED.toString())){
+                                lvTxState = TransState.STATE_RECOVERY_COMMITTED;
+                             }
+                             else if (stateString.equals(TransState.STATE_RECOVERY_ABORT.toString())){
+                                lvTxState = TransState.STATE_RECOVERY_ABORT;
+                             }
+                             else {
+                                lvTxState = TransState.STATE_BAD;
+                             }
+
+                             // get past the filler
+                             st.nextElement();
+
+                             // get past the place holder
+                             st.nextElement();
+                             String startIdToken = st.nextElement().toString();
+                             ts.setStartId(Long.parseLong(startIdToken));
+                             String commitIdToken = st.nextElement().toString();
+                             ts.setCommitId(Long.parseLong(commitIdToken));
+
+                             // Load the TransactionState object up with regions
+                             while (st.hasMoreElements()) {
+                                String tableNameToken = st.nextToken();
+                                Table tmpTable = connection.getTable(TableName.valueOf(tableNameToken));
+                                RegionLocator   rl = connection.getRegionLocator(table.getName());
+                                List<HRegionLocation> regions = rl.getAllRegionLocations();
+                                Iterator<HRegionLocation> it =  regions.iterator();
+                                while(it.hasNext()) { // iterate entries.
+                                   HRegionLocation loc = it.next();
+                                   HRegionInfo regionKey = loc.getRegionInfo();
+                                   if (LOG.isTraceEnabled()) LOG.trace("getTransactionStatesFromInterval: transaction: " + transidToken + " adding region: " + regionKey.getRegionNameAsString());
+                                   ServerName serverValue = loc.getServerName();
+                                   TransactionRegionLocation tloc = new TransactionRegionLocation(regionKey, serverValue);
+                                   ts.addRegion(tloc);
+                                }
+                             }
+                             ts.setStatus(lvTxState);
+
+                             if (LOG.isTraceEnabled()) LOG.trace("getTransactionStatesFromInterval: adding transid: "
+                                            + ts.getTransactionId() + " state: " + lvTxState + " to transList");
+                             transList.add(ts);
+                          } // if (! rowResult,isEmpty()))
+                       } // for (int i = 0; i < count
+                    } // TlogTransactionStatesFromIntervalResponse TSFI_response : result.values()
+                 } // if(result.size() >= 1)
+                 retry = false;
+              } catch (Exception e) {
+                 LOG.error("getTransactionStatesFromIntervalX retrying due to Exception: " + e);
+                 refresh = true;
+                 retry = true;
+              }
+              if (refresh) {
+
+               RegionLocator   rl = connection.getRegionLocator(table.getName());
+               rl.getAllRegionLocations();
+
+               if (LOG.isTraceEnabled()) LOG.trace("getTransactionStatesFromIntervalX -- locations being refreshed : " + table + " for ASN: " + auditSeqNum);
+               if(retryCount == TLOG_RETRY_ATTEMPTS) {
+                  LOG.error("Exceeded retry attempts (" + retryCount + ") in getTransactionStatesFromIntervalX for ASN: " + auditSeqNum);
+                     // We have received our reply in the form of an exception,
+                     // so decrement outstanding count and wake up waiters to avoid
+                     // getting hung forever
+                  IOException ie = new IOException("Exceeded retry attempts (" + retryCount + ") in getTransactionStatesFromIntervalX for ASN: " + auditSeqNum);
+                  throw ie;
+               }
+
+               if (LOG.isTraceEnabled()) LOG.trace("getTransactionStatesFromIntervalX -- setting retry, count: " + retryCount);
+               refresh = false;
+            }
+            retryCount++;
+
+            if (retryCount < TLOG_RETRY_ATTEMPTS && retry == true) {
+               try {
+                  Thread.sleep(retrySleep);
+               } catch(InterruptedException ex) {
+                  Thread.currentThread().interrupt();
+               }
+
+               retrySleep += TLOG_SLEEP_INCR;
+            }
+          } while (retryCount < TLOG_RETRY_ATTEMPTS && retry == true);
+
+          if (LOG.isTraceEnabled()) LOG.trace("getTransactionStatesFromIntervalX -- EXIT ASN: " + auditSeqNum);
+          return transList;
+      } //getTransactionStatesFromIntervalX
+   } // TlogCallable2  
+
    private class AuditBuffer{
       private ArrayList<Put> buffer;           // Each Put is an audit record
 
@@ -324,7 +684,6 @@ public class TmAuditTlog {
       private int bufferSize() {
          int lvSize;
          long threadId = Thread.currentThread().getId();
-         if (LOG.isTraceEnabled()) LOG.trace("BufferSize start in thread " + threadId );
          lvSize = buffer.size();
          if (LOG.isTraceEnabled()) LOG.trace("AuditBuffer bufferSize end; returning " + lvSize + " in thread " 
                     +  Thread.currentThread().getId());
@@ -345,6 +704,155 @@ public class TmAuditTlog {
       }
    }// End of class AuditBuffer
 
+   /**
+   * Method  : getTransactionStatesFromInterval
+   * Params  : ClusterId - Trafodion clusterId that was assigned to the beginner of the transaction.
+   *                       Transactions that originate from other clsters will be filtered out from the response.
+   *           nodeId    - Trafodion nodeId of the Tlog set that is to be read.  Typically this
+   *                       id is mapped to the Tlog set as follows Tlog<nodeId>
+   *           pvASN     - ASN after which all audit records will be returned
+   * Return  : ArrayList<TransactionState> 
+   * Purpose : Retrieve list of transactions from an interval
+   */
+   public ArrayList<TransactionState>  getTransactionStatesFromInterval(final long pv_clusterId, final long pv_nodeId, final long pv_ASN) throws IOException {
+
+     long threadId = Thread.currentThread().getId();
+     CompletionService<ArrayList<TransactionState>> compPool = new ExecutorCompletionService<ArrayList<TransactionState>>(tlogThreadPool);
+
+     try {
+        if (LOG.isTraceEnabled()) LOG.trace("getTransactionStatesFromInterval node: " + pv_nodeId
+                      + ", asn: " + pv_ASN + " in thread " + threadId);
+
+        Table targetTable;
+
+        String lv_tLogName = new String("TRAFODION._DTM_.TLOG" + String.valueOf(pv_nodeId));
+        targetTable = connection.getTable(TableName.valueOf(lv_tLogName));
+           
+        compPool.submit(new TlogCallable2(table, connection) {
+           public ArrayList<TransactionState> call() throws IOException {
+              if (LOG.isTraceEnabled()) LOG.trace("before getTransactionStatesFromIntervalX() ASN: "
+                        + pv_ASN + ", clusterId: " + pv_clusterId + " and node: " + pv_nodeId);
+              return getTransactionStatesFromIntervalX(pv_clusterId, pv_ASN);
+           }
+        });
+     } catch (Exception e) {
+        LOG.error("exception in getTransactionStatesFromInterval for interval ASN: " + pv_ASN
+                    + ", node: " + pv_nodeId + " " + e);
+        throw new IOException(e);
+     }
+     // all requests sent at this point, can record the count
+     if (LOG.isTraceEnabled()) LOG.trace("getTransactionStatesFromInterval tlog callable requests sent in thread " + threadId);
+     ArrayList<TransactionState> results = new ArrayList<TransactionState>();
+     try {
+        ArrayList<TransactionState> partialResult = compPool.take().get();
+        for (TransactionState ts : partialResult) {
+           results.add(ts);
+        }
+     }
+     catch (Exception e2) {
+       LOG.error("exception retrieving replies in getTransactionStatesFromInterval for interval ASN: " + pv_ASN
+                   + ", node: " + pv_nodeId + " " + e2);
+       throw new IOException(e2);
+     }
+     if (LOG.isTraceEnabled()) LOG.trace("getTransactionStatesFromInterval tlog callable requests completed in thread "
+         + threadId + ".  " + results.size() + " results returned.");
+     return results;
+   }
+
+   /**
+   * Method  : doTlogWrite
+   * Params  : regionName - name of Region
+   *           transactionId - transaction identifier
+   *           commitId - commitId for the transaction
+   *           put - record representing the commit/abort record for the transaction
+   * Return  : void
+   * Purpose : write commit/abort for a given transaction
+   */
+   public void doTlogWrite(final TransactionState transactionState, final String lvTxState, final Set<TransactionRegionLocation> regions, final boolean hasPeer, boolean forced, long recoveryASN) throws IOException {
+     int loopCount = 0;
+     long threadId = Thread.currentThread().getId();
+     final long lvTransid = transactionState.getTransactionId();
+     if (LOG.isTraceEnabled()) LOG.trace("doTlogWrite for " + transactionState.getTransactionId() + " in thread " + threadId);
+     StringBuilder tableString = new StringBuilder();
+     final long lvCommitId = transactionState.getCommitId();
+     if (regions != null) {
+        // Regions passed in indicate a state record where recovery might be needed following a crash.
+        // To facilitate branch notification we translate the regions into table names that can then
+        // be translated back into new region names following a restart.  THis allows us to ensure all
+        // branches reply prior to cleanup
+        Iterator<TransactionRegionLocation> it = regions.iterator();
+        List<String> tableNameList = new ArrayList<String>();
+        while (it.hasNext()) {
+           String name = new String(it.next().getRegionInfo().getTable().getNameAsString());
+           if ((name.length() > 0) && (tableNameList.contains(name) != true)){
+              // We have a table name not already in the list
+              tableNameList.add(name);
+              tableString.append(",");
+              tableString.append(name);
+           }
+        }
+        if (LOG.isTraceEnabled()) LOG.trace("table names: " + tableString.toString() + " in thread " + threadId);
+     }
+     long key = transactionState.getTransSeqNum();
+     if (LOG.isTraceEnabled()) LOG.trace("key: " + key + ", hex: " + Long.toHexString(key) + ", transid: " +  lvTransid
+   		  + " in thread " + threadId);
+     if (LOG.isTraceEnabled()) LOG.trace("transid: " +  lvTransid + " in thread " + threadId);
+     Put p = new Put(Bytes.toBytes(key));
+     String hasPeerS;
+     if (hasPeer) {
+        hasPeerS = new String ("1");
+     }
+     else {
+        hasPeerS = new String ("0");
+     }
+     long lvAsn;
+     if (recoveryASN == -1){
+        // This is a normal audit record so we manage the ASN
+        lvAsn = asn.get();
+     }
+     else {
+        // This is a recovery audit record so use the ASN passed in
+        lvAsn = recoveryASN;
+     }
+     if (LOG.isTraceEnabled()) LOG.trace("transid: " + lvTransid + " state: " + lvTxState + " ASN: " + lvAsn
+    		  + " in thread " + threadId);
+     p.add(TLOG_FAMILY, ASN_STATE, Bytes.toBytes(String.valueOf(lvAsn) + ","
+                       + String.valueOf(lvTransid) + "," + lvTxState
+                       + "," + Bytes.toString(filler)
+                       + "," + hasPeerS
+                       + "," + String.valueOf(lvCommitId)
+                       + "," + tableString.toString()));
+
+     try {
+        if (LOG.isTraceEnabled()) LOG.trace("doTlogWrite [" + lvTransid + "] in thread " + threadId);
+
+        Table table = connection.getTable(TableName.valueOf(getTlogTableNameBase()));
+        RegionLocator locator = connection.getRegionLocator(table.getName());
+        HRegionLocation location = locator.getRegionLocation(p.getRow());
+        CompletionService<Integer> compPool = new ExecutorCompletionService<Integer>(tlogThreadPool);
+
+        if (LOG.isTraceEnabled()) LOG.trace("doTlogWrite submitting tlog callable in thread " + threadId);
+        final Put p2 = new Put(p);
+
+        compPool.submit(new TlogCallable1(transactionState, location, connection) {
+           public Integer call() throws IOException {
+              if (LOG.isTraceEnabled()) LOG.trace("before doTlogWriteX() [" + transactionState.getTransactionId() + "]" );
+              return doTlogWriteX(location.getRegionInfo().getRegionName(), lvTransid,
+//                         transactionState.getCommitId(), p2, index);
+                         transactionState.getCommitId(), p2);
+           }
+        });
+     } catch (IOException e) {
+        LOG.error("exception in doTlogWrite for transaction: " + lvTransid + " "  + e);
+        //throw new CommitUnsuccessfulException(e);
+        throw e;
+     }
+     // all requests sent at this point, can record the count
+     if (LOG.isTraceEnabled()) LOG.trace("doTlogWrite tlog callable setting requests sent to 1 in thread " + threadId);
+     transactionState.completeSendInvoke(1);
+
+   }
+
    public class TmAuditTlogRegionSplitPolicy extends RegionSplitPolicy {
 
       @Override
@@ -368,7 +876,6 @@ public class TmAuditTlog {
       }
       tlogThreadPool = Executors.newFixedThreadPool(intThreads);
       controlPointDeferred = false;
-
       forceControlPoint = false;
       try {
          String controlPointFlush = System.getenv("TM_TLOG_FLUSH_CONTROL_POINT");
@@ -382,19 +889,6 @@ public class TmAuditTlog {
       }
       LOG.info("forceControlPoint is " + forceControlPoint);
 
-      useAutoFlush = true;
-      try {
-         String autoFlush = System.getenv("TM_TLOG_AUTO_FLUSH");
-         if (autoFlush != null){
-            useAutoFlush = (Integer.parseInt(autoFlush) != 0);
-            if (LOG.isTraceEnabled()) LOG.trace("autoFlush != null");
-         }
-      }
-      catch (NumberFormatException e) {
-         LOG.error("TM_TLOG_AUTO_FLUSH is not valid in ms.env");
-      }
-      LOG.info("useAutoFlush is " + useAutoFlush);
-
       ageCommitted = false;
       try {
          String ageCommittedRecords = System.getenv("TM_TLOG_AGE_COMMITTED_RECORDS");
@@ -536,42 +1030,42 @@ public class TmAuditTlog {
       if (LOG.isTraceEnabled()) LOG.trace("try new HBaseAuditControlPoint");
       tLogControlPoint = new HBaseAuditControlPoint(config, connection);
 
-      tlogAuditLock =    new Object[tlogNumLogs];
-      table = new HTable[tlogNumLogs];
+      // Get the asn from the last control point.  This ignores
+      // any asn increments between the last control point
+      // write and a system crash and could result in asn numbers
+      // being reused.  However this would just mean that some old
+      // records are held onto a bit longer before cleanup and is safe.
+      lvAsn = tLogControlPoint.getStartingAuditSeqNum(myClusterId); 
+      asn.set(lvAsn);
 
-         // Get the asn from the last control point.  This ignores 
-         // any asn increments between the last control point
-         // write and a system crash and could result in asn numbers
-         // being reused.  However this would just mean that some old 
-         // records are held onto a bit longer before cleanup and is safe.
-         asn.set(tLogControlPoint.getStartingAuditSeqNum());
+      LOG.info("Starting Audit Sequence Number is " + lvAsn);
 
       Admin admin  = connection.getAdmin();
-      for (int i = 0 ; i < tlogNumLogs; i++) {
-         tlogAuditLock[i]      = new Object();
-         String lv_tLogName = new String(TLOG_TABLE_NAME + "_LOG_" + Integer.toHexString(i));
-         boolean lvTlogExists = admin.tableExists(TableName.valueOf(lv_tLogName));
-         if (LOG.isTraceEnabled()) LOG.trace("Tlog table " + lv_tLogName + (lvTlogExists? " exists" : " does not exist" ));
-         HTableDescriptor desc = new HTableDescriptor(TableName.valueOf(lv_tLogName));
-         desc.addFamily(hcol);
-
-         if (lvTlogExists == false) {
-            // Need to prime the asn for future writes
-          try {
-              if (LOG.isTraceEnabled()) LOG.trace("Creating the table " + lv_tLogName);
-               admin.createTable(desc);
-               asn.set(1L);  // TLOG didn't exist previously, so start asn at 1
-            }
-            catch (TableExistsException e) {
-               // TODO: ignoring the exception
-               LOG.error("Table " + lv_tLogName + " already exists");
-            }
+      boolean lvTlogExists = admin.tableExists(TableName.valueOf(getTlogTableNameBase()));
+      if (LOG.isTraceEnabled()) LOG.trace("Tlog table " + getTlogTableNameBase() + (lvTlogExists? " exists" : " does not exist" ));
+      HTableDescriptor desc = new HTableDescriptor(TableName.valueOf(getTlogTableNameBase()));
+      desc.setRegionSplitPolicyClassName("org.apache.hadoop.hbase.regionserver.DisabledRegionSplitPolicy");
+      desc.addCoprocessor("org.apache.hadoop.hbase.coprocessor.transactional.TrxRegionEndpoint");
+      desc.addFamily(hcol);
+
+      if (lvTlogExists == false) {
+         // Need to prime the asn for future writes
+         try {
+            if (LOG.isTraceEnabled()) LOG.trace("Creating the table " + getTlogTableNameBase());
+            admin.createTable(desc);
+            asn.set(1L);  // TLOG didn't exist previously, so start asn at 1
          }
-         if (LOG.isTraceEnabled()) LOG.trace("try new HTable index " + i);
-         table[i] = new HTable(config, desc.getName());
-
-         table[i].setAutoFlushTo(this.useAutoFlush);
-
+         catch (TableExistsException e) {
+            LOG.error("Table " + getTlogTableNameBase() + " already exists");
+         }
+      }
+      try {
+         if (LOG.isTraceEnabled()) LOG.trace("try new Table ");
+         table = connection.getTable(TableName.valueOf(getTlogTableNameBase()));
+      }
+      catch(Exception e){
+         LOG.error("TmAuditTlog Exception; ", e);
+         throw new RuntimeException(e);
       }
       admin.close();
 
@@ -579,15 +1073,29 @@ public class TmAuditTlog {
       // This control point write needs to be delayed until after recovery completes, 
       // but is here as a placeholder
       if (LOG.isTraceEnabled()) LOG.trace("Starting a control point with asn value " + lvAsn);
-      tLogControlPointNum = tLogControlPoint.doControlPoint(lvAsn);
+      tLogControlPointNum = tLogControlPoint.doControlPoint(myClusterId, lvAsn, true);
 
       if (LOG.isTraceEnabled()) LOG.trace("Exit constructor()");
       return;
    }
 
-   public long getNextAuditSeqNum(int nid) throws IOException{
-      if (LOG.isTraceEnabled()) LOG.trace("getNextAuditSeqNum node: " + nid);
-      return tLogControlPoint.getNextAuditSeqNum(nid);
+   public long bumpControlPoint(final int clusterId, final int count) throws IOException {
+      if (LOG.isTraceEnabled()) LOG.trace("bumpControlPoint clusterId: " + clusterId + " count: " + count);
+      // Bump the bump the control point as requested, but make sure our asn is still set properly 
+      // reflecting what is stored in the table.  This ignores 
+      // any asn increments between the last control point
+      // write and a system crash and could result in asn numbers
+      // being reused.  However this would just mean that some old 
+      // records are held onto a bit longer before cleanup and is safe.
+      long lvReturn = tLogControlPoint.bumpControlPoint(clusterId, count);
+      asn.set(lvReturn);
+      if (LOG.isTraceEnabled()) LOG.trace("bumpControlPoint resetting asn to: " + lvReturn);
+      return lvReturn;
+   }
+
+   public long getNextAuditSeqNum(int clusterId, int nid) throws IOException{
+      if (LOG.isTraceEnabled()) LOG.trace("getNextAuditSeqNum clusterId: " + clusterId + " node: " + nid);
+      return tLogControlPoint.getNextAuditSeqNum(clusterId, nid);
    }
 
    public static long asnGetAndIncrement () {
@@ -595,11 +1103,14 @@ public class TmAuditTlog {
       return asn.getAndIncrement();
    }
 
-   public void putSingleRecord(final long lvTransid, final long lvCommitId, final String lvTxState, final Set<TransactionRegionLocation> regions, boolean forced) throws IOException {
-      putSingleRecord(lvTransid, lvCommitId, lvTxState, regions, forced, -1);
+   public void putSingleRecord(final long lvTransid, final long lvStartId, final long lvCommitId, final String lvTxState, 
+         final Set<TransactionRegionLocation> regions, final boolean hasPlaceHolder, boolean forced) throws IOException {
+      putSingleRecord(lvTransid, lvStartId, lvCommitId, lvTxState, regions, hasPlaceHolder, forced, -1);
    }
 
-   public void putSingleRecord(final long lvTransid, final long lvCommitId, final String lvTxState, final Set<TransactionRegionLocation> regions, boolean forced, long recoveryASN) throws IOException {
+   public void putSingleRecord(final long lvTransid, final long lvStartId, final long lvCommitId, final String lvTxState, 
+         final Set<TransactionRegionLocation> regions, final boolean hasPlaceHolder, boolean forced, long recoveryASN) throws IOException {
+
       long threadId = Thread.currentThread().getId();
       if (LOG.isTraceEnabled()) LOG.trace("putSingleRecord start in thread " + threadId);
       StringBuilder tableString = new StringBuilder();
@@ -609,10 +1120,12 @@ public class TmAuditTlog {
       long lvAsn;
       long startSynch = 0;
       long endSynch = 0;
-      int lv_lockIndex = 0;
       int lv_TimeIndex = (timeIndex.getAndIncrement() % 50 );
       long lv_TotalWrites = totalWrites.incrementAndGet();
       long lv_TotalRecords = totalRecords.incrementAndGet();
+      Table putTable;
+      putTable = connection.getTable(TableName.valueOf(getTlogTableNameBase()));
+
       if (regions != null) {
          // Regions passed in indicate a state record where recovery might be needed following a crash.
          // To facilitate branch notification we translate the regions into table names that can then
@@ -631,13 +1144,17 @@ public class TmAuditTlog {
          }
          if (LOG.isTraceEnabled()) LOG.trace("table names: " + tableString.toString() + " in thread " + threadId);
       }
-      //Create the Put as directed by the hashed key boolean
-      //create our own hashed key
-      long key = (((lvTransid & tLogHashKey) << tLogHashShiftFactor) + (lvTransid & 0xFFFFFFFF));
-      lv_lockIndex = (int)(lvTransid & tLogHashKey);
-      if (LOG.isTraceEnabled()) LOG.trace("key: " + key + ", hex: " + Long.toHexString(key) + ", transid: " +  lvTransid);
+      long key = lvTransid;
+      if (LOG.isTraceEnabled()) LOG.trace("key: " + key + ", hex: " + Long.toHexString(key) + ", transid: " +  lvTransid
+    		  + " in thread " + threadId);
       Put p = new Put(Bytes.toBytes(key));
-
+      String hasPlaceHolderS;
+      if (hasPlaceHolder) {
+         hasPlaceHolderS = new String ("1");
+      }
+      else {
+         hasPlaceHolderS = new String ("0");
+      }
       if (recoveryASN == -1){
          // This is a normal audit record so we manage the ASN
          lvAsn = asn.getAndIncrement();
@@ -651,9 +1168,13 @@ public class TmAuditTlog {
       p.add(TLOG_FAMILY, ASN_STATE, Bytes.toBytes(String.valueOf(lvAsn) + ","
                        + String.valueOf(lvTransid) + "," + lvTxState
                        + "," + Bytes.toString(filler)
+                       + "," + hasPlaceHolderS
+                       + "," + String.valueOf(lvStartId)
                        + "," + String.valueOf(lvCommitId)
                        + "," + tableString.toString()));
-
+      if (! forced){
+         p.setDurability(Durability.ASYNC_WAL);
+      }
 
       if (recoveryASN != -1){
          // We need to send this to a remote Tlog, not our local one, so open the appropriate table
@@ -661,65 +1182,164 @@ public class TmAuditTlog {
                   + " in thread " + threadId);
          Table recoveryTable;
          int lv_ownerNid = (int)TransactionState.getNodeId(lvTransid);
-         String lv_tLogName = new String("TRAFODION._DTM_.TLOG" + String.valueOf(lv_ownerNid) + "_LOG_" + Integer.toHexString(lv_lockIndex));
-         recoveryTable = connection.getTable(TableName.valueOf(lv_tLogName));
+         recoveryTable = connection.getTable(TableName.valueOf(getTlogTableNameBase()));
+         RegionLocator locator = connection.getRegionLocator(recoveryTable.getName());
 
          try {
-            recoveryTable.put(p);
+            boolean complete = false;
+            int retries = 0;
+            do {
+               try {
+                  retries++;
+                  if (LOG.isTraceEnabled()) LOG.trace("try recovery table.put in thread " + threadId + ", " + p );
+                  recoveryTable.put(p);
+                  complete = true;
+                  if (retries > 1){
+                      if (LOG.isTraceEnabled()) LOG.trace("Retry successful in putSingleRecord for transaction: "
+                             + lvTransid + " on recovery table " + getTlogTableNameBase());
+                   }
+               }
+               catch (RetriesExhaustedWithDetailsException rewde){
+                   LOG.error("Retrying putSingleRecord on recoveryTable for transaction: " + lvTransid + " on table "
+                           + getTlogTableNameBase() + " due to RetriesExhaustedWithDetailsException ", rewde);
+                   locator.getRegionLocation(p.getRow(), true);
+                   try {
+                      Thread.sleep(TlogRetryDelay); // 3 second default
+                   } catch (InterruptedException ie) {
+                   }
+                   if (retries == TlogRetryCount){
+                      LOG.error("putSingleRecord aborting due to excessive retries on recoveryTable for transaction: " + lvTransid + " on table "
+                               + getTlogTableNameBase() + " due to RetriesExhaustedWithDetailsException; aborting ");
+                      System.exit(1);
+                   }
+               }
+               catch (IOException e2){
+                   LOG.error("Retrying putSingleRecord on recoveryTable for transaction: " + lvTransid + " on table "
+                           + getTlogTableNameBase() + " due to Exception ", e2);
+                   locator.getRegionLocation(p.getRow(), true);
+                   try {
+                      Thread.sleep(TlogRetryDelay); // 3 second default
+                   } catch (InterruptedException ie) {
+                   }
+                   if (retries == TlogRetryCount){
+                      LOG.error("putSingleRecord aborting due to excessive retries on recoveryTable for transaction: " + lvTransid + " on table "
+                               + getTlogTableNameBase() + " due t Exception; aborting ");
+                      System.exit(1);
+                   }
+               }
+            } while (! complete && retries < TlogRetryCount);  // default give up after 5 minutes
+         }
+         catch (IOException e2){
+            // create record of the exception
+            LOG.error("putSingleRecord Exception in recoveryTable", e2);
+            throw e2;
          }
          finally {
-            recoveryTable.close();
+               locator.close();
+               recoveryTable.close();
          }
       }
       else {
          // This goes to our local TLOG
-         if (LOG.isTraceEnabled()) LOG.trace("TLOG putSingleRecord synchronizing tlogAuditLock[" + lv_lockIndex + "] in thread " + threadId );
          startSynch = System.nanoTime();
-            synchronized (tlogAuditLock[lv_lockIndex]) {
+         boolean complete = false;
+         int retries = 0;
+         do {
+            retries++;
+            try {
                endSynch = System.nanoTime();
-               try {
-                  if (LOG.isTraceEnabled()) LOG.trace("try table.put " + p );
-                  startTimes[lv_TimeIndex] = System.nanoTime();
-                  table[lv_lockIndex].put(p);
-                  if ((forced) && (useAutoFlush == false)) {
-                     if (LOG.isTraceEnabled()) LOG.trace("flushing commits");
-                     table[lv_lockIndex].flushCommits();
+               startTimes[lv_TimeIndex] = System.nanoTime();
+                  try {
+                     if (LOG.isTraceEnabled()) LOG.trace("try table.put in thread " + threadId + ", " + p );
+                     putTable.put(p);
+                     if (forced){
+                       if (LOG.isTraceEnabled()) LOG.trace("flushing commits in thread " + threadId);
+                     }
+                     endTimes[lv_TimeIndex] = System.nanoTime();
+                     complete = true;
+                     if (retries > 1){
+                        if (LOG.isTraceEnabled()) LOG.trace("Retry successful in putSingleRecord for transaction: " + lvTransid + " on table "
+                              + putTable.getName().getNameAsString());
+                     }
                   }
-                  endTimes[lv_TimeIndex] = System.nanoTime();
-               }
-               catch (IOException e2){
-                  // create record of the exception
-                  LOG.error("putSingleRecord Exception ", e2);
-                  throw e2;
-               }
-            } // End global synchronization
+                  catch (RetriesExhaustedWithDetailsException rewde){
+                     LOG.error("Retry " + retries + " putSingleRecord for transaction: " + lvTransid + " on table "
+                           + putTable.getName().getNameAsString() + " due to RetriesExhaustedWithDetailsException ", rewde);
+
+                     RegionLocator locator = connection.getRegionLocator(TableName.valueOf(getTlogTableNameBase()));
+                     locator.getRegionLocation(p.getRow(), true);
+                     try {
+                        Thread.sleep(TlogRetryDelay); // 3 second default
+                     } catch (InterruptedException ie) {
+                     }
+                     if (retries == TlogRetryCount){
+                        LOG.error("putSingleRecord aborting due to excessive retries for transaction: " + lvTransid + " on table "
+                                 + putTable.getName().getNameAsString() + " due to RetriesExhaustedWithDetailsException; aborting ");
+                        System.exit(1);
+                     }
+                 }
+                 catch (Exception e2){
+                    LOG.error("Retry " + retries + " putSingleRecord for transaction: " + lvTransid + " on table "
+                              + putTable.getName().getNameAsString() + " due to Exception ", e2);
+                    RegionLocator locator = connection.getRegionLocator(TableName.valueOf(getTlogTableNameBase()));
+                    locator.getRegionLocation(p.getRow(), true);
+                    try {
+                       Thread.sleep(TlogRetryDelay); // 3 second default
+                    } catch (InterruptedException ie) {
+                    }
+                    if (retries == TlogRetryCount){
+                       LOG.error("putSingleRecord aborting due to excessive retries for transaction: " + lvTransid + " on table "
+                                  + putTable.getName().getNameAsString() + " due to Exception; aborting ");
+                       System.exit(1);
+                    }
+                 }
+              }
+              catch (IOException e) {
+                 // create record of the exception
+                 LOG.error("PutSingleRecord for transaction:" + lvTransid + " Exception ", e);
+                 try {
+                    Thread.sleep(TlogRetryDelay); // 3 second default
+                 } catch (InterruptedException ie) {
+                 }
+                 RegionLocator locator = connection.getRegionLocator(TableName.valueOf(getTlogTableNameBase()));
+                 locator.getRegionLocation(p.getRow(), true);
+                 if (retries == TlogRetryCount){
+                    LOG.error("putSingleRecord retries exceeded for transaction: " + lvTransid + " on table "
+                           + putTable.getName().getNameAsString() + " due to Exception; Throwing exception");
+                    throw e;
+                 }
+              }
+           } while (! complete && retries < TlogRetryCount);  // default give up after 5 minutes
 
-         if (LOG.isTraceEnabled()) LOG.trace("TLOG putSingleRecord synchronization complete in thread " + threadId );
+           if ( retries > 1){
+              LOG.info("putSingleRecord for transaction: " + lvTransid + " on table "
+                    + putTable.getName().getNameAsString() + " successful after " + retries + " retries");
+           }
 
-         synchTimes[lv_TimeIndex] = endSynch - startSynch;
-         totalSynchTime += synchTimes[lv_TimeIndex];
-         totalWriteTime += (endTimes[lv_TimeIndex] - startTimes[lv_TimeIndex]);
-         if (synchTimes[lv_TimeIndex] > maxSynchTime) {
-            maxSynchTime = synchTimes[lv_TimeIndex];
-         }
-         if (synchTimes[lv_TimeIndex] < minSynchTime) {
-            minSynchTime = synchTimes[lv_TimeIndex];
-         }
-         if ((endTimes[lv_TimeIndex] - startTimes[lv_TimeIndex]) > maxWriteTime) {
-            maxWriteTime = (endTimes[lv_TimeIndex] - startTimes[lv_TimeIndex]);
-         }
-         if ((endTimes[lv_TimeIndex] - startTimes[lv_TimeIndex]) < minWriteTime) {
-            minWriteTime = (endTimes[lv_TimeIndex] - startTimes[lv_TimeIndex]);
-         }
+           synchTimes[lv_TimeIndex] = endSynch - startSynch;
+           totalSynchTime += synchTimes[lv_TimeIndex];
+           totalWriteTime += (endTimes[lv_TimeIndex] - startTimes[lv_TimeIndex]);
+           if (synchTimes[lv_TimeIndex] > maxSynchTime) {
+              maxSynchTime = synchTimes[lv_TimeIndex];
+           }
+           if (synchTimes[lv_TimeIndex] < minSynchTime) {
+              minSynchTime = synchTimes[lv_TimeIndex];
+           }
+           if ((endTimes[lv_TimeIndex] - startTimes[lv_TimeIndex]) > maxWriteTime) {
+              maxWriteTime = (endTimes[lv_TimeIndex] - startTimes[lv_TimeIndex]);
+           }
+           if ((endTimes[lv_TimeIndex] - startTimes[lv_TimeIndex]) < minWriteTime) {
+              minWriteTime = (endTimes[lv_TimeIndex] - startTimes[lv_TimeIndex]);
+           }
 
-         if (lv_TimeIndex == 49) {
-            timeIndex.set(1);  // Start over so we don't exceed the array size
-         }
+           if (lv_TimeIndex == 49) {
+              timeIndex.set(1);  // Start over so we don't exceed the array size
+           }
 
-         if (lv_TotalWrites == 59999) {
-            avgWriteTime = (double) (totalWriteTime/lv_TotalWrites);
-            avgSynchTime = (double) (totalSynchTime/lv_TotalWrites);
-            LOG.info("TLog Audit Write Report\n" + 
+           if (lv_TotalWrites == 59999) {
+              avgWriteTime = (double) (totalWriteTime/lv_TotalWrites);
+              avgSynchTime = (double) (totalSynchTime/lv_TotalWrites);
+              LOG.info("TLog Audit Write Report\n" +
                    "                        Total records: "
                        + lv_TotalRecords + " in " + lv_TotalWrites + " write operations\n" +
                    "                        Write time:\n" +
@@ -737,34 +1357,120 @@ public class TmAuditTlog {
                    "                                     Avg:  " 
                        + avgSynchTime / 1000 + " microseconds\n");
 
-            // Start at index 1 since there is no startTimes[0]
-            timeIndex.set(1);
-            endTimes[0]          = System.nanoTime();
-            totalWriteTime       = 0;
-            totalSynchTime       = 0;
-            totalPrepTime        = 0;
-            totalRecords.set(0);
-            totalWrites.set(0);
-            minWriteTime         = 50000;             // Some arbitrary high value
-            maxWriteTime         = 0;
-            minWriteTimeBuffSize = 0;
-            maxWriteTimeBuffSize = 0;
-            minSynchTime         = 50000;             // Some arbitrary high value
-            maxSynchTime         = 0;
-            minPrepTime          = 50000;            // Some arbitrary high value
-            maxPrepTime          = 0;
-            minBufferSize        = 1000;             // Some arbitrary high value
-            maxBufferSize        = 0;
-         }
-      }// End else revoveryASN == -1
-      if (LOG.isTraceEnabled()) LOG.trace("putSingleRecord exit");
+              // Start at index 1 since there is no startTimes[0]
+              timeIndex.set(1);
+              endTimes[0]          = System.nanoTime();
+              totalWriteTime       = 0;
+              totalSynchTime       = 0;
+              totalPrepTime        = 0;
+              totalRecords.set(0);
+              totalWrites.set(0);
+              minWriteTime         = 50000;             // Some arbitrary high value
+              maxWriteTime         = 0;
+              minWriteTimeBuffSize = 0;
+              maxWriteTimeBuffSize = 0;
+              minSynchTime         = 50000;             // Some arbitrary high value
+              maxSynchTime         = 0;
+              minPrepTime          = 50000;            // Some arbitrary high value
+              maxPrepTime          = 0;
+              minBufferSize        = 1000;             // Some arbitrary high value
+              maxBufferSize        = 0;
+           }
+        }// End else revoveryASN == -1
+        if (LOG.isTraceEnabled()) LOG.trace("putSingleRecord exit");
    }
 
    public static int getRecord(final long lvTransid) throws IOException {
       if (LOG.isTraceEnabled()) LOG.trace("getRecord start");
       TransState lvTxState = TransState.STATE_NOTX;
       String stateString;
-      int lv_lockIndex = (int)(lvTransid & tLogHashKey);
+      Table getTable;
+      getTable = connection.getTable(TableName.valueOf(getTlogTableNameBase()));
+      try {
+         String transidString = new String(String.valueOf(lvTransid));
+         Get g;
+         long key = lvTransid;
+         if (LOG.isTraceEnabled()) LOG.trace("key: " + key + " hex: " + Long.toHexString(key));
+         g = new Get(Bytes.toBytes(key));
+         try {
+            Result r = getTable.get(g);
+            byte [] value = r.getValue(TLOG_FAMILY, ASN_STATE);
+            stateString =  new String (Bytes.toString(value));
+            if (LOG.isTraceEnabled()) LOG.trace("stateString is " + stateString);
+            if (stateString.contains("COMMITTED")){
+                lvTxState = TransState.STATE_COMMITTED;
+            }
+            else if (stateString.contains("ABORTED")){
+               lvTxState = TransState.STATE_ABORTED;
+            }
+            else if (stateString.equals(TransState.STATE_ACTIVE.toString())){
+               lvTxState = TransState.STATE_ACTIVE;
+            }
+            else if (stateString.equals(TransState.STATE_PREPARED.toString())){
+               lvTxState = TransState.STATE_PREPARED;
+            }
+            else if (stateString.equals(TransState.STATE_NOTX.toString())){
+               lvTxState = TransState.STATE_NOTX;
+            }
+            else if (stateString.equals(TransState.STATE_FORGOTTEN.toString())){
+               lvTxState = TransState.STATE_FORGOTTEN;
+            }
+            else if (stateString.equals(TransState.STATE_ABORTING.toString())){
+               lvTxState = TransState.STATE_ABORTING;
+            }
+            else if (stateString.equals(TransState.STATE_COMMITTING.toString())){
+               lvTxState = TransState.STATE_COMMITTING;
+            }
+            else if (stateString.equals(TransState.STATE_PREPARING.toString())){
+               lvTxState = TransState.STATE_PREPARING;
+            }
+            else if (stateString.equals(TransState.STATE_FORGETTING.toString())){
+               lvTxState = TransState.STATE_FORGETTING;
+            }
+            else if (stateString.equals(TransState.STATE_FORGETTING_HEUR.toString())){
+               lvTxState = TransState.STATE_FORGETTING_HEUR;
+            }
+            else if (stateString.equals(TransState.STATE_BEGINNING.toString())){
+               lvTxState = TransState.STATE_BEGINNING;
+            }
+            else if (stateString.equals(TransState.STATE_HUNGCOMMITTED.toString())){
+              lvTxState = TransState.STATE_HUNGCOMMITTED;
+            }
+            else if (stateString.equals(TransState.STATE_HUNGABORTED.toString())){
+               lvTxState = TransState.STATE_HUNGABORTED;
+            }
+            else if (stateString.equals(TransState.STATE_IDLE.toString())){
+               lvTxState = TransState.STATE_IDLE;
+            }
+            else if (stateString.equals(TransState.STATE_FORGOTTEN_HEUR.toString())){
+               lvTxState = TransState.STATE_FORGOTTEN_HEUR;
+            }
+            else if (stateString.equals(TransState.STATE_ABORTING_PART2.toString())){
+               lvTxState = TransState.STATE_ABORTING_PART2;
+            }
+            else if (stateString.equals(TransState.STATE_TERMINATING.toString())){
+                lvTxState = TransState.STATE_TERMINATING;
+            }
+            else {
+               lvTxState = TransState.STATE_BAD;
+            }
+
+            if (LOG.isTraceEnabled()) LOG.trace("transid: " + lvTransid + " state: " + lvTxState);
+         }
+         catch (IOException e){
+             LOG.error("getRecord IOException ", e);
+             throw e;
+         }
+         catch (Exception e){
+             LOG.error("getRecord Exception ", e);
+             throw e;
+         }
+      }
+      catch (Exception e2) {
+            LOG.error("getRecord Exception2 ", e2);
+            e2.printStackTrace();
+      }
+
       if (LOG.isTraceEnabled()) LOG.trace("getRecord end; returning " + lvTxState);
       return lvTxState.getValue();
    }
@@ -772,17 +1478,18 @@ public class TmAuditTlog {
     public static String getRecord(final String transidString) throws IOException {
       if (LOG.isTraceEnabled()) LOG.trace("getRecord start");
       long lvTransid = Long.parseLong(transidString, 10);
-      int lv_lockIndex = (int)(lvTransid & tLogHashKey);
       String lvTxState = new String("NO RECORD");
+      Table getTable;
+      getTable = connection.getTable(TableName.valueOf(getTlogTableNameBase()));
+      try {
          Get g;
-         //create our own hashed key
-         long key = (((lvTransid & tLogHashKey) << tLogHashShiftFactor) + (lvTransid & 0xFFFFFFFF));
+         long key = lvTransid;
          if (LOG.isTraceEnabled()) LOG.trace("key: " + key + " hex: " + Long.toHexString(key));
          g = new Get(Bytes.toBytes(key));
          try {
-            Result r = table[lv_lockIndex].get(g);
-            byte [] value = r.getValue(TLOG_FAMILY, ASN_STATE);
-            StringTokenizer st = new StringTokenizer(value.toString(), ",");
+            Result r = getTable.get(g);
+            StringTokenizer st = 
+                 new StringTokenizer(Bytes.toString(r.getValue(TLOG_FAMILY, ASN_STATE)), ",");
             String asnToken = st.nextElement().toString();
             String transidToken = st.nextElement().toString();
             lvTxState = st.nextElement().toString();
@@ -791,6 +1498,10 @@ public class TmAuditTlog {
              LOG.error("getRecord IOException: ", e);
              throw e;
          }
+      } catch (IOException e){
+             LOG.error("getRecord Exception: ", e);
+             throw e;
+      }
       if (LOG.isTraceEnabled()) LOG.trace("getRecord end; returning String:" + lvTxState);
       return lvTxState;
    }
@@ -798,127 +1509,174 @@ public class TmAuditTlog {
 
    public static boolean deleteRecord(final long lvTransid) throws IOException {
       if (LOG.isTraceEnabled()) LOG.trace("deleteRecord start " + lvTransid);
-      String transidString = new String(String.valueOf(lvTransid));
-      int lv_lockIndex = (int)(lvTransid & tLogHashKey);
-      Delete d;
-      //create our own hashed key
-      long key = (((lvTransid & tLogHashKey) << tLogHashShiftFactor) + (lvTransid & 0xFFFFFFFF));
-      if (LOG.isTraceEnabled()) LOG.trace("key: " + key + " hex: " + Long.toHexString(key));
-      d = new Delete(Bytes.toBytes(key));
-      if (LOG.isTraceEnabled()) LOG.trace("deleteRecord  (" + lvTransid + ") ");
-      table[lv_lockIndex].delete(d);
+      Table deleteTable;
+      deleteTable = connection.getTable(TableName.valueOf(getTlogTableNameBase()));
+      try {
+         Delete d;
+         long key = lvTransid;
+         if (LOG.isTraceEnabled()) LOG.trace("key: " + key + " hex: " + Long.toHexString(key));
+         d = new Delete(Bytes.toBytes(key));
+         d.setDurability(Durability.SKIP_WAL);
+         if (LOG.isTraceEnabled()) LOG.trace("deleteRecord  (" + lvTransid + ") ");
+         deleteTable.delete(d);
+      }
+      catch (Exception e) {
+         LOG.error("deleteRecord Exception: ", e );
+      }
       if (LOG.isTraceEnabled()) LOG.trace("deleteRecord - exit");
       return true;
    }
 
    public boolean deleteAgedEntries(final long lvAsn) throws IOException {
       if (LOG.isTraceEnabled()) LOG.trace("deleteAgedEntries start:  Entries older than " + lvAsn + " will be removed");
-      Table deleteTable;
-      for (int i = 0; i < tlogNumLogs; i++) {
-         String lv_tLogName = new String(TLOG_TABLE_NAME + "_LOG_" + Integer.toHexString(i));
+      String lv_tLogName = new String(getTlogTableNameBase());
+      long deleteCount = 0;
 
-         if (LOG.isTraceEnabled()) LOG.trace("delete table is: " + lv_tLogName);
+      if (LOG.isTraceEnabled()) LOG.trace("delete table is: " + lv_tLogName);
 
-         deleteTable = connection.getTable(TableName.valueOf(lv_tLogName));
-         try {
-            boolean scanComplete = false;
-            Scan s = new Scan();
-            s.setCaching(100);
-            s.setCacheBlocks(false);
-            ArrayList<Delete> deleteList = new ArrayList<Delete>();
-            ResultScanner ss = deleteTable.getScanner(s);
+      // Use a BufferedMutator for client side buffered operations in the same was as autoFlush was used on HTables
+      BufferedMutator deleteMutator = connection.getBufferedMutator(TableName.valueOf(lv_tLogName));
+      Table deleteTable = connection.getTable(TableName.valueOf(lv_tLogName));
+      try {
+         boolean scanComplete = false;
+         Scan s = new Scan();
+         s.setCaching(100);
+         s.setCacheBlocks(false);
+         ArrayList<Delete> deleteList = new ArrayList<Delete>();
+         ResultScanner ss = deleteTable.getScanner(s);
 
-            try {
-               for (Result r : ss) {
-                  for (Cell cell : r.rawCells()) {
-                     StringTokenizer st =
+         try {
+            for (Result r : ss) {
+               if (scanComplete){
+                  if (LOG.isTraceEnabled()) LOG.trace("scanComplete");
+                  break;
+               }
+               for (Cell cell : r.rawCells()) {
+                  StringTokenizer st =
                         new StringTokenizer(Bytes.toString(CellUtil.cloneValue(cell)), ",");
-                     if (LOG.isTraceEnabled()) LOG.trace("string tokenizer success ");
-                     if (st.hasMoreElements()) {
-                        String asnToken = st.nextElement().toString() ;
-                        String transidToken = st.nextElement().toString() ;
-                        String stateToken = st.nextElement().toString() ;
-                        if ((Long.parseLong(asnToken) < lvAsn) && (stateToken.equals("FORGOTTEN"))) {
-                           String rowKey = new String(r.getRow());
+                  if (LOG.isTraceEnabled()) LOG.trace("string tokenizer success ");
+                  if (st.hasMoreElements()) {
+                     String asnToken = st.nextElement().toString();
+                     if (LOG.isTraceEnabled()) LOG.trace("asnToken: " + asnToken);
+                     if (Long.parseLong(asnToken) > lvAsn){
+                        if (LOG.isTraceEnabled()) LOG.trace("RawCells asnToken: " + asnToken
+                                      + " is greater than: " + lvAsn + ".  Scan complete");
+                        scanComplete = true;
+                        break;
+                     }
+                     String transidToken = st.nextElement().toString();
+                     String stateToken = st.nextElement().toString();
+                     if (LOG.isTraceEnabled()) LOG.trace("Transid: " + transidToken + " has state: " + stateToken);
+                     if (LOG.isTraceEnabled()){
+                        long tmp_trans = Long.parseLong(transidToken);
+                        LOG.trace("Transid: " + transidToken + " has sequence: "
+                                  + TransactionState.getTransSeqNum(tmp_trans)
+                                  + ", node: " + TransactionState.getNodeId(tmp_trans)
+                                  + ", clusterId: " + TransactionState.getClusterId(tmp_trans));
+                     }
+                     if ((Long.parseLong(asnToken) < lvAsn) && (stateToken.contains(TransState.STATE_FORGOTTEN.toString()))) {
+                        Delete del = new Delete(r.getRow());
+                        del.setDurability(Durability.SKIP_WAL);
+                        if (LOG.isTraceEnabled()) LOG.trace("adding transid: " + transidToken + " to delete list");
+//                        deleteList.add(del);
+//                        deleteTable.delete(del);
+                          deleteCount++;
+                          deleteMutator.mutate(del);
+                     }
+                     else if ((Long.parseLong(asnToken) < lvAsn) &&
+                             (stateToken.equals(TransState.STATE_COMMITTED.toString()) || stateToken.equals(TransState.STATE_ABORTED.toString()))) {
+                        if (ageCommitted) {
                            Delete del = new Delete(r.getRow());
+                           del.setDurability(Durability.SKIP_WAL);
                            if (LOG.isTraceEnabled()) LOG.trace("adding transid: " + transidToken + " to delete list");
-                           deleteList.add(del);
+//                           deleteList.add(del);
+//                           deleteTable.delete(del);
+                           deleteCount++;
+                           deleteMutator.mutate(del);
                         }
-                        else if ((Long.parseLong(asnToken) < lvAsn) &&
-                                (stateToken.equals("COMMITTED") || stateToken.equals("ABORTED"))) {
-                           if (ageCommitted) {
-                              Delete del = new Delete(r.getRow());
-                              if (LOG.isTraceEnabled()) LOG.trace("adding transid: " + transidToken + " to delete list");
-                              deleteList.add(del);
-                           }
-                           else {
-                              Get get = new Get(r.getRow());
-                              get.setMaxVersions(versions);  // will return last n versions of row
-                              Result lvResult = deleteTable.get(get);
-                              List<Cell> list = lvResult.getColumnCells(TLOG_FAMILY, ASN_STATE);  // returns all versions of this column
-                              for (Cell element : list) {
-                                 StringTokenizer stok = new StringTokenizer(Bytes.toString(CellUtil.cloneValue(element)), ",");
-                                 if (stok.hasMoreElements()) {
-                                    if (LOG.isTraceEnabled()) LOG.trace("Performing secondary search on (" + transidToken + ")");
-                                    asnToken = stok.nextElement().toString() ;
-                                    transidToken = stok.nextElement().toString() ;
-                                    stateToken = stok.nextElement().toString() ;
-                                    if ((Long.parseLong(asnToken) < lvAsn) && (stateToken.equals("FORGOTTEN"))) {
-                                       Delete del = new Delete(r.getRow());
-                                       if (LOG.isTraceEnabled()) LOG.trace("Secondary search found new delete - adding (" + transidToken + ") with asn: " + asnToken + " to delete list");
-                                       deleteList.add(del);
-                                       break;
-                                    }
-                                    else {
-                                       if (LOG.isTraceEnabled()) LOG.trace("Secondary search skipping entry with asn: " + asnToken + ", state: " 
-                                                + stateToken + ", transid: " + transidToken );
-                                    }
+                        else {
+                           Get get = new Get(r.getRow());
+                           get.setMaxVersions(versions);  // will return last n versions of row
+                           Result lvResult = deleteTable.get(get);
+                           List<Cell> list = lvResult.getColumnCells(TLOG_FAMILY, ASN_STATE);  // returns all versions of this column
+                           for (Cell element : list) {
+                              StringTokenizer stok = new StringTokenizer(Bytes.toString(CellUtil.cloneValue(element)), ",");
+                              if (stok.hasMoreElements()) {
+                                 if (LOG.isTraceEnabled()) LOG.trace("Performing secondary search on (" + transidToken + ")");
+                                 asnToken = stok.nextElement().toString() ;
+                                 transidToken = stok.nextElement().toString() ;
+                                 stateToken = stok.nextElement().toString() ;
+                                 if ((Long.parseLong(asnToken) < lvAsn) && (stateToken.contains(TransState.STATE_FORGOTTEN.toString()))) {
+                                    Delete del = new Delete(r.getRow());
+                                    del.setDurability(Durability.SKIP_WAL);
+                                    if (LOG.isTraceEnabled()) LOG.trace("Secondary search found new delete - adding (" + transidToken + ") with asn: " + asnToken + " to delete list");
+//                                    deleteList.add(del);
+//                                    deleteTable.delete(del);
+                                    deleteCount++;
+                                    deleteMutator.mutate(del);
+                                    break;
+                                 }
+                                 else {
+                                    if (LOG.isTraceEnabled()) LOG.trace("Secondary search skipping entry with asn: " + asnToken + ", state: "
+                                             + stateToken + ", transid: " + transidToken );
                                  }
                               }
                            }
-                        } else {
-                           if (LOG.isTraceEnabled()) LOG.trace("deleteAgedEntries skipping asn: " + asnToken + ", transid: " 
-                                     + transidToken + ", state: " + stateToken);
                         }
+                     } else {
+                        if (LOG.isTraceEnabled()) LOG.trace("deleteAgedEntries skipping asn: " + asnToken + ", transid: "
+                                  + transidToken + ", state: " + stateToken);
                      }
                   }
-              }
-           }
-           finally {
-              if (LOG.isTraceEnabled()) LOG.trace("deleteAgedEntries closing ResultScanner");
-              ss.close();
-           }
-           if (LOG.isTraceEnabled()) LOG.trace("attempting to delete list with " + deleteList.size()
-                   + " elements from table " + lv_tLogName);
-           deleteTable.delete(deleteList);
-        }
-        finally {
-              deleteTable.close();
-        }
+               }
+            }
+         }
+         catch(Exception e){
+            LOG.error("deleteAgedEntries Exception getting results for table " + lv_tLogName + "; ", e);
+            throw new RuntimeException(e);
+         }
+         finally {
+            if (LOG.isTraceEnabled()) LOG.trace("deleteAgedEntries closing ResultScanner");
+            ss.close();
+         }
+      }
+      catch (IOException e) {
+         LOG.error("deleteAgedEntries IOException setting up scan on table "
+                   + lv_tLogName + ", Exception: ", e);
+         e.printStackTrace();
+      }
+      finally {
+         try {
+            if (LOG.isTraceEnabled()) LOG.trace("deleteAgedEntries closing table for "
+                 + lv_tLogName +"; " + deleteCount + " entries deleted");
+            deleteMutator.close();
+            deleteTable.close();
+         }
+         catch (IOException e) {
+            LOG.error("deleteAgedEntries IOException closing table " + lv_tLogName + " Exception: " + e);
+         }
      }
      if (LOG.isTraceEnabled()) LOG.trace("deleteAgedEntries - exit");
      return true;
    }
 
-   public long writeControlPointRecords (final Map<Long, TransactionState> map) throws IOException {
-      int lv_lockIndex;
+   public long writeControlPointRecords (final int clusterId, final Map<Long, TransactionState> map) throws IOException {
       int cpWrites = 0;
       long startTime = System.nanoTime();
       long endTime;
 
       if (LOG.isTraceEnabled()) LOG.trace("Tlog " + getTlogTableNameBase()
-           + " writeControlPointRecords start with map size " + map.size());
+           + " writeControlPointRecords for clusterId " + clusterId + " start with map size " + map.size());
 
       try {
         for (Map.Entry<Long, TransactionState> e : map.entrySet()) {
          try {
             Long transid = e.getKey();
-            lv_lockIndex = (int)(transid & tLogHashKey);
             TransactionState value = e.getValue();
-            if (value.getStatus().equals("COMMITTED")){
+            if (value.getStatus().equals(TransState.STATE_COMMITTED.toString())){
                if (LOG.isTraceEnabled()) LOG.trace("writeControlPointRecords adding record for trans (" + transid + ") : state is " + value.getStatus());
                cpWrites++;
-               putSingleRecord(transid, value.getCommitId(), value.getStatus(), value.getParticipatingRegions(), forceControlPoint);
+               putSingleRecord(transid, value.getStartId(), value.getCommitId(), value.getStatus(), value.getParticipatingRegions(), value.hasPlaceHolder(), forceControlPoint);
             }
          }
          catch (IOException ex) {
@@ -930,7 +1688,7 @@ public class TmAuditTlog {
           LOG.info("writeControlPointRecords ConcurrentModificationException;  delaying control point ", cme);
           // Retu

<TRUNCATED>


[05/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

Posted by sa...@apache.org.
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/coprocessor/transactional/generated/TrxRegionProtos.java
----------------------------------------------------------------------
diff --git a/core/sqf/src/seatrans/hbase-trx/src/main/java/org/apache/hadoop/hbase/coprocessor/transactional/generated/TrxRegionProtos.java b/core/sqf/src/seatrans/hbase-trx/src/main/java/org/apache/hadoop/hbase/coprocessor/transactional/generated/TrxRegionProtos.java
index f571952..019297b 100755
--- a/core/sqf/src/seatrans/hbase-trx/src/main/java/org/apache/hadoop/hbase/coprocessor/transactional/generated/TrxRegionProtos.java
+++ b/core/sqf/src/seatrans/hbase-trx/src/main/java/org/apache/hadoop/hbase/coprocessor/transactional/generated/TrxRegionProtos.java
@@ -1,24 +1,3 @@
-// @@@ START COPYRIGHT @@@
-//
-// Licensed to the Apache Software Foundation (ASF) under one
-// or more contributor license agreements.  See the NOTICE file
-// distributed with this work for additional information
-// regarding copyright ownership.  The ASF licenses this file
-// to you under the Apache License, Version 2.0 (the
-// "License"); you may not use this file except in compliance
-// with the License.  You may obtain a copy of the License at
-//
-//   http://www.apache.org/licenses/LICENSE-2.0
-//
-// Unless required by applicable law or agreed to in writing,
-// software distributed under the License is distributed on an
-// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
-// KIND, either express or implied.  See the License for the
-// specific language governing permissions and limitations
-// under the License.
-//
-// @@@ END COPYRIGHT @@@
-
 // Generated by the protocol buffer compiler.  DO NOT EDIT!
 // source: TrxRegion.proto
 
@@ -71,6 +50,16 @@ public final class TrxRegionProtos {
      * <code>required bool dropTableRecorded = 4;</code>
      */
     boolean getDropTableRecorded();
+
+    // optional bool ignoreUnknownTransactionException = 5;
+    /**
+     * <code>optional bool ignoreUnknownTransactionException = 5;</code>
+     */
+    boolean hasIgnoreUnknownTransactionException();
+    /**
+     * <code>optional bool ignoreUnknownTransactionException = 5;</code>
+     */
+    boolean getIgnoreUnknownTransactionException();
   }
   /**
    * Protobuf type {@code AbortTransactionRequest}
@@ -143,6 +132,11 @@ public final class TrxRegionProtos {
               dropTableRecorded_ = input.readBool();
               break;
             }
+            case 40: {
+              bitField0_ |= 0x00000010;
+              ignoreUnknownTransactionException_ = input.readBool();
+              break;
+            }
           }
         }
       } catch (com.google.protobuf.InvalidProtocolBufferException e) {
@@ -247,11 +241,28 @@ public final class TrxRegionProtos {
       return dropTableRecorded_;
     }
 
+    // optional bool ignoreUnknownTransactionException = 5;
+    public static final int IGNOREUNKNOWNTRANSACTIONEXCEPTION_FIELD_NUMBER = 5;
+    private boolean ignoreUnknownTransactionException_;
+    /**
+     * <code>optional bool ignoreUnknownTransactionException = 5;</code>
+     */
+    public boolean hasIgnoreUnknownTransactionException() {
+      return ((bitField0_ & 0x00000010) == 0x00000010);
+    }
+    /**
+     * <code>optional bool ignoreUnknownTransactionException = 5;</code>
+     */
+    public boolean getIgnoreUnknownTransactionException() {
+      return ignoreUnknownTransactionException_;
+    }
+
     private void initFields() {
       regionName_ = com.google.protobuf.ByteString.EMPTY;
       transactionId_ = 0L;
       participantNum_ = 0;
       dropTableRecorded_ = false;
+      ignoreUnknownTransactionException_ = false;
     }
     private byte memoizedIsInitialized = -1;
     public final boolean isInitialized() {
@@ -293,6 +304,9 @@ public final class TrxRegionProtos {
       if (((bitField0_ & 0x00000008) == 0x00000008)) {
         output.writeBool(4, dropTableRecorded_);
       }
+      if (((bitField0_ & 0x00000010) == 0x00000010)) {
+        output.writeBool(5, ignoreUnknownTransactionException_);
+      }
       getUnknownFields().writeTo(output);
     }
 
@@ -318,6 +332,10 @@ public final class TrxRegionProtos {
         size += com.google.protobuf.CodedOutputStream
           .computeBoolSize(4, dropTableRecorded_);
       }
+      if (((bitField0_ & 0x00000010) == 0x00000010)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeBoolSize(5, ignoreUnknownTransactionException_);
+      }
       size += getUnknownFields().getSerializedSize();
       memoizedSerializedSize = size;
       return size;
@@ -442,6 +460,8 @@ public final class TrxRegionProtos {
         bitField0_ = (bitField0_ & ~0x00000004);
         dropTableRecorded_ = false;
         bitField0_ = (bitField0_ & ~0x00000008);
+        ignoreUnknownTransactionException_ = false;
+        bitField0_ = (bitField0_ & ~0x00000010);
         return this;
       }
 
@@ -486,6 +506,10 @@ public final class TrxRegionProtos {
           to_bitField0_ |= 0x00000008;
         }
         result.dropTableRecorded_ = dropTableRecorded_;
+        if (((from_bitField0_ & 0x00000010) == 0x00000010)) {
+          to_bitField0_ |= 0x00000010;
+        }
+        result.ignoreUnknownTransactionException_ = ignoreUnknownTransactionException_;
         result.bitField0_ = to_bitField0_;
         onBuilt();
         return result;
@@ -514,6 +538,9 @@ public final class TrxRegionProtos {
         if (other.hasDropTableRecorded()) {
           setDropTableRecorded(other.getDropTableRecorded());
         }
+        if (other.hasIgnoreUnknownTransactionException()) {
+          setIgnoreUnknownTransactionException(other.getIgnoreUnknownTransactionException());
+        }
         this.mergeUnknownFields(other.getUnknownFields());
         return this;
       }
@@ -692,6 +719,39 @@ public final class TrxRegionProtos {
         return this;
       }
 
+      // optional bool ignoreUnknownTransactionException = 5;
+      private boolean ignoreUnknownTransactionException_ ;
+      /**
+       * <code>optional bool ignoreUnknownTransactionException = 5;</code>
+       */
+      public boolean hasIgnoreUnknownTransactionException() {
+        return ((bitField0_ & 0x00000010) == 0x00000010);
+      }
+      /**
+       * <code>optional bool ignoreUnknownTransactionException = 5;</code>
+       */
+      public boolean getIgnoreUnknownTransactionException() {
+        return ignoreUnknownTransactionException_;
+      }
+      /**
+       * <code>optional bool ignoreUnknownTransactionException = 5;</code>
+       */
+      public Builder setIgnoreUnknownTransactionException(boolean value) {
+        bitField0_ |= 0x00000010;
+        ignoreUnknownTransactionException_ = value;
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>optional bool ignoreUnknownTransactionException = 5;</code>
+       */
+      public Builder clearIgnoreUnknownTransactionException() {
+        bitField0_ = (bitField0_ & ~0x00000010);
+        ignoreUnknownTransactionException_ = false;
+        onChanged();
+        return this;
+      }
+
       // @@protoc_insertion_point(builder_scope:AbortTransactionRequest)
     }
 
@@ -2492,13 +2552,23 @@ public final class TrxRegionProtos {
      */
     long getTransactionId();
 
-    // required bytes regionName = 2;
+    // required int64 startId = 2;
     /**
-     * <code>required bytes regionName = 2;</code>
+     * <code>required int64 startId = 2;</code>
+     */
+    boolean hasStartId();
+    /**
+     * <code>required int64 startId = 2;</code>
+     */
+    long getStartId();
+
+    // required bytes regionName = 3;
+    /**
+     * <code>required bytes regionName = 3;</code>
      */
     boolean hasRegionName();
     /**
-     * <code>required bytes regionName = 2;</code>
+     * <code>required bytes regionName = 3;</code>
      */
     com.google.protobuf.ByteString getRegionName();
   }
@@ -2558,8 +2628,13 @@ public final class TrxRegionProtos {
               transactionId_ = input.readInt64();
               break;
             }
-            case 18: {
+            case 16: {
               bitField0_ |= 0x00000002;
+              startId_ = input.readInt64();
+              break;
+            }
+            case 26: {
+              bitField0_ |= 0x00000004;
               regionName_ = input.readBytes();
               break;
             }
@@ -2619,17 +2694,33 @@ public final class TrxRegionProtos {
       return transactionId_;
     }
 
-    // required bytes regionName = 2;
-    public static final int REGIONNAME_FIELD_NUMBER = 2;
+    // required int64 startId = 2;
+    public static final int STARTID_FIELD_NUMBER = 2;
+    private long startId_;
+    /**
+     * <code>required int64 startId = 2;</code>
+     */
+    public boolean hasStartId() {
+      return ((bitField0_ & 0x00000002) == 0x00000002);
+    }
+    /**
+     * <code>required int64 startId = 2;</code>
+     */
+    public long getStartId() {
+      return startId_;
+    }
+
+    // required bytes regionName = 3;
+    public static final int REGIONNAME_FIELD_NUMBER = 3;
     private com.google.protobuf.ByteString regionName_;
     /**
-     * <code>required bytes regionName = 2;</code>
+     * <code>required bytes regionName = 3;</code>
      */
     public boolean hasRegionName() {
-      return ((bitField0_ & 0x00000002) == 0x00000002);
+      return ((bitField0_ & 0x00000004) == 0x00000004);
     }
     /**
-     * <code>required bytes regionName = 2;</code>
+     * <code>required bytes regionName = 3;</code>
      */
     public com.google.protobuf.ByteString getRegionName() {
       return regionName_;
@@ -2637,6 +2728,7 @@ public final class TrxRegionProtos {
 
     private void initFields() {
       transactionId_ = 0L;
+      startId_ = 0L;
       regionName_ = com.google.protobuf.ByteString.EMPTY;
     }
     private byte memoizedIsInitialized = -1;
@@ -2648,6 +2740,10 @@ public final class TrxRegionProtos {
         memoizedIsInitialized = 0;
         return false;
       }
+      if (!hasStartId()) {
+        memoizedIsInitialized = 0;
+        return false;
+      }
       if (!hasRegionName()) {
         memoizedIsInitialized = 0;
         return false;
@@ -2663,7 +2759,10 @@ public final class TrxRegionProtos {
         output.writeInt64(1, transactionId_);
       }
       if (((bitField0_ & 0x00000002) == 0x00000002)) {
-        output.writeBytes(2, regionName_);
+        output.writeInt64(2, startId_);
+      }
+      if (((bitField0_ & 0x00000004) == 0x00000004)) {
+        output.writeBytes(3, regionName_);
       }
       getUnknownFields().writeTo(output);
     }
@@ -2680,7 +2779,11 @@ public final class TrxRegionProtos {
       }
       if (((bitField0_ & 0x00000002) == 0x00000002)) {
         size += com.google.protobuf.CodedOutputStream
-          .computeBytesSize(2, regionName_);
+          .computeInt64Size(2, startId_);
+      }
+      if (((bitField0_ & 0x00000004) == 0x00000004)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeBytesSize(3, regionName_);
       }
       size += getUnknownFields().getSerializedSize();
       memoizedSerializedSize = size;
@@ -2800,8 +2903,10 @@ public final class TrxRegionProtos {
         super.clear();
         transactionId_ = 0L;
         bitField0_ = (bitField0_ & ~0x00000001);
-        regionName_ = com.google.protobuf.ByteString.EMPTY;
+        startId_ = 0L;
         bitField0_ = (bitField0_ & ~0x00000002);
+        regionName_ = com.google.protobuf.ByteString.EMPTY;
+        bitField0_ = (bitField0_ & ~0x00000004);
         return this;
       }
 
@@ -2837,6 +2942,10 @@ public final class TrxRegionProtos {
         if (((from_bitField0_ & 0x00000002) == 0x00000002)) {
           to_bitField0_ |= 0x00000002;
         }
+        result.startId_ = startId_;
+        if (((from_bitField0_ & 0x00000004) == 0x00000004)) {
+          to_bitField0_ |= 0x00000004;
+        }
         result.regionName_ = regionName_;
         result.bitField0_ = to_bitField0_;
         onBuilt();
@@ -2857,6 +2966,9 @@ public final class TrxRegionProtos {
         if (other.hasTransactionId()) {
           setTransactionId(other.getTransactionId());
         }
+        if (other.hasStartId()) {
+          setStartId(other.getStartId());
+        }
         if (other.hasRegionName()) {
           setRegionName(other.getRegionName());
         }
@@ -2869,6 +2981,10 @@ public final class TrxRegionProtos {
           
           return false;
         }
+        if (!hasStartId()) {
+          
+          return false;
+        }
         if (!hasRegionName()) {
           
           return false;
@@ -2928,37 +3044,70 @@ public final class TrxRegionProtos {
         return this;
       }
 
-      // required bytes regionName = 2;
+      // required int64 startId = 2;
+      private long startId_ ;
+      /**
+       * <code>required int64 startId = 2;</code>
+       */
+      public boolean hasStartId() {
+        return ((bitField0_ & 0x00000002) == 0x00000002);
+      }
+      /**
+       * <code>required int64 startId = 2;</code>
+       */
+      public long getStartId() {
+        return startId_;
+      }
+      /**
+       * <code>required int64 startId = 2;</code>
+       */
+      public Builder setStartId(long value) {
+        bitField0_ |= 0x00000002;
+        startId_ = value;
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>required int64 startId = 2;</code>
+       */
+      public Builder clearStartId() {
+        bitField0_ = (bitField0_ & ~0x00000002);
+        startId_ = 0L;
+        onChanged();
+        return this;
+      }
+
+      // required bytes regionName = 3;
       private com.google.protobuf.ByteString regionName_ = com.google.protobuf.ByteString.EMPTY;
       /**
-       * <code>required bytes regionName = 2;</code>
+       * <code>required bytes regionName = 3;</code>
        */
       public boolean hasRegionName() {
-        return ((bitField0_ & 0x00000002) == 0x00000002);
+        return ((bitField0_ & 0x00000004) == 0x00000004);
       }
       /**
-       * <code>required bytes regionName = 2;</code>
+       * <code>required bytes regionName = 3;</code>
        */
       public com.google.protobuf.ByteString getRegionName() {
         return regionName_;
       }
       /**
-       * <code>required bytes regionName = 2;</code>
+       * <code>required bytes regionName = 3;</code>
        */
       public Builder setRegionName(com.google.protobuf.ByteString value) {
         if (value == null) {
     throw new NullPointerException();
   }
-  bitField0_ |= 0x00000002;
+  bitField0_ |= 0x00000004;
         regionName_ = value;
         onChanged();
         return this;
       }
       /**
-       * <code>required bytes regionName = 2;</code>
+       * <code>required bytes regionName = 3;</code>
        */
       public Builder clearRegionName() {
-        bitField0_ = (bitField0_ & ~0x00000002);
+        bitField0_ = (bitField0_ & ~0x00000004);
         regionName_ = getDefaultInstance().getRegionName();
         onChanged();
         return this;
@@ -3550,23 +3699,33 @@ public final class TrxRegionProtos {
      */
     long getTransactionId();
 
-    // required int32 participantNum = 3;
+    // required int64 commitId = 3;
     /**
-     * <code>required int32 participantNum = 3;</code>
+     * <code>required int64 commitId = 3;</code>
+     */
+    boolean hasCommitId();
+    /**
+     * <code>required int64 commitId = 3;</code>
+     */
+    long getCommitId();
+
+    // required int32 participantNum = 4;
+    /**
+     * <code>required int32 participantNum = 4;</code>
      */
     boolean hasParticipantNum();
     /**
-     * <code>required int32 participantNum = 3;</code>
+     * <code>required int32 participantNum = 4;</code>
      */
     int getParticipantNum();
 
-    // optional bool ignoreUnknownTransactionException = 4;
+    // optional bool ignoreUnknownTransactionException = 5;
     /**
-     * <code>optional bool ignoreUnknownTransactionException = 4;</code>
+     * <code>optional bool ignoreUnknownTransactionException = 5;</code>
      */
     boolean hasIgnoreUnknownTransactionException();
     /**
-     * <code>optional bool ignoreUnknownTransactionException = 4;</code>
+     * <code>optional bool ignoreUnknownTransactionException = 5;</code>
      */
     boolean getIgnoreUnknownTransactionException();
   }
@@ -3633,11 +3792,16 @@ public final class TrxRegionProtos {
             }
             case 24: {
               bitField0_ |= 0x00000004;
-              participantNum_ = input.readInt32();
+              commitId_ = input.readInt64();
               break;
             }
             case 32: {
               bitField0_ |= 0x00000008;
+              participantNum_ = input.readInt32();
+              break;
+            }
+            case 40: {
+              bitField0_ |= 0x00000010;
               ignoreUnknownTransactionException_ = input.readBool();
               break;
             }
@@ -3713,33 +3877,49 @@ public final class TrxRegionProtos {
       return transactionId_;
     }
 
-    // required int32 participantNum = 3;
-    public static final int PARTICIPANTNUM_FIELD_NUMBER = 3;
+    // required int64 commitId = 3;
+    public static final int COMMITID_FIELD_NUMBER = 3;
+    private long commitId_;
+    /**
+     * <code>required int64 commitId = 3;</code>
+     */
+    public boolean hasCommitId() {
+      return ((bitField0_ & 0x00000004) == 0x00000004);
+    }
+    /**
+     * <code>required int64 commitId = 3;</code>
+     */
+    public long getCommitId() {
+      return commitId_;
+    }
+
+    // required int32 participantNum = 4;
+    public static final int PARTICIPANTNUM_FIELD_NUMBER = 4;
     private int participantNum_;
     /**
-     * <code>required int32 participantNum = 3;</code>
+     * <code>required int32 participantNum = 4;</code>
      */
     public boolean hasParticipantNum() {
-      return ((bitField0_ & 0x00000004) == 0x00000004);
+      return ((bitField0_ & 0x00000008) == 0x00000008);
     }
     /**
-     * <code>required int32 participantNum = 3;</code>
+     * <code>required int32 participantNum = 4;</code>
      */
     public int getParticipantNum() {
       return participantNum_;
     }
 
-    // optional bool ignoreUnknownTransactionException = 4;
-    public static final int IGNOREUNKNOWNTRANSACTIONEXCEPTION_FIELD_NUMBER = 4;
+    // optional bool ignoreUnknownTransactionException = 5;
+    public static final int IGNOREUNKNOWNTRANSACTIONEXCEPTION_FIELD_NUMBER = 5;
     private boolean ignoreUnknownTransactionException_;
     /**
-     * <code>optional bool ignoreUnknownTransactionException = 4;</code>
+     * <code>optional bool ignoreUnknownTransactionException = 5;</code>
      */
     public boolean hasIgnoreUnknownTransactionException() {
-      return ((bitField0_ & 0x00000008) == 0x00000008);
+      return ((bitField0_ & 0x00000010) == 0x00000010);
     }
     /**
-     * <code>optional bool ignoreUnknownTransactionException = 4;</code>
+     * <code>optional bool ignoreUnknownTransactionException = 5;</code>
      */
     public boolean getIgnoreUnknownTransactionException() {
       return ignoreUnknownTransactionException_;
@@ -3748,6 +3928,7 @@ public final class TrxRegionProtos {
     private void initFields() {
       regionName_ = com.google.protobuf.ByteString.EMPTY;
       transactionId_ = 0L;
+      commitId_ = 0L;
       participantNum_ = 0;
       ignoreUnknownTransactionException_ = false;
     }
@@ -3764,6 +3945,10 @@ public final class TrxRegionProtos {
         memoizedIsInitialized = 0;
         return false;
       }
+      if (!hasCommitId()) {
+        memoizedIsInitialized = 0;
+        return false;
+      }
       if (!hasParticipantNum()) {
         memoizedIsInitialized = 0;
         return false;
@@ -3782,10 +3967,13 @@ public final class TrxRegionProtos {
         output.writeInt64(2, transactionId_);
       }
       if (((bitField0_ & 0x00000004) == 0x00000004)) {
-        output.writeInt32(3, participantNum_);
+        output.writeInt64(3, commitId_);
       }
       if (((bitField0_ & 0x00000008) == 0x00000008)) {
-        output.writeBool(4, ignoreUnknownTransactionException_);
+        output.writeInt32(4, participantNum_);
+      }
+      if (((bitField0_ & 0x00000010) == 0x00000010)) {
+        output.writeBool(5, ignoreUnknownTransactionException_);
       }
       getUnknownFields().writeTo(output);
     }
@@ -3806,11 +3994,15 @@ public final class TrxRegionProtos {
       }
       if (((bitField0_ & 0x00000004) == 0x00000004)) {
         size += com.google.protobuf.CodedOutputStream
-          .computeInt32Size(3, participantNum_);
+          .computeInt64Size(3, commitId_);
       }
       if (((bitField0_ & 0x00000008) == 0x00000008)) {
         size += com.google.protobuf.CodedOutputStream
-          .computeBoolSize(4, ignoreUnknownTransactionException_);
+          .computeInt32Size(4, participantNum_);
+      }
+      if (((bitField0_ & 0x00000010) == 0x00000010)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeBoolSize(5, ignoreUnknownTransactionException_);
       }
       size += getUnknownFields().getSerializedSize();
       memoizedSerializedSize = size;
@@ -3932,10 +4124,12 @@ public final class TrxRegionProtos {
         bitField0_ = (bitField0_ & ~0x00000001);
         transactionId_ = 0L;
         bitField0_ = (bitField0_ & ~0x00000002);
-        participantNum_ = 0;
+        commitId_ = 0L;
         bitField0_ = (bitField0_ & ~0x00000004);
-        ignoreUnknownTransactionException_ = false;
+        participantNum_ = 0;
         bitField0_ = (bitField0_ & ~0x00000008);
+        ignoreUnknownTransactionException_ = false;
+        bitField0_ = (bitField0_ & ~0x00000010);
         return this;
       }
 
@@ -3975,10 +4169,14 @@ public final class TrxRegionProtos {
         if (((from_bitField0_ & 0x00000004) == 0x00000004)) {
           to_bitField0_ |= 0x00000004;
         }
-        result.participantNum_ = participantNum_;
+        result.commitId_ = commitId_;
         if (((from_bitField0_ & 0x00000008) == 0x00000008)) {
           to_bitField0_ |= 0x00000008;
         }
+        result.participantNum_ = participantNum_;
+        if (((from_bitField0_ & 0x00000010) == 0x00000010)) {
+          to_bitField0_ |= 0x00000010;
+        }
         result.ignoreUnknownTransactionException_ = ignoreUnknownTransactionException_;
         result.bitField0_ = to_bitField0_;
         onBuilt();
@@ -4002,6 +4200,9 @@ public final class TrxRegionProtos {
         if (other.hasTransactionId()) {
           setTransactionId(other.getTransactionId());
         }
+        if (other.hasCommitId()) {
+          setCommitId(other.getCommitId());
+        }
         if (other.hasParticipantNum()) {
           setParticipantNum(other.getParticipantNum());
         }
@@ -4021,6 +4222,10 @@ public final class TrxRegionProtos {
           
           return false;
         }
+        if (!hasCommitId()) {
+          
+          return false;
+        }
         if (!hasParticipantNum()) {
           
           return false;
@@ -4116,67 +4321,100 @@ public final class TrxRegionProtos {
         return this;
       }
 
-      // required int32 participantNum = 3;
+      // required int64 commitId = 3;
+      private long commitId_ ;
+      /**
+       * <code>required int64 commitId = 3;</code>
+       */
+      public boolean hasCommitId() {
+        return ((bitField0_ & 0x00000004) == 0x00000004);
+      }
+      /**
+       * <code>required int64 commitId = 3;</code>
+       */
+      public long getCommitId() {
+        return commitId_;
+      }
+      /**
+       * <code>required int64 commitId = 3;</code>
+       */
+      public Builder setCommitId(long value) {
+        bitField0_ |= 0x00000004;
+        commitId_ = value;
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>required int64 commitId = 3;</code>
+       */
+      public Builder clearCommitId() {
+        bitField0_ = (bitField0_ & ~0x00000004);
+        commitId_ = 0L;
+        onChanged();
+        return this;
+      }
+
+      // required int32 participantNum = 4;
       private int participantNum_ ;
       /**
-       * <code>required int32 participantNum = 3;</code>
+       * <code>required int32 participantNum = 4;</code>
        */
       public boolean hasParticipantNum() {
-        return ((bitField0_ & 0x00000004) == 0x00000004);
+        return ((bitField0_ & 0x00000008) == 0x00000008);
       }
       /**
-       * <code>required int32 participantNum = 3;</code>
+       * <code>required int32 participantNum = 4;</code>
        */
       public int getParticipantNum() {
         return participantNum_;
       }
       /**
-       * <code>required int32 participantNum = 3;</code>
+       * <code>required int32 participantNum = 4;</code>
        */
       public Builder setParticipantNum(int value) {
-        bitField0_ |= 0x00000004;
+        bitField0_ |= 0x00000008;
         participantNum_ = value;
         onChanged();
         return this;
       }
       /**
-       * <code>required int32 participantNum = 3;</code>
+       * <code>required int32 participantNum = 4;</code>
        */
       public Builder clearParticipantNum() {
-        bitField0_ = (bitField0_ & ~0x00000004);
+        bitField0_ = (bitField0_ & ~0x00000008);
         participantNum_ = 0;
         onChanged();
         return this;
       }
 
-      // optional bool ignoreUnknownTransactionException = 4;
+      // optional bool ignoreUnknownTransactionException = 5;
       private boolean ignoreUnknownTransactionException_ ;
       /**
-       * <code>optional bool ignoreUnknownTransactionException = 4;</code>
+       * <code>optional bool ignoreUnknownTransactionException = 5;</code>
        */
       public boolean hasIgnoreUnknownTransactionException() {
-        return ((bitField0_ & 0x00000008) == 0x00000008);
+        return ((bitField0_ & 0x00000010) == 0x00000010);
       }
       /**
-       * <code>optional bool ignoreUnknownTransactionException = 4;</code>
+       * <code>optional bool ignoreUnknownTransactionException = 5;</code>
        */
       public boolean getIgnoreUnknownTransactionException() {
         return ignoreUnknownTransactionException_;
       }
       /**
-       * <code>optional bool ignoreUnknownTransactionException = 4;</code>
+       * <code>optional bool ignoreUnknownTransactionException = 5;</code>
        */
       public Builder setIgnoreUnknownTransactionException(boolean value) {
-        bitField0_ |= 0x00000008;
+        bitField0_ |= 0x00000010;
         ignoreUnknownTransactionException_ = value;
         onChanged();
         return this;
       }
       /**
-       * <code>optional bool ignoreUnknownTransactionException = 4;</code>
+       * <code>optional bool ignoreUnknownTransactionException = 5;</code>
        */
       public Builder clearIgnoreUnknownTransactionException() {
-        bitField0_ = (bitField0_ & ~0x00000008);
+        bitField0_ = (bitField0_ & ~0x00000010);
         ignoreUnknownTransactionException_ = false;
         onChanged();
         return this;
@@ -4772,23 +5010,33 @@ public final class TrxRegionProtos {
      */
     long getTransactionId();
 
-    // required int32 participantNum = 3;
+    // required int64 commitId = 3;
     /**
-     * <code>required int32 participantNum = 3;</code>
+     * <code>required int64 commitId = 3;</code>
+     */
+    boolean hasCommitId();
+    /**
+     * <code>required int64 commitId = 3;</code>
+     */
+    long getCommitId();
+
+    // required int32 participantNum = 4;
+    /**
+     * <code>required int32 participantNum = 4;</code>
      */
     boolean hasParticipantNum();
     /**
-     * <code>required int32 participantNum = 3;</code>
+     * <code>required int32 participantNum = 4;</code>
      */
     int getParticipantNum();
 
-    // optional bool ignoreUnknownTransactionException = 4;
+    // optional bool ignoreUnknownTransactionException = 5;
     /**
-     * <code>optional bool ignoreUnknownTransactionException = 4;</code>
+     * <code>optional bool ignoreUnknownTransactionException = 5;</code>
      */
     boolean hasIgnoreUnknownTransactionException();
     /**
-     * <code>optional bool ignoreUnknownTransactionException = 4;</code>
+     * <code>optional bool ignoreUnknownTransactionException = 5;</code>
      */
     boolean getIgnoreUnknownTransactionException();
   }
@@ -4858,11 +5106,16 @@ public final class TrxRegionProtos {
             }
             case 24: {
               bitField0_ |= 0x00000002;
-              participantNum_ = input.readInt32();
+              commitId_ = input.readInt64();
               break;
             }
             case 32: {
               bitField0_ |= 0x00000004;
+              participantNum_ = input.readInt32();
+              break;
+            }
+            case 40: {
+              bitField0_ |= 0x00000008;
               ignoreUnknownTransactionException_ = input.readBool();
               break;
             }
@@ -4948,33 +5201,49 @@ public final class TrxRegionProtos {
       return transactionId_;
     }
 
-    // required int32 participantNum = 3;
-    public static final int PARTICIPANTNUM_FIELD_NUMBER = 3;
+    // required int64 commitId = 3;
+    public static final int COMMITID_FIELD_NUMBER = 3;
+    private long commitId_;
+    /**
+     * <code>required int64 commitId = 3;</code>
+     */
+    public boolean hasCommitId() {
+      return ((bitField0_ & 0x00000002) == 0x00000002);
+    }
+    /**
+     * <code>required int64 commitId = 3;</code>
+     */
+    public long getCommitId() {
+      return commitId_;
+    }
+
+    // required int32 participantNum = 4;
+    public static final int PARTICIPANTNUM_FIELD_NUMBER = 4;
     private int participantNum_;
     /**
-     * <code>required int32 participantNum = 3;</code>
+     * <code>required int32 participantNum = 4;</code>
      */
     public boolean hasParticipantNum() {
-      return ((bitField0_ & 0x00000002) == 0x00000002);
+      return ((bitField0_ & 0x00000004) == 0x00000004);
     }
     /**
-     * <code>required int32 participantNum = 3;</code>
+     * <code>required int32 participantNum = 4;</code>
      */
     public int getParticipantNum() {
       return participantNum_;
     }
 
-    // optional bool ignoreUnknownTransactionException = 4;
-    public static final int IGNOREUNKNOWNTRANSACTIONEXCEPTION_FIELD_NUMBER = 4;
+    // optional bool ignoreUnknownTransactionException = 5;
+    public static final int IGNOREUNKNOWNTRANSACTIONEXCEPTION_FIELD_NUMBER = 5;
     private boolean ignoreUnknownTransactionException_;
     /**
-     * <code>optional bool ignoreUnknownTransactionException = 4;</code>
+     * <code>optional bool ignoreUnknownTransactionException = 5;</code>
      */
     public boolean hasIgnoreUnknownTransactionException() {
-      return ((bitField0_ & 0x00000004) == 0x00000004);
+      return ((bitField0_ & 0x00000008) == 0x00000008);
     }
     /**
-     * <code>optional bool ignoreUnknownTransactionException = 4;</code>
+     * <code>optional bool ignoreUnknownTransactionException = 5;</code>
      */
     public boolean getIgnoreUnknownTransactionException() {
       return ignoreUnknownTransactionException_;
@@ -4983,6 +5252,7 @@ public final class TrxRegionProtos {
     private void initFields() {
       regionName_ = java.util.Collections.emptyList();
       transactionId_ = 0L;
+      commitId_ = 0L;
       participantNum_ = 0;
       ignoreUnknownTransactionException_ = false;
     }
@@ -4995,6 +5265,10 @@ public final class TrxRegionProtos {
         memoizedIsInitialized = 0;
         return false;
       }
+      if (!hasCommitId()) {
+        memoizedIsInitialized = 0;
+        return false;
+      }
       if (!hasParticipantNum()) {
         memoizedIsInitialized = 0;
         return false;
@@ -5013,10 +5287,13 @@ public final class TrxRegionProtos {
         output.writeInt64(2, transactionId_);
       }
       if (((bitField0_ & 0x00000002) == 0x00000002)) {
-        output.writeInt32(3, participantNum_);
+        output.writeInt64(3, commitId_);
       }
       if (((bitField0_ & 0x00000004) == 0x00000004)) {
-        output.writeBool(4, ignoreUnknownTransactionException_);
+        output.writeInt32(4, participantNum_);
+      }
+      if (((bitField0_ & 0x00000008) == 0x00000008)) {
+        output.writeBool(5, ignoreUnknownTransactionException_);
       }
       getUnknownFields().writeTo(output);
     }
@@ -5042,11 +5319,15 @@ public final class TrxRegionProtos {
       }
       if (((bitField0_ & 0x00000002) == 0x00000002)) {
         size += com.google.protobuf.CodedOutputStream
-          .computeInt32Size(3, participantNum_);
+          .computeInt64Size(3, commitId_);
       }
       if (((bitField0_ & 0x00000004) == 0x00000004)) {
         size += com.google.protobuf.CodedOutputStream
-          .computeBoolSize(4, ignoreUnknownTransactionException_);
+          .computeInt32Size(4, participantNum_);
+      }
+      if (((bitField0_ & 0x00000008) == 0x00000008)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeBoolSize(5, ignoreUnknownTransactionException_);
       }
       size += getUnknownFields().getSerializedSize();
       memoizedSerializedSize = size;
@@ -5168,10 +5449,12 @@ public final class TrxRegionProtos {
         bitField0_ = (bitField0_ & ~0x00000001);
         transactionId_ = 0L;
         bitField0_ = (bitField0_ & ~0x00000002);
-        participantNum_ = 0;
+        commitId_ = 0L;
         bitField0_ = (bitField0_ & ~0x00000004);
-        ignoreUnknownTransactionException_ = false;
+        participantNum_ = 0;
         bitField0_ = (bitField0_ & ~0x00000008);
+        ignoreUnknownTransactionException_ = false;
+        bitField0_ = (bitField0_ & ~0x00000010);
         return this;
       }
 
@@ -5212,10 +5495,14 @@ public final class TrxRegionProtos {
         if (((from_bitField0_ & 0x00000004) == 0x00000004)) {
           to_bitField0_ |= 0x00000002;
         }
-        result.participantNum_ = participantNum_;
+        result.commitId_ = commitId_;
         if (((from_bitField0_ & 0x00000008) == 0x00000008)) {
           to_bitField0_ |= 0x00000004;
         }
+        result.participantNum_ = participantNum_;
+        if (((from_bitField0_ & 0x00000010) == 0x00000010)) {
+          to_bitField0_ |= 0x00000008;
+        }
         result.ignoreUnknownTransactionException_ = ignoreUnknownTransactionException_;
         result.bitField0_ = to_bitField0_;
         onBuilt();
@@ -5246,6 +5533,9 @@ public final class TrxRegionProtos {
         if (other.hasTransactionId()) {
           setTransactionId(other.getTransactionId());
         }
+        if (other.hasCommitId()) {
+          setCommitId(other.getCommitId());
+        }
         if (other.hasParticipantNum()) {
           setParticipantNum(other.getParticipantNum());
         }
@@ -5261,6 +5551,10 @@ public final class TrxRegionProtos {
           
           return false;
         }
+        if (!hasCommitId()) {
+          
+          return false;
+        }
         if (!hasParticipantNum()) {
           
           return false;
@@ -5392,67 +5686,100 @@ public final class TrxRegionProtos {
         return this;
       }
 
-      // required int32 participantNum = 3;
+      // required int64 commitId = 3;
+      private long commitId_ ;
+      /**
+       * <code>required int64 commitId = 3;</code>
+       */
+      public boolean hasCommitId() {
+        return ((bitField0_ & 0x00000004) == 0x00000004);
+      }
+      /**
+       * <code>required int64 commitId = 3;</code>
+       */
+      public long getCommitId() {
+        return commitId_;
+      }
+      /**
+       * <code>required int64 commitId = 3;</code>
+       */
+      public Builder setCommitId(long value) {
+        bitField0_ |= 0x00000004;
+        commitId_ = value;
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>required int64 commitId = 3;</code>
+       */
+      public Builder clearCommitId() {
+        bitField0_ = (bitField0_ & ~0x00000004);
+        commitId_ = 0L;
+        onChanged();
+        return this;
+      }
+
+      // required int32 participantNum = 4;
       private int participantNum_ ;
       /**
-       * <code>required int32 participantNum = 3;</code>
+       * <code>required int32 participantNum = 4;</code>
        */
       public boolean hasParticipantNum() {
-        return ((bitField0_ & 0x00000004) == 0x00000004);
+        return ((bitField0_ & 0x00000008) == 0x00000008);
       }
       /**
-       * <code>required int32 participantNum = 3;</code>
+       * <code>required int32 participantNum = 4;</code>
        */
       public int getParticipantNum() {
         return participantNum_;
       }
       /**
-       * <code>required int32 participantNum = 3;</code>
+       * <code>required int32 participantNum = 4;</code>
        */
       public Builder setParticipantNum(int value) {
-        bitField0_ |= 0x00000004;
+        bitField0_ |= 0x00000008;
         participantNum_ = value;
         onChanged();
         return this;
       }
       /**
-       * <code>required int32 participantNum = 3;</code>
+       * <code>required int32 participantNum = 4;</code>
        */
       public Builder clearParticipantNum() {
-        bitField0_ = (bitField0_ & ~0x00000004);
+        bitField0_ = (bitField0_ & ~0x00000008);
         participantNum_ = 0;
         onChanged();
         return this;
       }
 
-      // optional bool ignoreUnknownTransactionException = 4;
+      // optional bool ignoreUnknownTransactionException = 5;
       private boolean ignoreUnknownTransactionException_ ;
       /**
-       * <code>optional bool ignoreUnknownTransactionException = 4;</code>
+       * <code>optional bool ignoreUnknownTransactionException = 5;</code>
        */
       public boolean hasIgnoreUnknownTransactionException() {
-        return ((bitField0_ & 0x00000008) == 0x00000008);
+        return ((bitField0_ & 0x00000010) == 0x00000010);
       }
       /**
-       * <code>optional bool ignoreUnknownTransactionException = 4;</code>
+       * <code>optional bool ignoreUnknownTransactionException = 5;</code>
        */
       public boolean getIgnoreUnknownTransactionException() {
         return ignoreUnknownTransactionException_;
       }
       /**
-       * <code>optional bool ignoreUnknownTransactionException = 4;</code>
+       * <code>optional bool ignoreUnknownTransactionException = 5;</code>
        */
       public Builder setIgnoreUnknownTransactionException(boolean value) {
-        bitField0_ |= 0x00000008;
+        bitField0_ |= 0x00000010;
         ignoreUnknownTransactionException_ = value;
         onChanged();
         return this;
       }
       /**
-       * <code>optional bool ignoreUnknownTransactionException = 4;</code>
+       * <code>optional bool ignoreUnknownTransactionException = 5;</code>
        */
       public Builder clearIgnoreUnknownTransactionException() {
-        bitField0_ = (bitField0_ & ~0x00000008);
+        bitField0_ = (bitField0_ & ~0x00000010);
         ignoreUnknownTransactionException_ = false;
         onChanged();
         return this;
@@ -10252,67 +10579,77 @@ public final class TrxRegionProtos {
      */
     long getTransactionId();
 
-    // required bytes regionName = 2;
+    // required int64 startId = 2;
     /**
-     * <code>required bytes regionName = 2;</code>
+     * <code>required int64 startId = 2;</code>
+     */
+    boolean hasStartId();
+    /**
+     * <code>required int64 startId = 2;</code>
+     */
+    long getStartId();
+
+    // required bytes regionName = 3;
+    /**
+     * <code>required bytes regionName = 3;</code>
      */
     boolean hasRegionName();
     /**
-     * <code>required bytes regionName = 2;</code>
+     * <code>required bytes regionName = 3;</code>
      */
     com.google.protobuf.ByteString getRegionName();
 
-    // required bytes row = 3;
+    // required bytes row = 4;
     /**
-     * <code>required bytes row = 3;</code>
+     * <code>required bytes row = 4;</code>
      */
     boolean hasRow();
     /**
-     * <code>required bytes row = 3;</code>
+     * <code>required bytes row = 4;</code>
      */
     com.google.protobuf.ByteString getRow();
 
-    // required bytes family = 4;
+    // required bytes family = 5;
     /**
-     * <code>required bytes family = 4;</code>
+     * <code>required bytes family = 5;</code>
      */
     boolean hasFamily();
     /**
-     * <code>required bytes family = 4;</code>
+     * <code>required bytes family = 5;</code>
      */
     com.google.protobuf.ByteString getFamily();
 
-    // required bytes qualifier = 5;
+    // required bytes qualifier = 6;
     /**
-     * <code>required bytes qualifier = 5;</code>
+     * <code>required bytes qualifier = 6;</code>
      */
     boolean hasQualifier();
     /**
-     * <code>required bytes qualifier = 5;</code>
+     * <code>required bytes qualifier = 6;</code>
      */
     com.google.protobuf.ByteString getQualifier();
 
-    // required bytes value = 6;
+    // required bytes value = 7;
     /**
-     * <code>required bytes value = 6;</code>
+     * <code>required bytes value = 7;</code>
      */
     boolean hasValue();
     /**
-     * <code>required bytes value = 6;</code>
+     * <code>required bytes value = 7;</code>
      */
     com.google.protobuf.ByteString getValue();
 
-    // required .MutationProto delete = 7;
+    // required .MutationProto delete = 8;
     /**
-     * <code>required .MutationProto delete = 7;</code>
+     * <code>required .MutationProto delete = 8;</code>
      */
     boolean hasDelete();
     /**
-     * <code>required .MutationProto delete = 7;</code>
+     * <code>required .MutationProto delete = 8;</code>
      */
     org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto getDelete();
     /**
-     * <code>required .MutationProto delete = 7;</code>
+     * <code>required .MutationProto delete = 8;</code>
      */
     org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProtoOrBuilder getDeleteOrBuilder();
   }
@@ -10372,34 +10709,39 @@ public final class TrxRegionProtos {
               transactionId_ = input.readInt64();
               break;
             }
-            case 18: {
+            case 16: {
               bitField0_ |= 0x00000002;
-              regionName_ = input.readBytes();
+              startId_ = input.readInt64();
               break;
             }
             case 26: {
               bitField0_ |= 0x00000004;
-              row_ = input.readBytes();
+              regionName_ = input.readBytes();
               break;
             }
             case 34: {
               bitField0_ |= 0x00000008;
-              family_ = input.readBytes();
+              row_ = input.readBytes();
               break;
             }
             case 42: {
               bitField0_ |= 0x00000010;
-              qualifier_ = input.readBytes();
+              family_ = input.readBytes();
               break;
             }
             case 50: {
               bitField0_ |= 0x00000020;
-              value_ = input.readBytes();
+              qualifier_ = input.readBytes();
               break;
             }
             case 58: {
+              bitField0_ |= 0x00000040;
+              value_ = input.readBytes();
+              break;
+            }
+            case 66: {
               org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.Builder subBuilder = null;
-              if (((bitField0_ & 0x00000040) == 0x00000040)) {
+              if (((bitField0_ & 0x00000080) == 0x00000080)) {
                 subBuilder = delete_.toBuilder();
               }
               delete_ = input.readMessage(org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.PARSER, extensionRegistry);
@@ -10407,7 +10749,7 @@ public final class TrxRegionProtos {
                 subBuilder.mergeFrom(delete_);
                 delete_ = subBuilder.buildPartial();
               }
-              bitField0_ |= 0x00000040;
+              bitField0_ |= 0x00000080;
               break;
             }
           }
@@ -10466,103 +10808,119 @@ public final class TrxRegionProtos {
       return transactionId_;
     }
 
-    // required bytes regionName = 2;
-    public static final int REGIONNAME_FIELD_NUMBER = 2;
+    // required int64 startId = 2;
+    public static final int STARTID_FIELD_NUMBER = 2;
+    private long startId_;
+    /**
+     * <code>required int64 startId = 2;</code>
+     */
+    public boolean hasStartId() {
+      return ((bitField0_ & 0x00000002) == 0x00000002);
+    }
+    /**
+     * <code>required int64 startId = 2;</code>
+     */
+    public long getStartId() {
+      return startId_;
+    }
+
+    // required bytes regionName = 3;
+    public static final int REGIONNAME_FIELD_NUMBER = 3;
     private com.google.protobuf.ByteString regionName_;
     /**
-     * <code>required bytes regionName = 2;</code>
+     * <code>required bytes regionName = 3;</code>
      */
     public boolean hasRegionName() {
-      return ((bitField0_ & 0x00000002) == 0x00000002);
+      return ((bitField0_ & 0x00000004) == 0x00000004);
     }
     /**
-     * <code>required bytes regionName = 2;</code>
+     * <code>required bytes regionName = 3;</code>
      */
     public com.google.protobuf.ByteString getRegionName() {
       return regionName_;
     }
 
-    // required bytes row = 3;
-    public static final int ROW_FIELD_NUMBER = 3;
+    // required bytes row = 4;
+    public static final int ROW_FIELD_NUMBER = 4;
     private com.google.protobuf.ByteString row_;
     /**
-     * <code>required bytes row = 3;</code>
+     * <code>required bytes row = 4;</code>
      */
     public boolean hasRow() {
-      return ((bitField0_ & 0x00000004) == 0x00000004);
+      return ((bitField0_ & 0x00000008) == 0x00000008);
     }
     /**
-     * <code>required bytes row = 3;</code>
+     * <code>required bytes row = 4;</code>
      */
     public com.google.protobuf.ByteString getRow() {
       return row_;
     }
 
-    // required bytes family = 4;
-    public static final int FAMILY_FIELD_NUMBER = 4;
+    // required bytes family = 5;
+    public static final int FAMILY_FIELD_NUMBER = 5;
     private com.google.protobuf.ByteString family_;
     /**
-     * <code>required bytes family = 4;</code>
+     * <code>required bytes family = 5;</code>
      */
     public boolean hasFamily() {
-      return ((bitField0_ & 0x00000008) == 0x00000008);
+      return ((bitField0_ & 0x00000010) == 0x00000010);
     }
     /**
-     * <code>required bytes family = 4;</code>
+     * <code>required bytes family = 5;</code>
      */
     public com.google.protobuf.ByteString getFamily() {
       return family_;
     }
 
-    // required bytes qualifier = 5;
-    public static final int QUALIFIER_FIELD_NUMBER = 5;
+    // required bytes qualifier = 6;
+    public static final int QUALIFIER_FIELD_NUMBER = 6;
     private com.google.protobuf.ByteString qualifier_;
     /**
-     * <code>required bytes qualifier = 5;</code>
+     * <code>required bytes qualifier = 6;</code>
      */
     public boolean hasQualifier() {
-      return ((bitField0_ & 0x00000010) == 0x00000010);
+      return ((bitField0_ & 0x00000020) == 0x00000020);
     }
     /**
-     * <code>required bytes qualifier = 5;</code>
+     * <code>required bytes qualifier = 6;</code>
      */
     public com.google.protobuf.ByteString getQualifier() {
       return qualifier_;
     }
 
-    // required bytes value = 6;
-    public static final int VALUE_FIELD_NUMBER = 6;
+    // required bytes value = 7;
+    public static final int VALUE_FIELD_NUMBER = 7;
     private com.google.protobuf.ByteString value_;
     /**
-     * <code>required bytes value = 6;</code>
+     * <code>required bytes value = 7;</code>
      */
     public boolean hasValue() {
-      return ((bitField0_ & 0x00000020) == 0x00000020);
+      return ((bitField0_ & 0x00000040) == 0x00000040);
     }
     /**
-     * <code>required bytes value = 6;</code>
+     * <code>required bytes value = 7;</code>
      */
     public com.google.protobuf.ByteString getValue() {
       return value_;
     }
 
-    // required .MutationProto delete = 7;
-    public static final int DELETE_FIELD_NUMBER = 7;
+    // required .MutationProto delete = 8;
+    public static final int DELETE_FIELD_NUMBER = 8;
     private org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto delete_;
     /**
-     * <code>required .MutationProto delete = 7;</code>
+     * <code>required .MutationProto delete = 8;</code>
      */
     public boolean hasDelete() {
-      return ((bitField0_ & 0x00000040) == 0x00000040);
+      return ((bitField0_ & 0x00000080) == 0x00000080);
     }
     /**
-     * <code>required .MutationProto delete = 7;</code>
+     * <code>required .MutationProto delete = 8;</code>
      */
     public org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto getDelete() {
       return delete_;
     }
     /**
-     * <code>required .MutationProto delete = 7;</code>
+     * <code>required .MutationProto delete = 8;</code>
      */
     public org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProtoOrBuilder getDeleteOrBuilder() {
       return delete_;
@@ -10570,6 +10928,7 @@ public final class TrxRegionProtos {
 
     private void initFields() {
       transactionId_ = 0L;
+      startId_ = 0L;
       regionName_ = com.google.protobuf.ByteString.EMPTY;
       row_ = com.google.protobuf.ByteString.EMPTY;
       family_ = com.google.protobuf.ByteString.EMPTY;
@@ -10586,6 +10945,10 @@ public final class TrxRegionProtos {
         memoizedIsInitialized = 0;
         return false;
       }
+      if (!hasStartId()) {
+        memoizedIsInitialized = 0;
+        return false;
+      }
       if (!hasRegionName()) {
         memoizedIsInitialized = 0;
         return false;
@@ -10625,22 +10988,25 @@ public final class TrxRegionProtos {
         output.writeInt64(1, transactionId_);
       }
       if (((bitField0_ & 0x00000002) == 0x00000002)) {
-        output.writeBytes(2, regionName_);
+        output.writeInt64(2, startId_);
       }
       if (((bitField0_ & 0x00000004) == 0x00000004)) {
-        output.writeBytes(3, row_);
+        output.writeBytes(3, regionName_);
       }
       if (((bitField0_ & 0x00000008) == 0x00000008)) {
-        output.writeBytes(4, family_);
+        output.writeBytes(4, row_);
       }
       if (((bitField0_ & 0x00000010) == 0x00000010)) {
-        output.writeBytes(5, qualifier_);
+        output.writeBytes(5, family_);
       }
       if (((bitField0_ & 0x00000020) == 0x00000020)) {
-        output.writeBytes(6, value_);
+        output.writeBytes(6, qualifier_);
       }
       if (((bitField0_ & 0x00000040) == 0x00000040)) {
-        output.writeMessage(7, delete_);
+        output.writeBytes(7, value_);
+      }
+      if (((bitField0_ & 0x00000080) == 0x00000080)) {
+        output.writeMessage(8, delete_);
       }
       getUnknownFields().writeTo(output);
     }
@@ -10657,27 +11023,31 @@ public final class TrxRegionProtos {
       }
       if (((bitField0_ & 0x00000002) == 0x00000002)) {
         size += com.google.protobuf.CodedOutputStream
-          .computeBytesSize(2, regionName_);
+          .computeInt64Size(2, startId_);
       }
       if (((bitField0_ & 0x00000004) == 0x00000004)) {
         size += com.google.protobuf.CodedOutputStream
-          .computeBytesSize(3, row_);
+          .computeBytesSize(3, regionName_);
       }
       if (((bitField0_ & 0x00000008) == 0x00000008)) {
         size += com.google.protobuf.CodedOutputStream
-          .computeBytesSize(4, family_);
+          .computeBytesSize(4, row_);
       }
       if (((bitField0_ & 0x00000010) == 0x00000010)) {
         size += com.google.protobuf.CodedOutputStream
-          .computeBytesSize(5, qualifier_);
+          .computeBytesSize(5, family_);
       }
       if (((bitField0_ & 0x00000020) == 0x00000020)) {
         size += com.google.protobuf.CodedOutputStream
-          .computeBytesSize(6, value_);
+          .computeBytesSize(6, qualifier_);
       }
       if (((bitField0_ & 0x00000040) == 0x00000040)) {
         size += com.google.protobuf.CodedOutputStream
-          .computeMessageSize(7, delete_);
+          .computeBytesSize(7, value_);
+      }
+      if (((bitField0_ & 0x00000080) == 0x00000080)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeMessageSize(8, delete_);
       }
       size += getUnknownFields().getSerializedSize();
       memoizedSerializedSize = size;
@@ -10798,22 +11168,24 @@ public final class TrxRegionProtos {
         super.clear();
         transactionId_ = 0L;
         bitField0_ = (bitField0_ & ~0x00000001);
-        regionName_ = com.google.protobuf.ByteString.EMPTY;
+        startId_ = 0L;
         bitField0_ = (bitField0_ & ~0x00000002);
-        row_ = com.google.protobuf.ByteString.EMPTY;
+        regionName_ = com.google.protobuf.ByteString.EMPTY;
         bitField0_ = (bitField0_ & ~0x00000004);
-        family_ = com.google.protobuf.ByteString.EMPTY;
+        row_ = com.google.protobuf.ByteString.EMPTY;
         bitField0_ = (bitField0_ & ~0x00000008);
-        qualifier_ = com.google.protobuf.ByteString.EMPTY;
+        family_ = com.google.protobuf.ByteString.EMPTY;
         bitField0_ = (bitField0_ & ~0x00000010);
-        value_ = com.google.protobuf.ByteString.EMPTY;
+        qualifier_ = com.google.protobuf.ByteString.EMPTY;
         bitField0_ = (bitField0_ & ~0x00000020);
+        value_ = com.google.protobuf.ByteString.EMPTY;
+        bitField0_ = (bitField0_ & ~0x00000040);
         if (deleteBuilder_ == null) {
           delete_ = org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.getDefaultInstance();
         } else {
           deleteBuilder_.clear();
         }
-        bitField0_ = (bitField0_ & ~0x00000040);
+        bitField0_ = (bitField0_ & ~0x00000080);
         return this;
       }
 
@@ -10849,26 +11221,30 @@ public final class TrxRegionProtos {
         if (((from_bitField0_ & 0x00000002) == 0x00000002)) {
           to_bitField0_ |= 0x00000002;
         }
-        result.regionName_ = regionName_;
+        result.startId_ = startId_;
         if (((from_bitField0_ & 0x00000004) == 0x00000004)) {
           to_bitField0_ |= 0x00000004;
         }
-        result.row_ = row_;
+        result.regionName_ = regionName_;
         if (((from_bitField0_ & 0x00000008) == 0x00000008)) {
           to_bitField0_ |= 0x00000008;
         }
-        result.family_ = family_;
+        result.row_ = row_;
         if (((from_bitField0_ & 0x00000010) == 0x00000010)) {
           to_bitField0_ |= 0x00000010;
         }
-        result.qualifier_ = qualifier_;
+        result.family_ = family_;
         if (((from_bitField0_ & 0x00000020) == 0x00000020)) {
           to_bitField0_ |= 0x00000020;
         }
-        result.value_ = value_;
+        result.qualifier_ = qualifier_;
         if (((from_bitField0_ & 0x00000040) == 0x00000040)) {
           to_bitField0_ |= 0x00000040;
         }
+        result.value_ = value_;
+        if (((from_bitField0_ & 0x00000080) == 0x00000080)) {
+          to_bitField0_ |= 0x00000080;
+        }
         if (deleteBuilder_ == null) {
           result.delete_ = delete_;
         } else {
@@ -10893,6 +11269,9 @@ public final class TrxRegionProtos {
         if (other.hasTransactionId()) {
           setTransactionId(other.getTransactionId());
         }
+        if (other.hasStartId()) {
+          setStartId(other.getStartId());
+        }
         if (other.hasRegionName()) {
           setRegionName(other.getRegionName());
         }
@@ -10920,6 +11299,10 @@ public final class TrxRegionProtos {
           
           return false;
         }
+        if (!hasStartId()) {
+          
+          return false;
+        }
         if (!hasRegionName()) {
           
           return false;
@@ -11003,198 +11386,231 @@ public final class TrxRegionProtos {
         return this;
       }
 
-      // required bytes regionName = 2;
+      // required int64 startId = 2;
+      private long startId_ ;
+      /**
+       * <code>required int64 startId = 2;</code>
+       */
+      public boolean hasStartId() {
+        return ((bitField0_ & 0x00000002) == 0x00000002);
+      }
+      /**
+       * <code>required int64 startId = 2;</code>
+       */
+      public long getStartId() {
+        return startId_;
+      }
+      /**
+       * <code>required int64 startId = 2;</code>
+       */
+      public Builder setStartId(long value) {
+        bitField0_ |= 0x00000002;
+        startId_ = value;
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>required int64 startId = 2;</code>
+       */
+      public Builder clearStartId() {
+        bitField0_ = (bitField0_ & ~0x00000002);
+        startId_ = 0L;
+        onChanged();
+        return this;
+      }
+
+      // required bytes regionName = 3;
       private com.google.protobuf.ByteString regionName_ = com.google.protobuf.ByteString.EMPTY;
       /**
-       * <code>required bytes regionName = 2;</code>
+       * <code>required bytes regionName = 3;</code>
        */
       public boolean hasRegionName() {
-        return ((bitField0_ & 0x00000002) == 0x00000002);
+        return ((bitField0_ & 0x00000004) == 0x00000004);
       }
       /**
-       * <code>required bytes regionName = 2;</code>
+       * <code>required bytes regionName = 3;</code>
        */
       public com.google.protobuf.ByteString getRegionName() {
         return regionName_;
       }
       /**
-       * <code>required bytes regionName = 2;</code>
+       * <code>required bytes regionName = 3;</code>
        */
       public Builder setRegionName(com.google.protobuf.ByteString value) {
         if (value == null) {
     throw new NullPointerException();
   }
-  bitField0_ |= 0x00000002;
+  bitField0_ |= 0x00000004;
         regionName_ = value;
         onChanged();
         return this;
       }
       /**
-       * <code>required bytes regionName = 2;</code>
+       * <code>required bytes regionName = 3;</code>
        */
       public Builder clearRegionName() {
-        bitField0_ = (bitField0_ & ~0x00000002);
+        bitField0_ = (bitField0_ & ~0x00000004);
         regionName_ = getDefaultInstance().getRegionName();
         onChanged();
         return this;
       }
 
-      // required bytes row = 3;
+      // required bytes row = 4;
       private com.google.protobuf.ByteString row_ = com.google.protobuf.ByteString.EMPTY;
       /**
-       * <code>required bytes row = 3;</code>
+       * <code>required bytes row = 4;</code>
        */
       public boolean hasRow() {
-        return ((bitField0_ & 0x00000004) == 0x00000004);
+        return ((bitField0_ & 0x00000008) == 0x00000008);
       }
       /**
-       * <code>required bytes row = 3;</code>
+       * <code>required bytes row = 4;</code>
        */
       public com.google.protobuf.ByteString getRow() {
         return row_;
       }
       /**
-       * <code>required bytes row = 3;</code>
+       * <code>required bytes row = 4;</code>
        */
       public Builder setRow(com.google.protobuf.ByteString value) {
         if (value == null) {
     throw new NullPointerException();
   }
-  bitField0_ |= 0x00000004;
+  bitField0_ |= 0x00000008;
         row_ = value;
         onChanged();
         return this;
       }
       /**
-       * <code>required bytes row = 3;</code>
+       * <code>required bytes row = 4;</code>
        */
       public Builder clearRow() {
-        bitField0_ = (bitField0_ & ~0x00000004);
+        bitField0_ = (bitField0_ & ~0x00000008);
         row_ = getDefaultInstance().getRow();
         onChanged();
         return this;
       }
 
-      // required bytes family = 4;
+      // required bytes family = 5;
       private com.google.protobuf.ByteString family_ = com.google.protobuf.ByteString.EMPTY;
       /**
-       * <code>required bytes family = 4;</code>
+       * <code>required bytes family = 5;</code>
        */
       public boolean hasFamily() {
-        return ((bitField0_ & 0x00000008) == 0x00000008);
+        return ((bitField0_ & 0x00000010) == 0x00000010);
       }
       /**
-       * <code>required bytes family = 4;</code>
+       * <code>required bytes family = 5;</code>
        */
       public com.google.protobuf.ByteString getFamily() {
         return family_;
       }
       /**
-       * <code>required bytes family = 4;</code>
+       * <code>required bytes family = 5;</code>
        */
       public Builder setFamily(com.google.protobuf.ByteString value) {
         if (value == null) {
     throw new NullPointerException();
   }
-  bitField0_ |= 0x00000008;
+  bitField0_ |= 0x00000010;
         family_ = value;
         onChanged();
         return this;
       }
       /**
-       * <code>required bytes family = 4;</code>
+       * <code>required bytes family = 5;</code>
        */
       public Builder clearFamily() {
-        bitField0_ = (bitField0_ & ~0x00000008);
+        bitField0_ = (bitField0_ & ~0x00000010);
         family_ = getDefaultInstance().getFamily();
         onChanged();
         return this;
       }
 
-      // required bytes qualifier = 5;
+      // required bytes qualifier = 6;
       private com.google.protobuf.ByteString qualifier_ = com.google.protobuf.ByteString.EMPTY;
       /**
-       * <code>required bytes qualifier = 5;</code>
+       * <code>required bytes qualifier = 6;</code>
        */
       public boolean hasQualifier() {
-        return ((bitField0_ & 0x00000010) == 0x00000010);
+        return ((bitField0_ & 0x00000020) == 0x00000020);
       }
       /**
-       * <code>required bytes qualifier = 5;</code>
+       * <code>required bytes qualifier = 6;</code>
        */
       public com.google.protobuf.ByteString getQualifier() {
         return qualifier_;
       }
       /**
-       * <code>required bytes qualifier = 5;</code>
+       * <code>required bytes qualifier = 6;</code>
        */
       public Builder setQualifier(com.google.protobuf.ByteString value) {
         if (value == null) {
     throw new NullPointerException();
   }
-  bitField0_ |= 0x00000010;
+  bitField0_ |= 0x00000020;
         qualifier_ = value;
         onChanged();
         return this;
       }
       /**
-       * <code>required bytes qualifier = 5;</code>
+       * <code>required bytes qualifier = 6;</code>
        */
       public Builder clearQualifier() {
-        bitField0_ = (bitField0_ & ~0x00000010);
+        bitField0_ = (bitField0_ & ~0x00000020);
         qualifier_ = getDefaultInstance().getQualifier();
         onChanged();
         return this;
       }
 
-      // required bytes value = 6;
+      // required bytes value = 7;
       private com.google.protobuf.ByteString value_ = com.google.protobuf.ByteString.EMPTY;
       /**
-       * <code>required bytes value = 6;</code>
+       * <code>required bytes value = 7;</code>
        */
       public boolean hasValue() {
-        return ((bitField0_ & 0x00000020) == 0x00000020);
+        return ((bitField0_ & 0x00000040) == 0x00000040);
       }
       /**
-       * <code>required bytes value = 6;</code>
+       * <code>required bytes value = 7;</code>
        */
       public com.google.protobuf.ByteString getValue() {
         return value_;
       }
       /**
-       * <code>required bytes value = 6;</code>
+       * <code>required bytes value = 7;</code>
        */
       public Builder setValue(com.google.protobuf.ByteString value) {
         if (value == null) {
     throw new NullPointerException();
   }
-  bitField0_ |= 0x00000020;
+  bitField0_ |= 0x00000040;
         value_ = value;
         onChanged();
         return this;
       }
       /**
-       * <code>required bytes value = 6;</code>
+       * <code>required bytes value = 7;</code>
        */
       public Builder clearValue() {
-        bitField0_ = (bitField0_ & ~0x00000020);
+        bitField0_ = (bitField0_ & ~0x00000040);
         value_ = getDefaultInstance().getValue();
         onChanged();
         return this;
       }
 
-      // required .MutationProto delete = 7;
+      // required .MutationProto delete = 8;
       private org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto delete_ = org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.getDefaultInstance();
       private com.google.protobuf.SingleFieldBuilder<
           org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto, org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.Builder, org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProtoOrBuilder> deleteBuilder_;
       /**
-       * <code>required .MutationProto delete = 7;</code>
+       * <code>required .MutationProto delete = 8;</code>
        */
       public boolean hasDelete() {
-        return ((bitField0_ & 0x00000040) == 0x00000040);
+        return ((bitField0_ & 0x00000080) == 0x00000080);
       }
       /**
-       * <code>required .MutationProto delete = 7;</code>
+       * <code>required .MutationProto delete = 8;</code>
        */
       public org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto getDelete() {
         if (deleteBuilder_ == null) {
@@ -11204,7 +11620,7 @@ public final class TrxRegionProtos {
         }
       }
       /**
-       * <code>required .MutationProto delete = 7;</code>
+       * <code>required .MutationProto delete = 8;</code>
        */
       public Builder setDelete(org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto value) {
         if (deleteBuilder_ == null) {
@@ -11216,11 +11632,11 @@ public final class TrxRegionProtos {
         } else {
           deleteBuilder_.setMessage(value);
         }
-        bitField0_ |= 0x00000040;
+        bitField0_ |= 0x00000080;
         return this;
       }
       /**
-       * <code>required .MutationProto delete = 7;</code>
+       * <code>required .MutationProto delete = 8;</code>
        */
       public Builder setDelete(
           org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.Builder builderForValue) {
@@ -11230,15 +11646,15 @@ public final class TrxRegionProtos {
         } else {
           deleteBuilder_.setMessage(builderForValue.build());
         }
-        bitField0_ |= 0x00000040;
+        bitField0_ |= 0x00000080;
         return this;
       }
       /**
-       * <code>required .MutationProto delete = 7;</code>
+       * <code>required .MutationProto delete = 8;</code>
        */
       public Builder mergeDelete(org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto value) {
         if (deleteBuilder_ == null) {
-          if (((bitField0_ & 0x00000040) == 0x00000040) &&
+          if (((bitField0_ & 0x00000080) == 0x00000080) &&
               delete_ != org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.getDefaultInstance()) {
             delete_ =
               org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.newBuilder(delete_).mergeFrom(value).buildPartial();
@@ -11249,11 +11665,11 @@ public final class TrxRegionProtos {
         } else {
           deleteBuilder_.mergeFrom(value);
         }
-        bitField0_ |= 0x00000040;
+        bitField0_ |= 0x00000080;
         return this;
       }
       /**
-       * <code>required .MutationProto delete = 7;</code>
+       * <code>required .MutationProto delete = 8;</code>
        */
       public Builder clearDelete() {
         if (deleteBuilder_ == null) {
@@ -11262,19 +11678,19 @@ public final class TrxRegionProtos {
         } else {
           deleteBuilder_.clear();
         }
-        bitField0_ = (bitField0_ & ~0x00000040);
+        bitField0_ = (bitField0_ & ~0x00000080);
         return this;
       }
       /**
-       * <code>required .MutationProto delete = 7;</code>
+       * <code>required .MutationProto delete = 8;</code>
        */
       public org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.Builder getDeleteBuilder() {
-        bitField0_ |= 0x00000040;
+        bitField0_ |= 0x00000080;
         onChanged();
         return getDeleteFieldBuilder().getBuilder();
       }
       /**
-       * <code>required .MutationProto delete = 7;</code>
+       * <code>required .MutationProto delete = 8;</code>
        */
       public org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProtoOrBuilder getDeleteOrBuilder() {
         if (deleteBuilder_ != null) {
@@ -11284,7 +11700,7 @@ public final class TrxRegionProtos {
         }
       }
       /**
-       * <code>required .MutationProto delete = 7;</code>
+       * <code>required .MutationProto delete = 8;</code>
        */
       private com.google.protobuf.SingleFieldBuilder<
           org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto, org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.Builder, org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProtoOrBuilder> 
@@ -11965,77 +12381,87 @@ public final class TrxRegionProtos {
      */
     long getTid();
 
-    // required bytes regionName = 2;
+    // required int64 commitId = 2;
     /**
-     * <code>required bytes regionName = 2;</code>
+     * <code>required int64 commitId = 2;</code>
+     */
+    boolean hasCommitId();
+    /**
+     * <code>required int64 commitId = 2;</code>
+     */
+    long getCommitId();
+
+    // required bytes regionName = 3;
+    /**
+     * <code>required bytes regionName = 3;</code>
      */
     boolean hasRegionName();
     /**
-     * <code>required bytes regionName = 2;</code>
+     * <code>required bytes regionName = 3;</code>
      */
     com.google.protobuf.ByteString getRegionName();
 
-    // required bytes row = 3;
+    // required bytes row = 4;
     /**
-     * <code>required bytes row = 3;</code>
+     * <code>required bytes row = 4;</code>
      */
     boolean hasRow();
     /**
-     * <code>required bytes row = 3;</code>
+     * <code>required bytes row = 4;</code>
      */
     com.google.protobuf.ByteString getRow();
 
-    // required bytes family = 4;
+    // required bytes family = 5;
     /**
-     * <code>required bytes family = 4;</code>
+     * <code>required bytes family = 5;</code>
      */
     boolean hasFamily();
     /**
-     * <code>required bytes family = 4;</code>
+     * <code>required bytes family = 5;</code>
      */
     com.google.protobuf.ByteString getFamily();
 
-    // required bytes qualifier = 5;
+    // required bytes qualifier = 6;
     /**
-     * <code>required bytes qualifier = 5;</code>
+     * <code>required bytes qualifier = 6;</code>
      */
     boolean hasQualifier();
     /**
-     * <code>required bytes qualifier = 5;</code>
+     * <code>required bytes qualifier = 6;</code>
      */
     com.google.protobuf.ByteString getQualifier();
 
-    // required bytes value = 6;
+    // required bytes value = 7;
     /**
-     * <code>required bytes value = 6;</code>
+     * <code>required bytes value = 7;</code>
      */
     boolean hasValue();
     /**
-     * <code>required bytes value = 6;</code>
+     * <code>required bytes value = 7;</code>
      */
     com.google.protobuf.ByteString getValue();
 
-    // required .MutationProto delete = 7;
+    // required .MutationProto delete = 8;
     /**
-     * <code>required .MutationProto delete = 7;</code>
+     * <code>required .MutationProto delete = 8;</code>
      */
     boolean hasDelete();
     /**
-     * <code>required .MutationProto delete = 7;</code>
+     * <code>required .MutationProto delete = 8;</code>
      */
     org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto getDelete();
     /**
-     * <code>required .MutationProto delete = 7;</code>
+     * <code>required .MutationProto delete = 8;</code>
      */
     org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProtoOrBuilder getDeleteOrBuilder();
 
-    // required bool autoCommit = 8;
+    // required bool autoCommit = 9;
     /**
-     * <code>required bool autoCommit = 8;</code>
+     * <code>required bool autoCommit = 9;</code>
      */
     boolean hasAutoCommit();
     /**
-     * <code>required bool autoCommit = 8;</code>
+     * <code>required bool autoCommit = 9;</code>
      */
     boolean getAutoCommit();
   }
@@ -12095,34 +12521,39 @@ public final class TrxRegionProtos {
               tid_ = input.readInt64();
               break;
             }
-            case 18: {
+            case 16: {
               bitField0_ |= 0x00000002;
-              regionName_ = input.readBytes();
+              commitId_ = input.readInt64();
               break;
             }
             case 26: {
               bitField0_ |= 0x00000004;
-              row_ = input.readBytes();
+              regionName_ = input.readBytes();
               break;
             }
             case 34: {
               bitField0_ |= 0x00000008;
-              family_ = input.readBytes();
+              row_ = input.readBytes();
               break;
             }
             case 42: {
               bitField0_ |= 0x00000010;
-              qualifier_ = input.readBytes();
+              family_ = input.readBytes();
               break;
             }
             case 50: {
               bitField0_ |= 0x00000020;
-              value_ = input.readBytes();
+              qualifier_ = input.readBytes();
               break;
             }
             case 58: {
+              bitField0_ |= 0x00000040;
+              value_ = input.readBytes();
+              break;
+            }
+            case 66: {
               org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.Builder subBuilder = null;
-              if (((bitField0_ & 0x00000040) == 0x00000040)) {
+              if (((bitField0_ & 0x00000080) == 0x00000080)) {
                 subBuilder = delete_.toBuilder();
               }
               delete_ = input.readMessage(org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.PARSER, extensionRegistry);
@@ -12130,11 +12561,11 @@ public final class TrxRegionProtos {
                 subBuilder.mergeFrom(delete_);
                 delete_ = subBuilder.buildPartial();
               }
-              bitField0_ |= 0x00000040;
+              bitField0_ |= 0x00000080;
               break;
             }
-            case 64: {
-              bitField0_ |= 0x00000080;
+            case 72: {
+              bitField0_ |= 0x00000100;
               autoCommit_ = input.readBool();
               break;
             }
@@ -12194,119 +12625,135 @@ public final class TrxRegionProtos {
       return tid_;
     }
 
-    // required bytes regionName = 2;
-    public static final int REGIONNAME_FIELD_NUMBER = 2;
+    // required int64 commitId = 2;
+    public static final int COMMITID_FIELD_NUMBER = 2;
+    private long commitId_;
+    /**
+     * <code>required int64 commitId = 2;</code>
+     */
+    public boolean hasCommitId() {
+      return ((bitField0_ & 0x00000002) == 0x00000002);
+    }
+    /**
+     * <code>required int64 commitId = 2;</code>
+     */
+    public long getCommitId() {
+      return commitId_;
+    }
+
+    // required bytes regionName = 3;
+    public static final int REGIONNAME_FIELD_NUMBER = 3;
     private com.google.protobuf.ByteString regionName_;
     /**
-     * <code>required bytes regionName = 2;</code>
+     * <code>required bytes regionName = 3;</code>
      */
     public boolean hasRegionName() {
-      return ((bitField0_ & 0x00000002) == 0x00000002);
+      return ((bitField0_ & 0x00000004) == 0x00000004);
     }
     /**
-     * <code>required bytes regionName = 2;</code>
+     * <code>required bytes regionName = 3;</code>
      */
     public com.google.protobuf.ByteString getRegionName() {
       return regionName_;
     }
 
-    // required bytes row = 3;
-    public static final int ROW_FIELD_NUMBER = 3;
+    // required bytes row = 4;
+    public static final int ROW_FIELD_NUMBER = 4;
     private com.google.protobuf.ByteString row_;
     /**
-     * <code>required bytes row = 3;</code>
+     * <code>required bytes row = 4;</code>
      */
     public boolean hasRow() {
-      return ((bitField0_ & 0x00000004) == 0x00000004);
+      return ((bitField0_ & 0x00000008) == 0x00000008);
     }
     /**
-     * <code>required bytes row = 3;</code>
+     * <code>required bytes row = 4;</code>
      */
     public com.google.protobuf.ByteString getRow() {
       return row_;
     }
 
-    // required bytes family = 4;
-    public static final int FAMILY_FIELD_NUMBER = 4;
+    // required bytes family = 5;
+    public static final int FAMILY_FIELD_NUMBER = 5;
     private com.google.protobuf.ByteString family_;
     /**
-     * <code>required bytes family = 4;</code>
+     * <code>required bytes family = 5;</code>
      */
     public boolean hasFamily() {
-      return ((bitField0_ & 0x00000008) == 0x00000008);
+      return ((bitField0_ & 0x00000010) == 0x00000010);
     }
     /**
-     * <code>required bytes family = 4;</code>
+     * <code>required bytes family = 5;</code>
      */
     public com.google.protobuf.ByteString getFamily() {
       return family_;
     }
 
-    // required bytes qualifier = 5;
-    public static final int QUALIFIER_FIELD_NUMBER = 5;
+    // required bytes qualifier = 6;
+    public static final int QUALIFIER_FIELD_NUMBER = 6;
     private com.google.protobuf.ByteString qualifier_;
     /**
-     * <code>required bytes qualifier = 5;</code>
+     * <code>required bytes qualifier = 6;</code>
      */
     public boolean hasQualifier() {
-      return ((bitField0_ & 0x00000010) == 0x00000010);
+      return ((bitField0_ & 0x00000020) == 0x00000020);
     }
     /**
-     * <code>required bytes qualifier = 5;</code>
+     * <code>required bytes qualifier = 6;</code>
      */
     public com.google.protobuf.ByteString getQualifier() {
       return qualifier_;
     }
 
-    // required bytes value = 6;
-    public static final int VALUE_FIELD_NUMBER = 6;
+    // required bytes value = 7;
+    public static final int VALUE_FIELD_NUMBER = 7;
     private com.google.protobuf.ByteString value_;
     /**
-     * <code>required bytes value = 6;</code>
+     * <code>required bytes value = 7;</code>
      */
     public boolean hasValue() {
-      return ((bitField0_ & 0x00000020) == 0x00000020);
+      return ((bitField0_ & 0x00000040) == 0x00000040);
     }
     /**
-     * <code>required bytes value = 6;</code>
+     * <code>required bytes value = 7;</code>
      */
     public com.google.protobuf.ByteString getValue() {
       return value_;
     }
 
-    // required .MutationProto delete = 7;
-    public static final int DELETE_FIELD_NUMBER = 7;
+    // required .MutationProto delete = 8;
+    public static final int DELETE_FIELD_NUMBER = 8;
     private org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto delete_;
     /**
-     * <code>required .MutationProto delete = 7;</code>
+     * <code>required .MutationProto delete = 8;</code>
      */
     public boolean hasDelete() {
-      return ((bitField0_ & 0x00000040) == 0x00000040);
+      return ((bitField0_ & 0x00000080) == 0x00000080);
     }
     /**
-     * <code>required .MutationProto delete = 7;</code>
+     * <code>required .MutationProto delete = 8;</code>
      */
     public org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto getDelete() {
       return delete_;
     }
     /**
-     * <code>required .MutationProto delete = 7;</code>
+     * <code>required .MutationProto delete = 8;</code>
      */
     public org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProtoOrBuilder getDeleteOrBuilder() {
       return delete_;
     }
 
-    // required bool autoCommit = 8;
-    public static final int AUTOCOMMIT_FIELD_NUMBER = 8;
+    // required bool autoCommit = 9;
+    public static final int AUTOCOMMIT_FIELD_NUMBER = 9;
     private boolean autoCommit_;
     /**
-     * <code>required bool autoCommit = 8;</code>
+     * <code>required bool autoCommit = 9;</code>
      */
     public boolean hasAutoCommit() {
-      return ((bitField0_ & 0x00000080) == 0x00000080);
+      return ((bitField0_ & 0x00000100) == 0x00000100);
     }
     /**
-     * <code>required bool autoCommit = 8;</code>
+     * <code>required bool autoCommit = 9;</code>
      */
     public boolean getAutoCommit() {
       return autoCommit_;
@@ -12314,6 +12761,7 @@ public final class TrxRegionProtos {
 
     private void initFields() {
       tid_ = 0L;
+      commitId_ = 0L;
       regionName_ = com.google.protobuf.ByteString.EMPTY;
       row_ = com.google.protobuf.ByteString.EMPTY;
       family_ = com.google.protobuf.ByteString.EMPTY;
@@ -12331,6 +12779,10 @@ public final class TrxRegionProtos {
         memoizedIsInitialized = 0;
         return false;
       }
+      if (!hasCommitId()) {
+        memoizedIsInitialized = 0;
+        return false;
+      }
       if (!hasRegionName()) {
         memoizedIsInitialized = 0;
         return false;
@@ -12374,25 +12826,28 @@ public final class TrxRegionProtos {
         output.writeInt64(1, tid_);
       }
       if (((bitField0_ & 0x00000002) == 0x00000002)) {
-        output.writeBytes(2, regionName_);
+        output.writeInt64(2, commitId_);
       }
       if (((bitField0_ & 0x00000004) == 0x00000004)) {
-        output.writeBytes(3, row_);
+        output.writeBytes(3, regionName_);
       }
       if (((bitField0_ & 0x00000008) == 0x00000008)) {
-        output.writeBytes(4, family_);
+        output.writeBytes(4, row_);
       }
       if (((bitField0_ & 0x00000010) == 0x00000010)) {
-        output.writeBytes(5, qualifier_);
+        output.writeBytes(5, family_);
       }
       if (((bitField0_ & 0x00000020) == 0x00000020)) {
-        output.writeBytes(6, value_);
+        output.writeBytes(6, qualifier_);
       }
       if (((bitField0_ & 0x00000040) == 0x00000040)) {
-        output.writeMessage(7, delete_);
+        output.writeBytes(7, value_);
       }
       if (((bitField0_ & 0x00000080) == 0x00000080)) {
-        output.writeBool(8, autoCommit_);
+        output.writeMessage(8, delete_);
+      }
+      if (((bitField0_ & 0x00000100) == 0x00000100)) {
+        output.writeBool(9, autoCommit_);
       }
       getUnknownFields().writeTo(output);
     }
@@ -12409,31 +12864,35 @@ public final class TrxRegionProtos {
       }
       if (((bitField0_ & 0x00000002) == 0x00000002)) {
         size += com.google.protobuf.CodedOutputStream
-          .computeBytesSize(2, regionName_);
+          .computeInt64Size(2, commitId_);
       }
       if (((bitField0_ & 0x00000004) == 0x00000004)) {
         size += com.google.protobuf.CodedOutputStream
-          .computeBytesSize(3, row_);
+          .computeBytesSize(3, regionName_);
       }
       if (((bitField0_ & 0x00000008) == 0x00000008)) {
         size += com.google.protobuf.CodedOutputStream
-          .computeBytesSize(4, family_);
+          .computeBytesSize(4, row_);
       }
       if (((bitField0_ & 0x00000010) == 0x00000010)) {
         size += com.google.protobuf.CodedOutputStream
-          .computeBytesSize(5, qualifier_);
+          .computeBytesSize(5, family_);
       }
       if (((bitField0_ & 0x00000020) == 0x00000020)) {
         size += com.google.protobuf.CodedOutputStream
-          .computeBytesSize(6, value_);
+          .computeBytesSize(6, qualifier_);
       }
       if (((bitField0_ & 0x00000040) == 0x00000040)) {
         size += com.google.protobuf.CodedOutputStream
-          .computeMessageSize(7, delete_);
+          .computeBytesSize(7, value_);
       }
       if (((bitField0_ & 0x00000080) == 0x00000080)) {
         size += com.google.protobuf.CodedOutputStream
-          .computeBoolSize(8, autoCommit_);
+          .computeMessageSize(8, delete_);
+      }
+      if (((bitField0_ & 0x00000100) == 0x00000100)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeBoolSize(9, autoCommit_);
       }
       size += getUnknownFields().getSerializedSize();
       memoizedSerializedSize = size;
@@ -12554,24 +13013,26 @@ public final class TrxRegionProtos {
         super.clear();
         tid_ = 0L;
         bitField0_ = (bitField0_ & ~0x00000001);
-        regionName_ = com.google.protobuf.ByteString.EMPTY;
+        commitId_ = 0L;
         bitField0_ = (bitField0_ & ~0x00000002);
-        row_ = com.google.protobuf.ByteString.EMPTY;
+        regionName_ = com.google.protobuf.ByteString.EMPTY;
         bitField0_ = (bitField0_ & ~0x00000004);
-        family_ = com.google.protobuf.ByteString.EMPTY;
+        row_ = com.google.protobuf.ByteString.EMPTY;
         bitField0_ = (bitField0_ & ~0x00000008);
-        qualifier_ = com.google.protobuf.ByteString.EMPTY;
+        family_ = com.google.protobuf.ByteString.EMPTY;
         bitField0_ = (bitField0_ & ~0x00000010);
-        value_ = com.google.protobuf.ByteString.EMPTY;
+        qualifier_ = com.google.protobuf.ByteString.EMPTY;
         bitField0_ = (bitField0_ & ~0x00000020);
+        value_ = com.google.protobuf.ByteString.EMPTY;
+        bitField0_ = (bitField0_ & ~0x00000040);
         if (deleteBuilder_ == null) {
           delete_ = org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.getDefaultInstance();
         } else {
           deleteBuilder_.clear();
         }
-        bitField0_ = (bitField0_ & ~0x00000040);
-        autoCommit_ = false;
         bitField0_ = (bitField0_ & ~0x00000080);
+        autoCommit_ = false;
+        bitField0_ = (bitField0_ & ~0x00000100);
         return this;
       }
 
@@ -12607,33 +13068,37 @@ public final class TrxRegionProtos {
         if (((from_bitField0_ & 0x00000002) == 0x00000002)) {
           to_bitField0_ |= 0x00000002;
         }
-        result.regionName_ = regionName_;
+        result.commitId_ = commitId_;
         if (((from_bitField0_ & 0x00000004) == 0x00000004)) {
           to_bitField0_ |= 0x00000004;
         }
-        result.row_ = row_;
+        result.regionName_ = regionName_;
         if (((from_bitField0_ & 0x00000008) == 0x00000008)) {
           to_bitField0_ |= 0x00000008;
         }
-        result.family_ = family_;
+        result.row_ = row_;
         if (((from_bitField0_ & 0x00000010) == 0x00000010)) {
           to_bitField0_ |= 0x00000010;
         }
-        result.qualifier_ = qualifier_;
+        result.family_ = family_;
         if (((from_bitField0_ & 0x00000020) == 0x00000020)) {
           to_bitField0_ |= 0x00000020;
         }
-        result.value_ = value_;
+        result.qualifier_ = qualifier_;
         if (((from_bitField0_ & 0x00000040) == 0x00000040)) {
           to_bitField0_ |= 0x00000040;
         }
+        result.value_ = value_;
+        if (((from_bitField0_ & 0x00000080) == 0x00000080)) {
+          to_bitField0_ |= 0x00000080;
+        }
         if (deleteBuilder_ == null) {
           result.delete_ = delete_;
         } else {
           result.delete_ = deleteBuilder_.build();
         }
-        if (((from_bitField0_ & 0x00000080) == 0x00000080)) {
-          to_bitField0_ |= 0x00000080;
+        if (((from_bitField0_ & 0x00000100) == 0x00000100)) {
+          to_bitField0_ |= 0x00000100;
         }
         result.autoCommit_ = autoCommit_;
         result.bitField0_ = to_bitField0_;
@@ -12655,6 +13120,9 @@ public final class TrxRegionProtos {
         if (other.hasTid()) {
           setTid(other.getTid());
         }
+        if (other.hasCommitId()) {
+          setCommitId(other.getCommitId());
+        }
         if (other.hasRegionName()) {
           setRegionName(other.getRegionName());
         }
@@ -12685,6 +13153,10 @@ public final class TrxRegionProtos {
           
           return false;
         }
+        if (!hasCommitId()) {
+          
+          return false;
+        }
         if (!hasRegionName()) {
           
           return false;
@@ -12772,198 +13244,231 @@ public final class TrxRegionProtos {
         return this;
       }
 
-      // required bytes regionName = 2;
+      // required int64 commitId = 2;
+      private long commitId_ ;
+      /**
+       * <code>required int64 commitId = 2;</code>
+       */
+      public boolean hasCommitId() {
+        return ((bitField0_ & 0x00000002) == 0x00000002);
+      }
+      /**
+       * <code>required int64 commitId = 2;</code>
+       */
+      public long getCommitId() {
+        return commitId_;
+      }
+      /**
+       * <code>required int64 commitId = 2;</code>
+       */
+      public Builder setCommitId(long value) {
+        bitField0_ |= 0x00000002;
+        commitId_ = value;
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>required int64 commitId = 2;</code>
+       */
+      public Builder clearCommitId() {
+        bitField0_ = (bitField0_ & ~0x00000002);
+     

<TRUNCATED>


[19/22] incubator-trafodion git commit: Merge branch 'master' of github.com:apache/incubator-trafodion into sean_traf

Posted by sa...@apache.org.
Merge branch 'master' of github.com:apache/incubator-trafodion into sean_traf


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

Branch: refs/heads/master
Commit: 85115b091a53ac0c54ba91cd61121635ffd79c6d
Parents: 41fcd8d 8216550
Author: Sean Broeder <sb...@edev05.esgyn.local>
Authored: Thu May 11 17:00:16 2017 +0000
Committer: Sean Broeder <sb...@edev05.esgyn.local>
Committed: Thu May 11 17:00:16 2017 +0000

----------------------------------------------------------------------
 .../native/ResStatisticsStatement.cpp           |   72 +-
 .../jdbc_type2/native/ResStatisticsStatement.h  |    9 +
 .../odbc/nsksrvrcore/ResStatisticsStatement.cpp |   88 +-
 .../odbc/nsksrvrcore/ResStatisticsStatement.h   |    7 +
 .../run_full_trafodion_backup.sh                |    2 +-
 .../sql/scripts/install_hadoop_regr_test_env    |  109 +-
 core/sqf/sql/scripts/install_local_hadoop       |   15 +-
 core/sql/cli/sqlcli.h                           |    2 +
 core/sql/comexe/ComTdbHashGrby.h                |    2 +
 core/sql/executor/ExExeUtil.h                   |    8 +-
 core/sql/executor/ExExeUtilGetStats.cpp         |  444 ++--
 core/sql/executor/ExStats.cpp                   |  311 +--
 core/sql/executor/ExStats.h                     |   30 +-
 core/sql/executor/ex_hash_grby.cpp              |   21 +-
 core/sql/executor/ex_hash_grby.h                |   11 +
 core/sql/executor/ex_hashj.cpp                  |   26 +-
 core/sql/executor/ex_hashj.h                    |   18 +-
 core/sql/executor/ex_sort.cpp                   |  321 ++-
 core/sql/executor/ex_sort.h                     |   46 +-
 core/sql/generator/GenPreCode.cpp               |    1 -
 core/sql/optimizer/OptimizerSimulator.cpp       |    2 +-
 core/sql/optimizer/RelExeUtil.cpp               |    3 +-
 core/sql/optimizer/RelExeUtil.h                 |    7 +-
 core/sql/parser/AllStmtDDL.h                    |    1 -
 core/sql/parser/BindStmtDDL.cpp                 |   26 -
 core/sql/parser/ElemDDLNode.cpp                 |    6 -
 core/sql/parser/ElemDDLNode.h                   |    2 -
 core/sql/parser/ParKeyWords.cpp                 |    2 -
 core/sql/parser/StmtDDLNode.cpp                 |  195 +-
 core/sql/parser/StmtDDLPublish.h                |  269 ---
 core/sql/parser/sqlparser.y                     |  100 +-
 core/sql/regress/core/EXPECTEDRTS               | 1903 +++++++++---------
 core/sql/regress/core/FILTERRTS                 |   18 +-
 core/sql/regress/executor/DIFF140.KNOWN         |    4 +
 core/sql/regress/executor/EXPECTED130           |  220 +-
 core/sql/regress/executor/TEST130               |   90 +-
 core/sql/regress/hive/EXPECTED003               |    6 +-
 core/sql/regress/hive/EXPECTED005               |  104 +-
 core/sql/regress/hive/EXPECTED009               |   22 +-
 core/sql/regress/hive/EXPECTED018               |  717 +++----
 core/sql/regress/hive/FILTER005                 |    4 +-
 core/sql/regress/hive/TEST003                   |   52 +-
 .../hive/TEST003_create_hive_tables.hive        |   24 +-
 core/sql/regress/hive/TEST005                   |   48 +-
 core/sql/regress/hive/TEST005_a.hive.sql        |   16 +-
 core/sql/regress/hive/TEST009                   |    6 +-
 core/sql/regress/hive/TEST009_a.hive.sql        |    4 +-
 core/sql/regress/hive/TEST009_b.hive.sql        |    2 +-
 core/sql/regress/hive/TEST018                   |  148 +-
 .../hive/TEST018_create_hive_tables.hive        |   14 +-
 core/sql/regress/seabase/EXPECTED024            |    6 +-
 core/sql/regress/seabase/TEST024                |    2 -
 core/sql/sort/Qsort.cpp                         |    9 +-
 core/sql/sort/Qsort.h                           |    3 +-
 core/sql/sort/SortAlgo.cpp                      |    3 +-
 core/sql/sort/SortAlgo.h                        |    3 +-
 core/sql/sort/SortTopN.cpp                      |    9 +-
 core/sql/sort/SortTopN.h                        |    3 +-
 core/sql/sort/SortUtil.cpp                      |  106 +-
 core/sql/sort/SortUtil.h                        |    6 +-
 core/sql/sort/TourTree.cpp                      |    8 +-
 core/sql/sort/TourTree.h                        |    1 +
 core/sql/sqlci/sqlci_lex.ll                     |    2 -
 core/sql/sqlci/sqlci_yacc.y                     |    4 -
 core/sql/sqlcomp/CmpSeabaseDDL.h                |    2 -
 core/sql/sqlcomp/CmpSeabaseDDLcommon.cpp        |   74 -
 core/sql/sqlcomp/PrivMgrComponentPrivileges.cpp |    9 +
 core/sql/sqlcomp/nadefaults.cpp                 |   14 +-
 .../src/asciidoc/_chapters/sql_statements.adoc  |    2 +-
 .../src/asciidoc/_chapters/sql_utilities.adoc   |  333 ++-
 docs/src/site/markdown/documentation.md         |   23 +-
 docs/src/site/markdown/download.md              |   48 +-
 docs/src/site/markdown/index.md                 |    4 +-
 docs/src/site/markdown/release-notes-2-1-0.md   |  164 ++
 docs/src/site/markdown/release-notes.md         |    1 +
 75 files changed, 3303 insertions(+), 3094 deletions(-)
----------------------------------------------------------------------



[07/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

Posted by sa...@apache.org.
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/coprocessor/transactional/TrxRegionEndpoint.java.tmpl
----------------------------------------------------------------------
diff --git a/core/sqf/src/seatrans/hbase-trx/src/main/java/org/apache/hadoop/hbase/coprocessor/transactional/TrxRegionEndpoint.java.tmpl b/core/sqf/src/seatrans/hbase-trx/src/main/java/org/apache/hadoop/hbase/coprocessor/transactional/TrxRegionEndpoint.java.tmpl
index 403df5f..78ed97e 100644
--- a/core/sqf/src/seatrans/hbase-trx/src/main/java/org/apache/hadoop/hbase/coprocessor/transactional/TrxRegionEndpoint.java.tmpl
+++ b/core/sqf/src/seatrans/hbase-trx/src/main/java/org/apache/hadoop/hbase/coprocessor/transactional/TrxRegionEndpoint.java.tmpl
@@ -28,6 +28,7 @@ import java.io.IOException;
 import org.apache.commons.codec.binary.Hex;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hbase.HBaseConfiguration;
 import org.apache.hadoop.hbase.Coprocessor;
 import org.apache.hadoop.hbase.CoprocessorEnvironment;
 import org.apache.hadoop.hbase.coprocessor.ColumnInterpreter;
@@ -56,6 +57,7 @@ import java.util.ListIterator;
 import java.util.LinkedList;
 import java.util.List;
 import java.util.Map;
+import java.util.NavigableMap;
 import java.util.Map.Entry;
 import java.util.NavigableSet;
 import java.util.Set;
@@ -93,6 +95,7 @@ import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.CellUtil;
+import org.apache.hadoop.hbase.client.HBaseAdmin;
 #ifdef HDP2.3 HDP2.4 CDH5.5 CDH5.7 APACHE1.2
 import org.apache.hadoop.hbase.ChoreService;
 import org.apache.hadoop.hbase.ScheduledChore;
@@ -107,6 +110,7 @@ import org.apache.hadoop.hbase.client.Scan;
 import org.apache.hadoop.hbase.client.ScannerTimeoutException;
 import org.apache.hadoop.hbase.client.Result;
 import org.apache.hadoop.hbase.client.transactional.MemoryUsageException;
+import org.apache.hadoop.hbase.client.transactional.NonPendingTransactionException;
 import org.apache.hadoop.hbase.client.transactional.OutOfOrderProtocolException;
 import org.apache.hadoop.hbase.client.transactional.UnknownTransactionException;
 import org.apache.hadoop.hbase.client.transactional.BatchException;
@@ -116,6 +120,10 @@ import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.CellUtil;
+#ifdef HDP2.3 HDP2.4 CDH5.7 APACHE1.2
+import org.apache.hadoop.hbase.ChoreService;
+import org.apache.hadoop.hbase.ScheduledChore;
+#endif
 import org.apache.hadoop.hbase.filter.FilterBase;
 import org.apache.hadoop.hbase.filter.FilterList;
 import org.apache.hadoop.hbase.KeyValueUtil;
@@ -142,18 +150,21 @@ import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.Tag;
-import org.apache.hadoop.hbase.client.Mutation;
 import org.apache.hadoop.hbase.regionserver.RegionServerServices;
 import org.apache.hadoop.hbase.Stoppable;
 import org.apache.hadoop.hbase.NotServingRegionException;
 import org.apache.hadoop.hbase.UnknownScannerException;
 import org.apache.hadoop.hbase.regionserver.HRegion;
+import org.apache.hadoop.hbase.regionserver.InternalScanner;
 import org.apache.hadoop.hbase.regionserver.KeyValueScanner;
 import org.apache.hadoop.hbase.regionserver.LeaseException;
 import org.apache.hadoop.hbase.regionserver.LeaseListener;
 import org.apache.hadoop.hbase.regionserver.Leases;
 import org.apache.hadoop.hbase.regionserver.Leases.LeaseStillHeldException;
 //import org.apache.hadoop.hbase.regionserver.MultiVersionConsistencyControl;
+#ifdef CDH5.7 APACHE1.2
+import org.apache.hadoop.hbase.regionserver.Region;
+#endif
 import org.apache.hadoop.hbase.regionserver.RegionCoprocessorHost;
 import org.apache.hadoop.hbase.regionserver.RegionScanner;
 import org.apache.hadoop.hbase.regionserver.WrongRegionException;
@@ -235,6 +246,10 @@ import org.apache.hadoop.hbase.coprocessor.transactional.generated.TrxRegionProt
 import org.apache.hadoop.hbase.coprocessor.transactional.generated.TrxRegionProtos.TransactionalAggregateRequest;
 import org.apache.hadoop.hbase.coprocessor.transactional.generated.TrxRegionProtos.TransactionalAggregateResponse;
 import org.apache.hadoop.hbase.coprocessor.transactional.generated.TrxRegionProtos.TrxRegionService;
+#ifdef CDH5.7 APACHE1.2
+import org.apache.hadoop.hbase.regionserver.MultiVersionConcurrencyControl.WriteEntry;
+import org.apache.hadoop.hbase.regionserver.MultiVersionConcurrencyControl;
+#endif
 import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
 import org.apache.hadoop.hbase.zookeeper.ZKUtil;
 import org.apache.zookeeper.KeeperException;
@@ -307,8 +322,6 @@ CoprocessorService, Coprocessor {
   private final Boolean splitDelayEnabled = false;
   private final Boolean doWALHlog = false;
   static Leases transactionLeases = null;
-  // Joanie: commenting out scanner leases for now
-  //static Leases scannerLeases = null;
   CleanOldTransactionsChore cleanOldTransactionsThread;
   static MemoryUsageChore memoryUsageThread = null;
   Stoppable stoppable = new StoppableImplementation();
@@ -318,7 +331,7 @@ CoprocessorService, Coprocessor {
   private int regionState = 0; 
   private Path recoveryTrxPath = null;
   private int cleanAT = 0;
-  private long onlineEpoch = EnvironmentEdgeManager.currentTime();  
+  private long onlineEpoch = EnvironmentEdgeManager.currentTime();
   
   private long[] commitCheckTimes   = new long[50];
   private long[] hasConflictTimes   = new long[50];
@@ -347,8 +360,13 @@ CoprocessorService, Coprocessor {
   private double avgWriteToLogTime    =    0;
 
   private HRegionInfo regionInfo = null;
+#ifdef CDH5.7 APACHE1.2
+  private Region m_Region = null;
+#else
   private HRegion m_Region = null;
+#endif
   private String m_regionName = null;
+  private String m_regionDetails = null;
   private boolean m_isTrafodionMetadata = false;
 #ifdef CDH5.7 APACHE1.2
   private HRegion t_Region = null;
@@ -376,7 +394,7 @@ CoprocessorService, Coprocessor {
 
   private static final int MINIMUM_LEASE_TIME = 7200 * 1000;
   private static final int LEASE_CHECK_FREQUENCY = 1000;
-  private static final int DEFAULT_SLEEP = 60 * 1000;
+  private static final int DEFAULT_SLEEP = 30 * 1000; // 30 seconds
   private static final int DEFAULT_MEMORY_THRESHOLD = 100; // 100% memory used
   private static final int DEFAULT_STARTUP_MEMORY_THRESHOLD = 90; // initial value : 90% memory used
   private static final int DEFAULT_MEMORY_SLEEP = 15 * 1000;
@@ -386,6 +404,7 @@ CoprocessorService, Coprocessor {
   private static final boolean DEFAULT_SKIP_WAL = false;
   private static final boolean DEFAULT_COMMIT_EDIT = false;
   private static final boolean DEFAULT_SUPPRESS_OOP = false;
+  private static final boolean DEFAULT_TM_USE_COMMIT_ID_IN_CELLS = false;
   private static final String SLEEP_CONF = "hbase.transaction.clean.sleep";
   private static final String LEASE_CONF  = "hbase.transaction.lease.timeout";
   private static final String MEMORY_THRESHOLD = "hbase.transaction.memory.threshold";
@@ -397,6 +416,7 @@ CoprocessorService, Coprocessor {
   private static final String CONF_COMMIT_EDIT  = "hbase.trafodion.full.commit.edit";
   private static final String SUPPRESS_OOP = "hbase.transaction.suppress.OOP.exception";
   private static final String CHECK_ROW = "hbase.transaction.check.row";
+  private static final String CONF_TM_USE_COMMIT_ID_IN_CELLS = "hbase.transaction.use.commitId";
   protected static int transactionLeaseTimeout = 0;
   private static int scannerLeaseTimeoutPeriod = 0;
   private static int scannerThreadWakeFrequency = 0;
@@ -406,6 +426,7 @@ CoprocessorService, Coprocessor {
   private static int asyncWal = DEFAULT_ASYNC_WAL;
   private static boolean skipWal = DEFAULT_SKIP_WAL;
   private static boolean fullEditInCommit = DEFAULT_COMMIT_EDIT;
+  private static boolean useCommitIdInCells = DEFAULT_TM_USE_COMMIT_ID_IN_CELLS;
   private static MemoryMXBean memoryBean = null;
   private static float memoryPercentage = 0;
   private static boolean memoryThrottle = false;
@@ -449,6 +470,7 @@ CoprocessorService, Coprocessor {
   
   // TBD Maybe we should just use HashMap to improve the performance, ConcurrentHashMap could be too strict
   static ConcurrentHashMap<String, Object> transactionsEPCPMap;
+  long choreCount = 1;
   // TrxRegionService methods
     
   @Override
@@ -459,44 +481,31 @@ CoprocessorService, Coprocessor {
 
     long transactionId = request.getTransactionId();
     boolean dropTableRecorded = request.getDropTableRecorded();
-    if (LOG.isTraceEnabled()) LOG.trace("TrxRegionEndpoint coprocessor: abortTransaction - txId " + transactionId +
-             ", dropTableRecoded " + dropTableRecorded + ", regionName " + regionInfo.getRegionNameAsString());
+    boolean ignoreUnknownTransaction = request.getIgnoreUnknownTransactionException();
+    if (LOG.isTraceEnabled()) LOG.trace("abortTransaction - txId " + transactionId +
+             ", dropTableRecoded " + dropTableRecorded + ", regionName " + m_regionDetails);
 
     IOException ioe = null;
     UnknownTransactionException ute = null;
-    WrongRegionException wre = null;
     Throwable t = null;
 
-    /*  commenting out for the time being
-    java.lang.String name = ((com.google.protobuf.ByteString) request.getRegionName()).toStringUtf8();
-
-    // First test if this region matches our region name
-    if (!name.equals(regionInfo.getRegionNameAsString())) {
-        wre = new WrongRegionException("Request Region Name, " +
-        name + ",  does not match this region, " +
-        regionInfo.getRegionNameAsString());
-        if (LOG.isTraceEnabled()) LOG.trace("TrxRegionEndpoint coprocessor:abortTransaction threw WrongRegionException" +
-      "Request Region Name, " +
-        name + ",  does not match this region, " +
-        regionInfo.getRegionNameAsString());
-    } else 
-    */
-    {
       // Process in local memory
+      int participant = request.getParticipantNum();
       try {
-        abortTransaction(transactionId, dropTableRecorded);
+        abortTransaction(transactionId, dropTableRecorded, ignoreUnknownTransaction);
       } catch (UnknownTransactionException u) {
-        if (LOG.isDebugEnabled()) LOG.debug("TrxRegionEndpoint coprocessor:abort - txId " 
+        LOG.error("TrxRegionEndpoint coprocessor:abort - txId "
 					    + transactionId 
+					    + " participant " + participant
 					    + ", Caught UnknownTransactionException after internal abortTransaction call - " 
 					    + u.getMessage() + " " 
 					    + stackTraceToString(u));
 	ute = u;
       } catch (IOException e) {
-        if (LOG.isDebugEnabled()) LOG.debug("TrxRegionEndpoint coprocessor:abort - txId " + transactionId + ", Caught IOException after internal abortTransaction call - " + e.getMessage() + " " + stackTraceToString(e));
+        LOG.error("TrxRegionEndpoint coprocessor:abort - txId " + transactionId + " participant " + participant
+         + ", Caught IOException after internal abortTransaction call - ", e);
         ioe = e;
       }
-    }
 
     org.apache.hadoop.hbase.coprocessor.transactional.generated.TrxRegionProtos.AbortTransactionResponse.Builder abortTransactionResponseBuilder = AbortTransactionResponse.newBuilder();
 
@@ -508,12 +517,6 @@ CoprocessorService, Coprocessor {
       abortTransactionResponseBuilder.setException(t.toString());
     }
 
-    if (wre != null)
-    {
-      abortTransactionResponseBuilder.setHasException(true);
-      abortTransactionResponseBuilder.setException(wre.toString());
-    }
-
     if (ioe != null)
     {
       abortTransactionResponseBuilder.setHasException(true);
@@ -543,20 +546,20 @@ CoprocessorService, Coprocessor {
     String requestRegionName;
     IOException ioe = null;
     UnknownTransactionException ute = null;
-    WrongRegionException wre = null;
     Throwable t = null;
 
-    if (LOG.isTraceEnabled()) LOG.trace("TrxRegionEndpoint coprocessor: abortMultiple - txId " + transactionId + ", master regionName " + regionInfo.getRegionNameAsString());
-    if (LOG.isTraceEnabled()) LOG.trace("TrxRegionEndpoint coprocessor: abortMultiple - txId " + transactionId + " number of region is commitMultiple " + numOfRegion);
+    if (LOG.isTraceEnabled()) LOG.trace("abortMultiple - txId " + transactionId + " number of region is commitMultiple "
+            + numOfRegion + ", master regionName " + m_regionName);
 
     org.apache.hadoop.hbase.coprocessor.transactional.generated.TrxRegionProtos.AbortTransactionMultipleResponse.Builder abortTransactionMultipleResponseBuilder = AbortTransactionMultipleResponse.newBuilder();
     abortTransactionMultipleResponseBuilder.setHasException(false);
 
+    int participant = 0;
     while (i < numOfRegion) {
          requestRegionName = request.getRegionName(i).toStringUtf8();    
          abortTransactionMultipleResponseBuilder.addException(BatchException.EXCEPTION_OK.toString());
-
          try {
+              participant = request.getParticipantNum();
               if (LOG.isTraceEnabled()) LOG.trace("TrxRegionEndpoint abortMultiple begins for region " + requestRegionName);
               TrxRegionEndpoint regionEPCP = (TrxRegionEndpoint) transactionsEPCPMap.get(requestRegionName+trxkeyEPCPinstance);
               if (regionEPCP == null) {
@@ -570,10 +573,11 @@ CoprocessorService, Coprocessor {
               if (LOG.isTraceEnabled()) LOG.trace("TrxRegionEndpoint abortMultiple ends");
              // abortTransaction(transactionId);
          } catch (UnknownTransactionException u) {
-              if (LOG.isDebugEnabled()) LOG.debug("TrxRegionEndpoint coprocessor:abort - txId " + transactionId + ", Caught UnknownTransactionException after internal abortTransaction call - " + u.getMessage() + " " + stackTraceToString(u));
+              LOG.error("TrxRegionEndpoint coprocessor:abort - txId " + transactionId + " participant " + participant + ", Caught UnknownTransactionException after internal abortTransaction call - " + u.getMessage() + " " + stackTraceToString(u));
               ute = u;
          } catch (IOException e) {
-              if (LOG.isDebugEnabled()) LOG.debug("TrxRegionEndpoint coprocessor:abort - txId " + transactionId + ", Caught IOException after internal abortTransaction call - " + e.getMessage() + " " + stackTraceToString(e));
+              LOG.error("TrxRegionEndpoint coprocessor:abort - txId " + transactionId + " participant " + participant
+               + ", Caught IOException after internal abortTransaction call - ", e);
               ioe = e;
          }
 
@@ -583,12 +587,6 @@ CoprocessorService, Coprocessor {
               abortTransactionMultipleResponseBuilder.setException(i, t.toString());
          }
 
-         if (wre != null)
-         {
-              abortTransactionMultipleResponseBuilder.setHasException(true);
-              abortTransactionMultipleResponseBuilder.setException(i, wre.toString());
-         }
-
          if (ioe != null)
          {
               abortTransactionMultipleResponseBuilder.setHasException(true);
@@ -618,39 +616,26 @@ CoprocessorService, Coprocessor {
 
     Throwable t = null;
     java.lang.String name = ((com.google.protobuf.ByteString) request.getRegionName()).toStringUtf8();
-    WrongRegionException wre = null;
     MemoryUsageException mue = null;
     long transactionId = request.getTransactionId();
+    long startId = request.getStartId();
 
-    if (LOG.isTraceEnabled()) LOG.trace("TrxRegionEndpoint coprocessor: beginTransaction - txId "  + transactionId + ", regionName " + regionInfo.getRegionNameAsString());
-
-    // First test if this region matches our region name
+    if (LOG.isTraceEnabled()) LOG.trace("beginTransaction - txId "  + transactionId + ", regionName " + m_regionDetails);
 
-    /* commenting it out for the time-being
-    if (!name.equals(regionInfo.getRegionNameAsString())) {
-       wre = new WrongRegionException("Request Region Name, " +
-        name + ",  does not match this region, " +
-        regionInfo.getRegionNameAsString());
-        if (LOG.isTraceEnabled()) LOG.trace("TrxRegionEndpoint coprocessor:beginTransaction threw WrongRegionException" +
-      "Request Region Name, " +
-        name + ",  does not match this region, " +
-        regionInfo.getRegionNameAsString());
-     }else 
-    */
     {
       if (memoryThrottle == true) {
         if(memoryUsageWarnOnly == true)  {
-          LOG.warn("TrxRegionEndpoint coprocessor: beginTransaction - performing memoryPercentage " + memoryPercentage + ", warning memory usage exceeds indicated percentage");
+          LOG.warn("beginTransaction - performing memoryPercentage " + memoryPercentage + ", warning memory usage exceeds indicated percentage");
         }
         else {
-          if (LOG.isTraceEnabled()) LOG.trace("TrxRegionEndpoint coprocessor: beginTransaction - performing memoryPercentage " + memoryPercentage + ", generating memory usage exceeds indicated percentage");
+          if (LOG.isTraceEnabled()) LOG.trace("beginTransaction - performing memoryPercentage " + memoryPercentage + ", generating memory usage exceeds indicated percentage");
           mue = new MemoryUsageException("beginTransaction memory usage exceeds " + memoryUsageThreshold + " percent, trxId is " + transactionId);
         }
       }
       else
       {
         try {
-          beginTransaction(transactionId);
+          beginTransaction(transactionId, startId);
         } catch (Throwable e) {
            if (LOG.isWarnEnabled()) LOG.warn("beginTransaction - txId "
                  + transactionId + ", Caught exception ", e);
@@ -669,15 +654,9 @@ CoprocessorService, Coprocessor {
       beginTransactionResponseBuilder.setException(t.toString());
     }
 
-    if (wre != null)
-    {
-      beginTransactionResponseBuilder.setHasException(true);
-      beginTransactionResponseBuilder.setException(wre.toString());
-    }
-
     if (mue != null)
     {
-      if (LOG.isTraceEnabled()) LOG.trace("TrxRegionEndpoint coprocessor: beginTransaction - performing memoryPercentage " + memoryPercentage + ", posting memory usage exceeds indicated percentage");
+      if (LOG.isTraceEnabled()) LOG.trace("beginTransaction - performing memoryPercentage " + memoryPercentage + ", posting memory usage exceeds indicated percentage");
       beginTransactionResponseBuilder.setHasException(true);
       beginTransactionResponseBuilder.setException(mue.toString());
     }
@@ -694,16 +673,17 @@ CoprocessorService, Coprocessor {
     CommitResponse response = CommitResponse.getDefaultInstance();
 
     Throwable t = null;
-    WrongRegionException wre = null;
     long transactionId = request.getTransactionId();
+    long commitId = request.getCommitId();
     final int participantNum = request.getParticipantNum();
 
-    if (LOG.isDebugEnabled()) LOG.debug("TrxRegionEndpoint coprocessor: commit - txId "
-         + transactionId + ", participantNum " + participantNum + ", regionName " + regionInfo.getRegionNameAsString());
+    if (LOG.isDebugEnabled()) LOG.debug("commit - txId "
+         + transactionId + ", commitId " + commitId + ", participantNum " + participantNum
+         + ", region " + m_regionDetails);
 
     // Process local memory
     try {
-       commit(transactionId, participantNum, request.getIgnoreUnknownTransactionException());
+       commit(transactionId, commitId, participantNum, request.getIgnoreUnknownTransactionException());
     } catch (Throwable e) {
        if (LOG.isWarnEnabled()) LOG.warn("commit - txId " + transactionId
            + ", Caught exception ", e);
@@ -720,12 +700,6 @@ CoprocessorService, Coprocessor {
       commitResponseBuilder.setException(t.toString());
     }
 
-    if (wre != null)
-    {
-      commitResponseBuilder.setHasException(true);
-      commitResponseBuilder.setException(wre.toString());
-    }
-
     CommitResponse cresponse = commitResponseBuilder.build();
 
     done.run(cresponse);
@@ -738,14 +712,14 @@ CoprocessorService, Coprocessor {
     CommitMultipleResponse response = CommitMultipleResponse.getDefaultInstance();
 
     Throwable t = null;
-    WrongRegionException wre = null;
     long transactionId = request.getTransactionId();
+    long commitId = request.getCommitId();
     int i = 0;
     int numOfRegion = request.getRegionNameCount();
     String requestRegionName;
 
-    if (LOG.isTraceEnabled()) LOG.trace("TrxRegionEndpoint coprocessor: commitMultiple - txId " + transactionId + " master regionName " + regionInfo.getRegionNameAsString());
-    if (LOG.isTraceEnabled()) LOG.trace("TrxRegionEndpoint coprocessor: commitMultiple - txId " + transactionId + " number of region is commitMultiple " + numOfRegion);
+    if (LOG.isTraceEnabled()) LOG.trace("commitMultiple - txId " + transactionId + " master regionName " + regionInfo.getRegionNameAsString());
+    if (LOG.isTraceEnabled()) LOG.trace("commitMultiple - txId " + transactionId + " number of region is commitMultiple " + numOfRegion);
 
     org.apache.hadoop.hbase.coprocessor.transactional.generated.TrxRegionProtos.CommitMultipleResponse.Builder commitMultipleResponseBuilder = CommitMultipleResponse.newBuilder();
     commitMultipleResponseBuilder.setHasException(false);
@@ -763,7 +737,7 @@ CoprocessorService, Coprocessor {
                  commitMultipleResponseBuilder.setException(i, BatchException.EXCEPTION_REGIONNOTFOUND_ERR.toString());
               }
               else {
-                 regionEPCP.commit(transactionId, i, request.getIgnoreUnknownTransactionException());
+                 regionEPCP.commit(transactionId, commitId, i, request.getIgnoreUnknownTransactionException());
               }
               if (LOG.isTraceEnabled()) LOG.trace("TrxRegionEndpoint commitMultiple ends");
              //commit(transactionId, request.getIgnoreUnknownTransactionException());
@@ -779,12 +753,6 @@ CoprocessorService, Coprocessor {
               commitMultipleResponseBuilder.setException(i, t.toString());
          }
 
-         if (wre != null)
-         {
-              commitMultipleResponseBuilder.setHasException(true);
-              commitMultipleResponseBuilder.setException(i, wre.toString());
-         }
-
          i++; // move to next region 
 
     } // end of while-loop on all the regions in thecommitMultiple request
@@ -806,11 +774,10 @@ CoprocessorService, Coprocessor {
     long startEpoch = request.getStartEpoch();
     final int participantNum = request.getParticipantNum();
     Throwable t = null;
-    WrongRegionException wre = null;
 
     // Process local memory
     try {
-       if (LOG.isDebugEnabled()) LOG.debug("commitIfPossible - txId "  + transactionId + ", regionName, " + regionInfo.getRegionNameAsString() + "calling internal commitIfPossible");
+       if (LOG.isDebugEnabled()) LOG.debug("commitIfPossible - txId "  + transactionId + ", regionName, " + m_regionDetails + "calling internal commitIfPossible");
        reply = commitIfPossible(transactionId, startEpoch, commitId, participantNum);
     } catch (Throwable e) {
        if (LOG.isWarnEnabled()) LOG.warn("commitIfPossible - txId " + transactionId
@@ -828,12 +795,6 @@ CoprocessorService, Coprocessor {
       commitIfPossibleResponseBuilder.setException(t.toString());
     }
 
-    if (wre != null)
-    {
-      commitIfPossibleResponseBuilder.setHasException(true);
-      commitIfPossibleResponseBuilder.setException(wre.toString());
-    }
-
     CommitIfPossibleResponse cresponse = commitIfPossibleResponseBuilder.build();
     done.run(cresponse);
   }
@@ -850,7 +811,6 @@ CoprocessorService, Coprocessor {
     UnknownTransactionException ute = null;
     CommitConflictException cce = null;
     Throwable t = null;
-    WrongRegionException wre = null;
     long transactionId = request.getTransactionId();
     long startEpoch = request.getStartEpoch();
     int participantNum = request.getParticipantNum();
@@ -858,13 +818,13 @@ CoprocessorService, Coprocessor {
 
     if (LOG.isTraceEnabled()) LOG.trace("commitRequest - txId "
          + transactionId + ", startEpoch " + startEpoch + ", participantNum " + participantNum + ", dropTableRecorded " + dropTableRecorded + 
-         ", regionName " + regionInfo.getRegionNameAsString());
+         ", regionName " + m_regionDetails);
 
     // Process local memory
     try {
       status = commitRequest(transactionId, startEpoch, participantNum, dropTableRecorded);
     } catch (UnknownTransactionException u) {
-       if (LOG.isTraceEnabled()) LOG.trace("TrxRegionEndpoint coprocessor: commitRequest - txId " + transactionId + ", Caught UnknownTransactionException after internal commitRequest call - " + u.toString());
+       LOG.info("TrxRegionEndpoint coprocessor: commitRequest - txId " + transactionId + ", Caught UnknownTransactionException after internal commitRequest call in region " + m_regionDetails, u);
        ute = u;
        status = COMMIT_UNSUCCESSFUL_FROM_COPROCESSOR;
     } catch (CommitConflictException c) {
@@ -872,7 +832,8 @@ CoprocessorService, Coprocessor {
        cce = c;
        status = COMMIT_CONFLICT;
     } catch (IOException e) {
-       if (LOG.isTraceEnabled()) LOG.trace("TrxRegionEndpoint coprocessor: commitRequest - txId " + transactionId + ", Caught IOException after internal commitRequest call - "+ e.toString());
+       LOG.error("TrxRegionEndpoint coprocessor: commitRequest - txId " + transactionId + " participant " + participantNum
+         +" , Caught IOException after internal commitRequest call - ", e);
        ioe = e;
        status = COMMIT_UNSUCCESSFUL_FROM_COPROCESSOR;
     }
@@ -893,12 +854,6 @@ CoprocessorService, Coprocessor {
       commitRequestResponseBuilder.setException(cce.toString());
     }
 
-    if (wre != null)
-    {
-      commitRequestResponseBuilder.setHasException(true);
-      commitRequestResponseBuilder.setException(wre.toString());
-    }
-
     if (ioe != null)
     {
       commitRequestResponseBuilder.setHasException(true);
@@ -928,7 +883,6 @@ CoprocessorService, Coprocessor {
     IOException ioe = null;
     UnknownTransactionException ute = null;
     Throwable t = null;
-    WrongRegionException wre = null;
     long transactionId = request.getTransactionId();
     long startEpoch = request.getStartEpoch();
     int i = 0;
@@ -945,20 +899,20 @@ CoprocessorService, Coprocessor {
          requestRegionName = request.getRegionName(i).toStringUtf8();    
 /*
          if (LOG.isTraceEnabled()) LOG.trace("EPCP AA0 Region Key " + Hex.encodeHexString(request.getRegionName(i).toStringUtf8().getBytes()));
-         if (LOG.isTraceEnabled()) LOG.trace("EPCP AA0 Region Key " + this.m_Region.getRegionNameAsString());
+         if (LOG.isTraceEnabled()) LOG.trace("EPCP AA0 Region Key " + regionInfo.getRegionNameAsString());
          if (LOG.isTraceEnabled()) LOG.trace("EPCP AA0 Region Key " + this.m_Region.getRegionName().toString());
          if (LOG.isTraceEnabled()) LOG.trace("EPCP AA0 Region Key " + Hex.encodeHexString(ByteString.copyFrom(this.m_Region.getRegionName()).toString().getBytes()));
          if (LOG.isTraceEnabled()) LOG.trace("EPCP AA1 Region Key " + Hex.encodeHexString(requestRegionName.getBytes()));
-         if (LOG.isTraceEnabled()) LOG.trace("EPCP AA2 Region Key " + Hex.encodeHexString(this.m_Region.getRegionNameAsString().getBytes()));
+         if (LOG.isTraceEnabled()) LOG.trace("EPCP AA2 Region Key " + Hex.encodeHexString(regionInfo.getRegionNameAsString().getBytes()));
          if (LOG.isTraceEnabled()) LOG.trace("EPCP AA2 Region Key " + Hex.encodeHexString(this.m_Region.getRegionName()));
 
          if (requestRegionName.equals(ByteString.copyFrom(this.m_Region.getRegionName()).toString())) {
             if (LOG.isTraceEnabled()) { LOG.trace("EPCP BB0 Region Key matches !! " + request.getRegionName(i).toString()); }
          }
-         if (Arrays.equals(request.getRegionName(i).toStringUtf8().getBytes(), this.m_Region.getRegionNameAsString().getBytes())) {
+         if (Arrays.equals(request.getRegionName(i).toStringUtf8().getBytes(), regionInfo.getRegionNameAsString().getBytes())) {
             if (LOG.isTraceEnabled()) { LOG.trace("EPCP BB1 Region Key matches !! " + request.getRegionName(i).toString()); }
          }
-         if (request.getRegionName(i).toStringUtf8().equals(this.m_Region.getRegionNameAsString())) {
+         if (request.getRegionName(i).toStringUtf8().equals(regionInfo.getRegionNameAsString())) {
             if (LOG.isTraceEnabled()) { LOG.trace("EPCP BB2 Region Key matches !! " + request.getRegionName(i).toString()); }
          }
 */         
@@ -979,10 +933,11 @@ CoprocessorService, Coprocessor {
               if (LOG.isTraceEnabled()) LOG.trace("TrxRegionEndpoint commitRequestMultiple ends");
              //status = commitRequest(transactionId);
          } catch (UnknownTransactionException u) {
-              if (LOG.isTraceEnabled()) LOG.trace("TrxRegionEndpoint coprocessor: commitRequestMultiple - txId " + transactionId + ", Caught UnknownTransactionException after internal commitRequest call - " + u.toString());
+              if (LOG.isTraceEnabled()) LOG.trace("commitRequestMultiple - txId " + transactionId + ", Caught UnknownTransactionException after internal commitRequest call - " + u.toString());
               ute = u;
          } catch (IOException e) {
-              if (LOG.isTraceEnabled()) LOG.trace("TrxRegionEndpoint coprocessor: commitRequestMultiple - txId " + transactionId + ", Caught IOException after internal commitRequest call - "+ e.toString());
+              LOG.error("commitRequestMultiple - txId " + transactionId
+                  + ", Caught IOException after internal commitRequest call - ", e);
               ioe = e;
          }
 
@@ -992,12 +947,6 @@ CoprocessorService, Coprocessor {
               commitRequestMultipleResponseBuilder.setException(i, BatchException.EXCEPTION_SKIPREMAININGREGIONS_OK.toString());
          }
 
-         if (wre != null)
-         {
-              commitRequestMultipleResponseBuilder.setHasException(true);
-              commitRequestMultipleResponseBuilder.setException(i, BatchException.EXCEPTION_SKIPREMAININGREGIONS_OK.toString());
-         }
-
          if (ioe != null)
          {
               commitRequestMultipleResponseBuilder.setHasException(true);
@@ -1033,9 +982,9 @@ CoprocessorService, Coprocessor {
     Delete delete = null;
     Throwable t = null;
     MemoryUsageException mue = null;
-    WrongRegionException wre = null;
     boolean result = false;
     long tid = request.getTid();
+    long commitId = request.getCommitId();
     boolean autoCommit = request.getAutoCommit();
 
     java.lang.String name = ((com.google.protobuf.ByteString) request.getRegionName()).toStringUtf8();
@@ -1056,7 +1005,6 @@ CoprocessorService, Coprocessor {
     }
 
     if (mue == null && 
-        wre == null && 
         type == MutationType.DELETE && 
         proto.hasRow()){
       try {
@@ -1079,7 +1027,8 @@ CoprocessorService, Coprocessor {
            if (t == null) {
 
              try {
-               result = checkAndDeleteRegionTx(tid,
+               result = checkAndDeleteRegionTx(tid, tid,
+                commitId,
                 request.getRow().toByteArray(),
                 request.getFamily().toByteArray(),
                 request.getQualifier().toByteArray(),
@@ -1110,11 +1059,6 @@ CoprocessorService, Coprocessor {
        checkAndDeleteRegionTxResponseBuilder.setException(t.toString());
      }
 
-     if (wre != null){
-       checkAndDeleteRegionTxResponseBuilder.setHasException(true);
-       checkAndDeleteRegionTxResponseBuilder.setException(wre.toString());
-     }
-
      if (mue != null){
        if (LOG.isWarnEnabled()) LOG.warn("checkAndDeleteRegionTx - performing memoryPercentage " + memoryPercentage
         + ", posting memory usage exceeds indicated percentage");
@@ -1140,41 +1084,28 @@ CoprocessorService, Coprocessor {
     Delete delete = null;
     Throwable t = null;
     MemoryUsageException mue = null;
-    WrongRegionException wre = null;
     boolean result = false;
     long transactionId = request.getTransactionId();
+    long startId = request.getStartId();
+
 
     java.lang.String name = ((com.google.protobuf.ByteString) request.getRegionName()).toStringUtf8();
 
     // First test if this region matches our region name
 
-    /* commenting it out for the time-being
-
-    if (!name.equals(regionInfo.getRegionNameAsString())) {
-       wre = new WrongRegionException("Request Region Name, " +
-        name + ",  does not match this region, " +
-        regionInfo.getRegionNameAsString());
-        if (LOG.isTraceEnabled()) LOG.trace("TrxRegionEndpoint coprocessor:checkAndDelete threw WrongRegionException" +
-      "Request Region Name, " +
-        name + ",  does not match this region, " +
-        regionInfo.getRegionNameAsString());
-   }
-    */
-
     org.apache.hadoop.hbase.coprocessor.transactional.generated.TrxRegionProtos.CheckAndDeleteResponse.Builder checkAndDeleteResponseBuilder = CheckAndDeleteResponse.newBuilder();
 
       if (memoryThrottle == true) {
         if(memoryUsageWarnOnly == true)  {
-          LOG.warn("TrxRegionEndpoint coprocessor: checkAndDelete - performing memoryPercentage " + memoryPercentage + ", warning memory usage exceeds indicated percentage");
+          LOG.warn("checkAndDelete - performing memoryPercentage " + memoryPercentage + ", warning memory usage exceeds indicated percentage");
         }
         else {
-          if (LOG.isTraceEnabled()) LOG.trace("TrxRegionEndpoint coprocessor: checkAndDelete - performing memoryPercentage " + memoryPercentage + ", generating memory usage exceeds indicated percentage");
+          if (LOG.isTraceEnabled()) LOG.trace("checkAndDelete - performing memoryPercentage " + memoryPercentage + ", generating memory usage exceeds indicated percentage");
           mue = new MemoryUsageException("checkAndDelete memory usage exceeds " + memoryUsageThreshold + " percent, trxId is " + transactionId);
         }
       }
 
     if (mue == null && 
-        wre == null && 
         type == MutationType.DELETE && 
         proto.hasRow())
     {
@@ -1200,6 +1131,7 @@ CoprocessorService, Coprocessor {
       
           try {
            result = checkAndDelete(transactionId,
+               startId,
                request.getRow().toByteArray(),
                request.getFamily().toByteArray(),
                request.getQualifier().toByteArray(),
@@ -1221,7 +1153,7 @@ CoprocessorService, Coprocessor {
       checkAndDeleteResponseBuilder.setResult(result);
     }
 
-    if (LOG.isTraceEnabled()) LOG.trace("TrxRegionEndpoint coprocessor: checkAndDelete - txId " + transactionId + ", result is " + result);
+    if (LOG.isTraceEnabled()) LOG.trace("checkAndDelete - txId " + transactionId + ", result is " + result);
 
     checkAndDeleteResponseBuilder.setHasException(false);
 
@@ -1231,15 +1163,9 @@ CoprocessorService, Coprocessor {
       checkAndDeleteResponseBuilder.setException(t.toString());
     }
 
-    if (wre != null)
-    {
-      checkAndDeleteResponseBuilder.setHasException(true);
-      checkAndDeleteResponseBuilder.setException(wre.toString());
-    }
-
     if (mue != null)
     {
-      if (LOG.isTraceEnabled()) LOG.trace("TrxRegionEndpoint coprocessor: checkAndDelete - performing memoryPercentage " + memoryPercentage + ", posting memory usage exceeds indicated percentage");
+      if (LOG.isTraceEnabled()) LOG.trace("checkAndDelete - performing memoryPercentage " + memoryPercentage + ", posting memory usage exceeds indicated percentage");
       checkAndDeleteResponseBuilder.setHasException(true);
       checkAndDeleteResponseBuilder.setException(mue.toString());
     }
@@ -1265,40 +1191,24 @@ CoprocessorService, Coprocessor {
     MutationType type = proto.getMutateType();
     Put put = null;
     MemoryUsageException mue = null;
-    WrongRegionException wre = null;
     Throwable t = null;
     boolean result = false;
     long transactionId = request.getTransactionId();
-
-    /* commenting it out for the time-being
-    java.lang.String name = ((com.google.protobuf.ByteString) request.getRegionName()).toStringUtf8();
-
-    // First test if this region matches our region name
-    if (!name.equals(regionInfo.getRegionNameAsString())) {
-      wre =  new WrongRegionException("Request Region Name, " +
-        name + ",  does not match this region, " +
-        regionInfo.getRegionNameAsString());
-        if (LOG.isTraceEnabled()) LOG.trace("TrxRegionEndpoint coprocessor:checkAndPut threw WrongRegionException" +
-      "Request Region Name, " +
-        name + ",  does not match this region, " +
-        regionInfo.getRegionNameAsString());
-   }
-    */
+    long startId = request.getStartId();
 
     org.apache.hadoop.hbase.coprocessor.transactional.generated.TrxRegionProtos.CheckAndPutResponse.Builder checkAndPutResponseBuilder = CheckAndPutResponse.newBuilder();
 
     if (memoryThrottle == true) {
         if(memoryUsageWarnOnly == true)  {
-          LOG.warn("TrxRegionEndpoint coprocessor: checkAndPut - performing memoryPercentage " + memoryPercentage + ", warning memory usage exceeds indicated percentage");
+          LOG.warn("checkAndPut - performing memoryPercentage " + memoryPercentage + ", warning memory usage exceeds indicated percentage");
         }
         else {
           mue = new MemoryUsageException("checkAndPut memory usage exceeds " + memoryUsageThreshold + " percent, trxId is " + transactionId);
-          if (LOG.isTraceEnabled()) LOG.trace("TrxRegionEndpoint coprocessor: checkAndPut - performing memoryPercentage " + memoryPercentage + ", generating memory usage exceeds indicated percentage exception");
+          if (LOG.isTraceEnabled()) LOG.trace("checkAndPut - performing memoryPercentage " + memoryPercentage + ", generating memory usage exceeds indicated percentage exception");
         }
     }
 
     if (mue == null &&
-        wre == null && 
         type == MutationType.PUT && 
         proto.hasRow())
     {
@@ -1335,6 +1245,7 @@ CoprocessorService, Coprocessor {
       
           try {
            result = checkAndPut(transactionId,
+               startId,
                request.getRow().toByteArray(),
                request.getFamily().toByteArray(),
                request.getQualifier().toByteArray(),
@@ -1356,16 +1267,10 @@ CoprocessorService, Coprocessor {
       checkAndPutResponseBuilder.setResult(result);
     }
 
-    if (LOG.isTraceEnabled()) LOG.trace("TrxRegionEndpoint coprocessor: checkAndPut - txId " + transactionId + ", result is " + result);
+    if (LOG.isTraceEnabled()) LOG.trace("checkAndPut - txId " + transactionId + ", result is " + result);
 
     checkAndPutResponseBuilder.setHasException(false);
 
-    if (wre != null)
-    {
-      checkAndPutResponseBuilder.setHasException(true);
-      checkAndPutResponseBuilder.setException(wre.toString());
-    }
-
     if (t != null)
     {
       checkAndPutResponseBuilder.setHasException(true);
@@ -1374,7 +1279,7 @@ CoprocessorService, Coprocessor {
 
     if (mue != null)
     {
-      if (LOG.isTraceEnabled()) LOG.trace("TrxRegionEndpoint coprocessor: checkAndPut - performing memoryPercentage " + memoryPercentage + ", posting memory usage exceeds indicated percentage exception");
+      if (LOG.isTraceEnabled()) LOG.trace("checkAndPut - performing memoryPercentage " + memoryPercentage + ", posting memory usage exceeds indicated percentage exception");
       checkAndPutResponseBuilder.setHasException(true);
       checkAndPutResponseBuilder.setException(mue.toString());
     }
@@ -1397,14 +1302,15 @@ CoprocessorService, Coprocessor {
     com.google.protobuf.ByteString qualifier = null;
     com.google.protobuf.ByteString value = null;
     long tid = request.getTid();
+    long commitId = request.getCommitId();
     MutationProto proto = request.getPut();
     MutationType type = proto.getMutateType();
     boolean autoCommit = request.getAutoCommit();
     Put put = null;
     MemoryUsageException mue = null;
-    WrongRegionException wre = null;
     Throwable t = null;
     boolean result = false;
+    long startId = tid;
 
     org.apache.hadoop.hbase.coprocessor.transactional.generated.TrxRegionProtos.CheckAndPutRegionTxResponse.Builder checkAndPutRegionTxResponseBuilder = CheckAndPutRegionTxResponse.newBuilder();
 
@@ -1419,7 +1325,6 @@ CoprocessorService, Coprocessor {
     }
 
     if (mue == null &&
-        wre == null && 
         type == MutationType.PUT && 
         proto.hasRow())
     {
@@ -1456,6 +1361,7 @@ CoprocessorService, Coprocessor {
       
           try {
            result = checkAndPutRegionTx(tid,
+               startId, commitId,
                request.getRow().toByteArray(),
                request.getFamily().toByteArray(),
                request.getQualifier().toByteArray(),
@@ -1482,12 +1388,6 @@ CoprocessorService, Coprocessor {
 
     checkAndPutRegionTxResponseBuilder.setHasException(false);
 
-    if (wre != null)
-    {
-      checkAndPutRegionTxResponseBuilder.setHasException(true);
-      checkAndPutRegionTxResponseBuilder.setException(wre.toString());
-    }
-
     if (t != null)
     {
       checkAndPutRegionTxResponseBuilder.setHasException(true);
@@ -1514,27 +1414,11 @@ CoprocessorService, Coprocessor {
     RegionScanner scanner = null;
     Throwable t = null;
     OutOfOrderProtocolException oop = null;
-    WrongRegionException wre = null;
     Exception ce = null;
     long transId = request.getTransactionId();
     long scannerId = request.getScannerId();
 
-    if (LOG.isTraceEnabled()) LOG.trace("TrxRegionEndpoint coprocessor: closeScanner - txId " + transId + ", scanner id " + scannerId + ", regionName " + regionInfo.getRegionNameAsString());
-
-    /* commenting it out for the time-being
-    java.lang.String name = ((com.google.protobuf.ByteString) request.getRegionName()).toStringUtf8();
-
-    // First test if this region matches our region name
-    if (!name.equals(regionInfo.getRegionNameAsString())) {
-       wre = new WrongRegionException("Request Region Name, " +
-        name + ",  does not match this region, " +
-        regionInfo.getRegionNameAsString());
-        if (LOG.isTraceEnabled()) LOG.trace("TrxRegionEndpoint coprocessor: closeScanner threw WrongRegionException" +
-      "Request Region Name, " +
-        name + ",  does not match this region, " +
-        regionInfo.getRegionNameAsString());
-    } else {
-    */
+    if (LOG.isTraceEnabled()) LOG.trace("closeScanner - txId " + transId + ", scanner id " + scannerId + ", regionName " + m_regionDetails);
 
     // There should be a matching key in the transactionsById map
     // associated with this transaction id.  If there is not
@@ -1547,16 +1431,15 @@ CoprocessorService, Coprocessor {
 
     if (keyFound != true)
     {
-      if (LOG.isTraceEnabled()) LOG.trace("TrxRegionEndpoint coprocessor: closeScanner - Unknown transaction [" + transId
-             + "] in region ["
-             + m_Region.getRegionInfo().getRegionNameAsString()
+      if (LOG.isTraceEnabled()) LOG.trace("closeScanner - Unknown transaction [" + transId
+             + "] in region [" + m_regionDetails
              + "], will create an OutOfOrderProtocol exception ");
       oop = new OutOfOrderProtocolException("closeScanner does not have an active transaction with an open scanner, txId: " + transId);
     }
 
     if (oop == null) {
       try {
-         // we want to allow closing scaners and remove operations up until the very end.
+         // we want to allow closing scanners and remove operations up until the very end.
          checkBlockAll(transId);
          scanner = removeScanner(scannerId);
 
@@ -1564,22 +1447,22 @@ CoprocessorService, Coprocessor {
              scanner.close();
          }
          else
-           if (LOG.isTraceEnabled()) LOG.trace("TrxRegionEndpoint coprocessor: closeScanner - txId " + transId + ", scanner was null for scanner id " + scannerId);
+           if (LOG.isTraceEnabled()) LOG.trace("closeScanner - txId " + transId + ", scanner was null for scanner id " + scannerId);
 
 /*
          try {
            scannerLeases.cancelLease(getScannerLeaseId(scannerId));
          } catch (LeaseException le) {
            // ignore
-           if (LOG.isTraceEnabled()) LOG.trace("TrxRegionEndpoint coprocessor: closeScanner failed to get a lease " + scannerId);
+           if (LOG.isTraceEnabled()) LOG.trace("closeScanner failed to get a lease " + scannerId);
          }
 */
 
       } catch(Exception e) {
-        if (LOG.isTraceEnabled()) LOG.trace("TrxRegionEndpoint coprocessor: closeScanner - txId " + transId + ", Caught exception " + e.getMessage() + " " + stackTraceToString(e));
+        if (LOG.isTraceEnabled()) LOG.trace("closeScanner - txId " + transId + ", Caught exception " + e.getMessage() + " " + stackTraceToString(e));
         ce = e;
       } catch(Throwable e) {
-         if (LOG.isTraceEnabled()) LOG.trace("TrxRegionEndpoint coprocessor: closeScanner - txId " + transId + ", Caught exception " + e.getMessage() + " " + stackTraceToString(e));
+         if (LOG.isTraceEnabled()) LOG.trace("closeScanner - txId " + transId + ", Caught exception " + e.getMessage() + " " + stackTraceToString(e));
          t = e;
       }
     }
@@ -1594,12 +1477,6 @@ CoprocessorService, Coprocessor {
       closeResponseBuilder.setException(t.toString());
     }
 
-    if (wre != null)
-    {
-      closeResponseBuilder.setHasException(true);
-      closeResponseBuilder.setException(wre.toString());
-    }
-
     if (ce != null)
     {
       closeResponseBuilder.setHasException(true);
@@ -1612,10 +1489,10 @@ CoprocessorService, Coprocessor {
       {
         closeResponseBuilder.setHasException(true);
         closeResponseBuilder.setException(oop.toString());
-        LOG.warn("TrxRegionEndpoint coprocessor: closeScanner - OutOfOrderProtocolException, transaction was not found, txId: " + transId + ",returned exception" + ", regionName " + regionInfo.getRegionNameAsString());
+        LOG.warn("closeScanner - OutOfOrderProtocolException, transaction was not found, txId: " + transId + ",returned exception" + ", region " + m_regionDetails);
       }
       else
-        LOG.warn("TrxRegionEndpoint coprocessor: closeScanner - suppressing OutOfOrderProtocolException, transaction was not found, txId: " + transId + ", regionName " + regionInfo.getRegionNameAsString());
+        LOG.warn("closeScanner - suppressing OutOfOrderProtocolException, transaction was not found, txId: " + transId + ", region " + m_regionDetails);
     }
 
     CloseScannerResponse cresponse = closeResponseBuilder.build();
@@ -1636,34 +1513,20 @@ CoprocessorService, Coprocessor {
    MutationType type;
    Throwable t = null;
    MemoryUsageException mue = null;
-   WrongRegionException wre = null;
    long transactionId = request.getTransactionId();
+   long startId = request.getStartId();
 
-    /* commenting it out for the time-being
-    java.lang.String name = ((com.google.protobuf.ByteString) request.getRegionName()).toStringUtf8();
-
-    // First test if this region matches our region name
-    if (!name.equals(regionInfo.getRegionNameAsString())) {
-       wre = new WrongRegionException("Request Region Name, " +
-        name + ",  does not match this region, " +
-        regionInfo.getRegionNameAsString());
-        if (LOG.isTraceEnabled()) LOG.trace("TrxRegionEndpoint coprocessor:deleteMultiple threw WrongRegionException" +
-      "Request Region Name, " +
-        name + ",  does not match this region, " +
-        regionInfo.getRegionNameAsString());
-   } 
-    */
    if (memoryThrottle == true) {
         if(memoryUsageWarnOnly == true)  {
-          LOG.warn("TrxRegionEndpoint coprocessor: deleteMultiple - performing memoryPercentage " + memoryPercentage + ", warning memory usage exceeds indicated percentage");
+          LOG.warn("deleteMultiple - performing memoryPercentage " + memoryPercentage + ", warning memory usage exceeds indicated percentage");
         }
         else {
-          if (LOG.isTraceEnabled()) LOG.trace("TrxRegionEndpoint coprocessor: deleteMultiple - performing memoryPercentage " + memoryPercentage + ", generating memory usage exceeds indicated percentage exception");
+          if (LOG.isTraceEnabled()) LOG.trace("deleteMultiple - performing memoryPercentage " + memoryPercentage + ", generating memory usage exceeds indicated percentage exception");
           mue = new MemoryUsageException("deleteMultiple memory usage exceeds " + memoryUsageThreshold + " percent, trxId is " + transactionId);
        }
    }
 
-   if (mue == null && wre == null) {
+   if (mue == null) {
      for (org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto proto : results)
      { 
        delete = null;
@@ -1687,19 +1550,19 @@ CoprocessorService, Coprocessor {
            if ((delete != null) && (t == null))
            {
              try {
-               delete(transactionId, delete);
+               delete(transactionId, startId, delete);
              } catch (Throwable e) {
                if (LOG.isWarnEnabled()) LOG.warn("deleteMultiple - txId " + transactionId
                      + ", Caught exception ", e);
              t = e;
              }
 
-             if (LOG.isTraceEnabled()) LOG.trace("TrxRegionEndpoint coprocessor: deleteMultiple - txId "  + transactionId + ", regionName " + regionInfo.getRegionNameAsString() + ", type " + type + ", row " + Bytes.toStringBinary(proto.getRow().toByteArray()) + ", row in hex " + Hex.encodeHexString(proto.getRow().toByteArray()));
+             if (LOG.isTraceEnabled()) LOG.trace("deleteMultiple - txId "  + transactionId + ", region " + m_regionDetails + ", type " + type + ", row " + Bytes.toStringBinary(proto.getRow().toByteArray()) + ", row in hex " + Hex.encodeHexString(proto.getRow().toByteArray()));
            }
          }
        }
        else
-         if (LOG.isTraceEnabled()) LOG.trace("TrxRegionEndpoint coprocessor: deleteMultiple - txId "  + transactionId + ", regionName " + regionInfo.getRegionNameAsString() + ", delete proto was null");
+         if (LOG.isTraceEnabled()) LOG.trace("deleteMultiple - txId "  + transactionId + ", region " + m_regionDetails + ", delete proto was null");
 
       }
     }
@@ -1714,15 +1577,9 @@ CoprocessorService, Coprocessor {
       deleteMultipleTransactionalResponseBuilder.setException(t.toString());
     }
 
-    if (wre != null)
-    {
-      deleteMultipleTransactionalResponseBuilder.setHasException(true);
-      deleteMultipleTransactionalResponseBuilder.setException(wre.toString());
-    }
-
     if (mue != null)
     {
-      if (LOG.isTraceEnabled()) LOG.trace("TrxRegionEndpoint coprocessor: deleteMultiple - performing memoryPercentage " + memoryPercentage + ", posting memory usage exceeds indicated percentage");
+      if (LOG.isTraceEnabled()) LOG.trace("deleteMultiple - performing memoryPercentage " + memoryPercentage + ", posting memory usage exceeds indicated percentage");
       deleteMultipleTransactionalResponseBuilder.setHasException(true);
       deleteMultipleTransactionalResponseBuilder.setException(mue.toString());
     }
@@ -1744,8 +1601,8 @@ CoprocessorService, Coprocessor {
     Boolean autoCommit = request.getAutoCommit();
     Throwable t = null;
     MemoryUsageException mue = null;
-    WrongRegionException wre = null;
     long tid = request.getTid();
+    long commitId = request.getCommitId();
 
     if (memoryThrottle == true) {
         if(memoryUsageWarnOnly == true)  {
@@ -1770,7 +1627,7 @@ CoprocessorService, Coprocessor {
       // Process in local memory
       if ((delete != null) && (t == null)){
         try {
-           deleteRegionTx(tid, delete, autoCommit);
+           deleteRegionTx(tid, tid, commitId, delete, autoCommit);
         } catch (Throwable e) {
            if (LOG.isWarnEnabled()) LOG.warn("deleteRegionTx - tid " + tid
                 + ", Caught exception after internal deleteRegionTx - ", e);
@@ -1778,7 +1635,7 @@ CoprocessorService, Coprocessor {
         }
 
         if (LOG.isTraceEnabled()) LOG.trace("deleteRegionTx - tid "  + tid + ", regionName "
-           + regionInfo.getRegionNameAsString() + ", type " + type + ", row "
+           + m_regionDetails + ", type " + type + ", row "
            + Bytes.toStringBinary(proto.getRow().toByteArray()) + ", row in hex "
            + Hex.encodeHexString(proto.getRow().toByteArray()));
       }
@@ -1798,12 +1655,6 @@ CoprocessorService, Coprocessor {
       deleteRegionTxResponseBuilder.setException(t.toString());
     }
 
-    if (wre != null)
-    {
-      deleteRegionTxResponseBuilder.setHasException(true);
-      deleteRegionTxResponseBuilder.setException(wre.toString());
-    }
-
     if (mue != null)
     {
       if (LOG.isTraceEnabled()) LOG.trace("deleteRegionTx - performing memoryPercentage "
@@ -1828,30 +1679,15 @@ CoprocessorService, Coprocessor {
     Delete delete = null;
     Throwable t = null;
     MemoryUsageException mue = null;
-    WrongRegionException wre = null;
     long transactionId = request.getTransactionId();
-
-    /* commenting it out for the time-being
-    java.lang.String name = ((com.google.protobuf.ByteString) request.getRegionName()).toStringUtf8();
-
-    // First test if this region matches our region name
-    if (!name.equals(regionInfo.getRegionNameAsString())) {
-       wre = new WrongRegionException("Request Region Name, " +
-        name + ",  does not match this region, " +
-        regionInfo.getRegionNameAsString());
-        if (LOG.isTraceEnabled()) LOG.trace("TrxRegionEndpoint coprocessor:delete threw WrongRegionException" +
-      "Request Region Name, " +
-        name + ",  does not match this region, " +
-        regionInfo.getRegionNameAsString());
-    }
-    */
+    long startId = request.getStartId();
 
     if (memoryThrottle == true) {
         if(memoryUsageWarnOnly == true)  {
-          LOG.warn("TrxRegionEndpoint coprocessor: delete - performing memoryPercentage " + memoryPercentage + ", warning memory usage exceeds indicated percentage");
+          LOG.warn("delete - performing memoryPercentage " + memoryPercentage + ", warning memory usage exceeds indicated percentage");
         }
         else {
-          if (LOG.isTraceEnabled()) LOG.trace("TrxRegionEndpoint coprocessor: delete - performing memoryPercentage " + memoryPercentage + ", generating memory usage exceeds indicated percentage");
+          if (LOG.isTraceEnabled()) LOG.trace("delete - performing memoryPercentage " + memoryPercentage + ", generating memory usage exceeds indicated percentage");
           mue = new MemoryUsageException("delete memory usage exceeds " + memoryUsageThreshold + " percent, trxId is " + transactionId);
         }
     }
@@ -1870,14 +1706,14 @@ CoprocessorService, Coprocessor {
      if ((delete != null) && (t == null))
      {
           try {
-            delete(transactionId, delete);
+            delete(transactionId, startId, delete);
           } catch (Throwable e) {
             if (LOG.isWarnEnabled()) LOG.warn("delete - txId " + transactionId
                  + ", Caught exception after internal delete - ", e);
             t = e;
           }
 
-          if (LOG.isTraceEnabled()) LOG.trace("TrxRegionEndpoint coprocessor: delete - txId "  + transactionId + ", regionName " + regionInfo.getRegionNameAsString() + ", type " + type + ", row " + Bytes.toStringBinary(proto.getRow().toByteArray()) + ", row in hex " + Hex.encodeHexString(proto.getRow().toByteArray()));
+          if (LOG.isTraceEnabled()) LOG.trace("delete - txId "  + transactionId + ", region " + m_regionDetails + ", type " + type + ", row " + Bytes.toStringBinary(proto.getRow().toByteArray()) + ", row in hex " + Hex.encodeHexString(proto.getRow().toByteArray()));
         }
     }
     org.apache.hadoop.hbase.coprocessor.transactional.generated.TrxRegionProtos.DeleteTransactionalResponse.Builder deleteTransactionalResponseBuilder = DeleteTransactionalResponse.newBuilder();
@@ -1890,15 +1726,9 @@ CoprocessorService, Coprocessor {
       deleteTransactionalResponseBuilder.setException(t.toString());
     }
 
-    if (wre != null)
-    {
-      deleteTransactionalResponseBuilder.setHasException(true);
-      deleteTransactionalResponseBuilder.setException(wre.toString());
-    }
-
     if (mue != null)
     {
-      if (LOG.isTraceEnabled()) LOG.trace("TrxRegionEndpoint coprocessor: delete - performing memoryPercentage " + memoryPercentage + ", posting memory usage exceeds indicated percentage");
+      if (LOG.isTraceEnabled()) LOG.trace("delete - performing memoryPercentage " + memoryPercentage + ", posting memory usage exceeds indicated percentage");
       deleteTransactionalResponseBuilder.setHasException(true);
       deleteTransactionalResponseBuilder.setException(mue.toString());
     }
@@ -1920,30 +1750,16 @@ CoprocessorService, Coprocessor {
     Exception ge = null;
     IOException gioe = null;
     MemoryUsageException mue = null;
-    WrongRegionException wre = null;
     org.apache.hadoop.hbase.client.Result result2 = null;
     long transactionId = request.getTransactionId();
+    long startId = request.getStartId();
     boolean exceptionThrown = false;
 
-    /* commenting it out for the time-being
-    java.lang.String name = ((com.google.protobuf.ByteString) request.getRegionName()).toStringUtf8();
-
-    // First test if this region matches our region name
-    if (!name.equals(regionInfo.getRegionNameAsString())) {
-       wre = new WrongRegionException("Request Region Name, " +
-        name + ",  does not match this region, " +
-        regionInfo.getRegionNameAsString());
-        if (LOG.isTraceEnabled()) LOG.trace("TrxRegionEndpoint coprocessor:get threw WrongRegionException" +
-      "Request Region Name, " +
-        name + ",  does not match this region, " +
-        regionInfo.getRegionNameAsString());
-    } else { */
-
       if (memoryThrottle == true) {
         if(memoryUsageWarnOnly == true)  
-          LOG.warn("TrxRegionEndpoint coprocessor: get - performing memoryPercentage " + memoryPercentage + ", warning memory usage exceeds indicated percentage");
+          LOG.warn("get - performing memoryPercentage " + memoryPercentage + ", warning memory usage exceeds indicated percentage");
         else {
-          if (LOG.isTraceEnabled()) LOG.trace("TrxRegionEndpoint coprocessor: get - performing memoryPercentage " + memoryPercentage + ", generating memory usage exceeds indicated percentage exception");
+          if (LOG.isTraceEnabled()) LOG.trace("get - performing memoryPercentage " + memoryPercentage + ", generating memory usage exceeds indicated percentage exception");
           mue = new MemoryUsageException("get memory usage exceeds " + memoryUsageThreshold + " percent, trxId is " + transactionId);
           exceptionThrown = true;
         }
@@ -1973,17 +1789,20 @@ CoprocessorService, Coprocessor {
                 String rowKey = Bytes.toString(row);
                 String getRowKey = Bytes.toString(getrow);
 
-                LOG.trace("TrxRegionEndpoint coprocessor: get - txId " + transactionId + ", Calling getScanner for regionName " + regionInfo.getRegionNameAsString() + ", row = " + Bytes.toStringBinary(row) + ", row in hex " + Hex.encodeHexString(row) + ", getrow = " + Bytes.toStringBinary(getrow) + ", getrow in hex " + Hex.encodeHexString(getrow));
+                LOG.trace("get - txId " + transactionId +
+                          ", Calling getScanner for region " + m_regionDetails +
+                          ", row = " + Bytes.toStringBinary(row) + ", row in hex " + Hex.encodeHexString(row) +
+                          ", getrow = " + Bytes.toStringBinary(getrow) + ", getrow in hex " + Hex.encodeHexString(getrow));
               }
 
-              scanner = getScanner(transactionId, scan);
+              scanner = getScanner(transactionId, startId, scan);
 
               if (scanner != null)
                 scanner.next(results);
 
               result2 = Result.create(results);
   
-              if (LOG.isTraceEnabled()) LOG.trace("TrxRegionEndpoint coprocessor: get - txId " + transactionId + ", getScanner result2 isEmpty is " 
+              if (LOG.isTraceEnabled()) LOG.trace("get - txId " + transactionId + ", getScanner result2 isEmpty is "
 		   + result2.isEmpty() 
 		   + ", row " 
 		   + Bytes.toStringBinary(result2.getRow())
@@ -1991,7 +1810,7 @@ CoprocessorService, Coprocessor {
 		   + result2.size()); 
 
             } catch(Throwable e) {
-              if (LOG.isTraceEnabled()) LOG.trace("TrxRegionEndpoint coprocessor: get - txId " + transactionId + ", Caught exception " + e.getMessage() + " " + stackTraceToString(e));
+              if (LOG.isTraceEnabled()) LOG.trace("get - txId " + transactionId + ", Caught exception " + e.getMessage() + " " + stackTraceToString(e));
               t = e;
             }
             finally {
@@ -1999,14 +1818,13 @@ CoprocessorService, Coprocessor {
                 try {
                   scanner.close();
                 } catch(Exception e) {
-                  if (LOG.isTraceEnabled()) LOG.trace("TrxRegionEndpoint coprocessor: get - txId " + transactionId + ", Caught exception " + e.getMessage() + " " + stackTraceToString(e));
+                  if (LOG.isTraceEnabled()) LOG.trace("get - txId " + transactionId + ", Caught exception " + e.getMessage() + " " + stackTraceToString(e));
                   ge = e;
                 }
               }
             }
          } // ExceptionThrown
       } // End of MemoryUsageCheck
-  //}  // End of WrongRegionCheck
 
     org.apache.hadoop.hbase.coprocessor.transactional.generated.TrxRegionProtos.GetTransactionalResponse.Builder getResponseBuilder = GetTransactionalResponse.newBuilder();
 
@@ -2016,9 +1834,9 @@ CoprocessorService, Coprocessor {
    }
    else
    {
-     if (t == null && wre == null && ge == null)
-       gioe = new IOException("TrxRegionEndpoint coprocessor: get - result2 was null");
-     if (LOG.isTraceEnabled()) LOG.trace("TrxRegionEndpoint coprocessor: get - txId " + transactionId + ", result2 was null ");
+     if (t == null && ge == null)
+       gioe = new IOException("get - result2 was null");
+     if (LOG.isTraceEnabled()) LOG.trace("get - txId " + transactionId + ", result2 was null ");
    }
       
    getResponseBuilder.setHasException(false);
@@ -2028,12 +1846,6 @@ CoprocessorService, Coprocessor {
      getResponseBuilder.setHasException(true);
      getResponseBuilder.setException(t.toString());
    }
-      
-   if (wre != null)
-   {
-     getResponseBuilder.setHasException(true);
-     getResponseBuilder.setException(wre.toString());
-   }
 
    if (ge != null)
    {
@@ -2049,7 +1861,7 @@ CoprocessorService, Coprocessor {
 
    if (mue != null)
    {
-     if (LOG.isTraceEnabled()) LOG.trace("TrxRegionEndpoint coprocessor: get - performing memoryPercentage " + memoryPercentage + ", posting memory usage exceeds indicated percentage exception");
+     if (LOG.isTraceEnabled()) LOG.trace("get - performing memoryPercentage " + memoryPercentage + ", posting memory usage exceeds indicated percentage exception");
      getResponseBuilder.setHasException(true);
      getResponseBuilder.setException(mue.toString());
    }
@@ -2069,7 +1881,6 @@ CoprocessorService, Coprocessor {
     RegionScanner scannert = null;
     Throwable t = null;
     MemoryUsageException mue = null;
-    WrongRegionException wre = null;
     boolean exceptionThrown = false;
     NullPointerException npe = null;        
     Exception ge = null;
@@ -2079,31 +1890,18 @@ CoprocessorService, Coprocessor {
     long scannerId = 0L;
     boolean isLoadingCfsOnDemandSet = false;
     long transId = request.getTransactionId();
+    long startId = request.getStartId();
 
-    /* commenting it out for the time-being
-    java.lang.String name = ((com.google.protobuf.ByteString) request.getRegionName()).toStringUtf8();
+    if (LOG.isTraceEnabled()) LOG.trace("openScanner - ENTER txId " + transId + " in region " + m_regionDetails);
 
-    // First test if this region matches our region name
-    
-    if (!name.equals(regionInfo.getRegionNameAsString())) {
-       wre = new WrongRegionException("Request Region Name, " +
-        name + ",  does not match this region, " +
-        regionInfo.getRegionNameAsString());
-        if (LOG.isTraceEnabled()) LOG.trace("TrxRegionEndpoint coprocessor:openScanner threw WrongRegionException" +
-      "Request Region Name, " +
-        name + ",  does not match this region, " +
-        regionInfo.getRegionNameAsString());
-        exceptionThrown = true;
-    } else 
-    */
     {
     
       if (memoryThrottle == true) {
         if(memoryUsageWarnOnly == true)  {
-          LOG.warn("TrxRegionEndpoint coprocessor: openScanner - performing memoryPercentage " + memoryPercentage + ", warning memory usage exceeds indicated percentage");
+          LOG.warn("openScanner - performing memoryPercentage " + memoryPercentage + ", warning memory usage exceeds indicated percentage");
         }
         else {
-          if (LOG.isTraceEnabled()) LOG.trace("TrxRegionEndpoint coprocessor: openScanner - performing memoryPercentage " + memoryPercentage + ", generating memory usage exceeds indicated percentage");
+          if (LOG.isTraceEnabled()) LOG.trace("openScanner - performing memoryPercentage " + memoryPercentage + ", generating memory usage exceeds indicated percentage");
           exceptionThrown = true;
           mue = new MemoryUsageException("openScanner memory usage exceeds " + memoryUsageThreshold + " percent, trxId is " + transId);
         }
@@ -2113,7 +1911,7 @@ CoprocessorService, Coprocessor {
         try {
             scan = ProtobufUtil.toScan(request.getScan());
           if (scan == null)
-            if (LOG.isTraceEnabled()) LOG.trace("TrxRegionEndpoint coprocessor: openScanner - txId " + transId + ", scan was null");
+            if (LOG.isTraceEnabled()) LOG.trace("openScanner - txId " + transId + ", scan was null");
         } catch (Throwable e) {
           if (LOG.isWarnEnabled()) LOG.warn("openScanner - txId " + transId
              + ", Caught exception ", e);
@@ -2123,8 +1921,8 @@ CoprocessorService, Coprocessor {
 
         if (!exceptionThrown) {
           if (scan == null) {
-            if (LOG.isTraceEnabled()) LOG.trace("TrxRegionEndpoint coprocessor: openScanner - txId " + transId + ", scan is null");
-            npe = new NullPointerException("TrxRegionEndpoint coprocessor: openScanner - txId " + transId + ", scan is null ");
+            if (LOG.isTraceEnabled()) LOG.trace("openScanner - txId " + transId + ", scan is null");
+            npe = new NullPointerException("openScanner - txId " + transId + ", scan is null ");
             ioe =  new IOException("Invalid arguments to openScanner", npe);
             exceptionThrown = true;
           }
@@ -2146,36 +1944,36 @@ CoprocessorService, Coprocessor {
 
         if (!exceptionThrown) {
           try {
-            scanner = getScanner(transId, scan);
+            scanner = getScanner(transId, startId, scan);
         
             if (scanner != null) {
-              if (LOG.isTraceEnabled()) LOG.trace("TrxRegionEndpoint coprocessor: openScanner - txId " + transId + ", called getScanner, scanner is " + scanner);
+              if (LOG.isTraceEnabled()) LOG.trace("openScanner - txId " + transId + ", called getScanner, scanner is " + scanner);
               // Add the scanner to the map
               scannerId = addScanner(transId, scanner, this.m_Region);
-              if (LOG.isTraceEnabled()) LOG.trace("TrxRegionEndpoint coprocessor: openScanner - txId " + transId + ", called addScanner, scanner id " + scannerId + ", regionName " + regionInfo.getRegionNameAsString());
+              if (LOG.isTraceEnabled()) LOG.trace("openScanner - txId " + transId + ", called addScanner, scanner id " + scannerId + ", region " + m_regionDetails);
             }
             else
-              if (LOG.isTraceEnabled()) LOG.trace("TrxRegionEndpoint coprocessor: openScanner - txId " + transId + ", getScanner returned null, scanner id " + scannerId + ", regionName " + regionInfo.getRegionNameAsString());
+              if (LOG.isTraceEnabled()) LOG.trace("openScanner - txId " + transId + ", getScanner returned null, scanner id " + scannerId + ", region " + m_regionDetails);
        
           } catch (LeaseStillHeldException llse) {
 /*
             try {
                 scannerLeases.cancelLease(getScannerLeaseId(scannerId));
               } catch (LeaseException le) {
-                  if (LOG.isTraceEnabled()) LOG.trace("TrxRegionEndpoint coprocessor: getScanner failed to get a lease " + scannerId);
+                  if (LOG.isTraceEnabled()) LOG.trace("getScanner failed to get a lease " + scannerId);
               }
 */
-            LOG.error("TrxRegionEndpoint coprocessor: openScanner - txId " + transId + ", getScanner Error opening scanner, " + llse.toString());
+            LOG.error("openScanner - txId " + transId + ", getScanner Error opening scanner, ", llse);
             exceptionThrown = true;
             lse = llse;
           } catch (IOException e) {
-            LOG.error("TrxRegionEndpoint coprocessor: openScanner - txId " + transId + ", getScanner Error opening scanner, " + e.toString());
+            LOG.error("openScanner - txId " + transId + ", getScanner Error opening scanner, ", e);
             exceptionThrown = true;
             ioe = e;
           }
         }
 
-        if (LOG.isTraceEnabled()) LOG.trace("TrxRegionEndpoint coprocessor: openScanner - txId " + transId + ", scanner id " + scannerId + ", regionName " + regionInfo.getRegionNameAsString());
+        if (LOG.isTraceEnabled()) LOG.trace("openScanner - txId " + transId + ", scanner id " + scannerId + ", region " + m_regionDetails);
       }
     }
 
@@ -2190,12 +1988,6 @@ CoprocessorService, Coprocessor {
       openResponseBuilder.setException(t.toString());
     }
 
-    if (wre != null)
-    {
-      openResponseBuilder.setHasException(true);
-      openResponseBuilder.setException(wre.toString());
-    }
-
     if (ioe != null)
     {
       openResponseBuilder.setHasException(true);
@@ -2210,7 +2002,7 @@ CoprocessorService, Coprocessor {
 
     if (mue != null)
     {
-      if (LOG.isTraceEnabled()) LOG.trace("TrxRegionEndpoint coprocessor: openScanner - performing memoryPercentage " + memoryPercentage + ", posting memory usage exceeds indicated percentage");
+      if (LOG.isTraceEnabled()) LOG.trace("openScanner - performing memoryPercentage " + memoryPercentage + ", posting memory usage exceeds indicated percentage");
       openResponseBuilder.setHasException(true);
       openResponseBuilder.setException(mue.toString());
     }
@@ -2232,7 +2024,6 @@ CoprocessorService, Coprocessor {
     OutOfOrderScannerNextException ooo = null;
     UnknownScannerException use = null;
     MemoryUsageException mue = null;
-    WrongRegionException wre = null;
     Exception ne = null;
     Scan scan = null;
     List<Cell> cellResults = new ArrayList<Cell>();
@@ -2241,6 +2032,7 @@ CoprocessorService, Coprocessor {
 
     long scannerId = request.getScannerId();
     long transId = request.getTransactionId();
+    long startId = request.getStartId();
     int numberOfRows = request.getNumberOfRows();
     boolean closeScanner = request.getCloseScanner();
     long nextCallSeq = request.getNextCallSeq();
@@ -2250,31 +2042,17 @@ CoprocessorService, Coprocessor {
 
     boolean exceptionThrown = false;
 
-    if (LOG.isTraceEnabled()) LOG.trace("TrxRegionEndpoint coprocessor: performScan - txId " + transId + ", scanner id " + scannerId + ", numberOfRows " + numberOfRows + ", nextCallSeq " + nextCallSeq + ", closeScanner is " + closeScanner + ", region is " + regionInfo.getRegionNameAsString());
+    if (LOG.isTraceEnabled()) LOG.trace("performScan - txId " + transId + ", scanner id " + scannerId + ", numberOfRows " + numberOfRows
+           + ", nextCallSeq " + nextCallSeq + ", closeScanner is " + closeScanner + ", region is " + m_regionDetails);
 
-    /* commenting it out for the time-being
-    java.lang.String name = ((com.google.protobuf.ByteString) request.getRegionName()).toStringUtf8();
-
-    // First test if this region matches our region name
-    
-    if (!name.equals(regionInfo.getRegionNameAsString())) {
-       wre = new WrongRegionException("Request Region Name, " +
-        name + ",  does not match this region, " +
-        regionInfo.getRegionNameAsString());
-        if (LOG.isTraceEnabled()) LOG.trace("TrxRegionEndpoint coprocessor:performScan threw WrongRegionException" +
-      "Request Region Name, " +
-        name + ",  does not match this region, " +
-        regionInfo.getRegionNameAsString());
-    } else 
-    */
     {
     
       if (memoryThrottle == true) {
         if(memoryUsageWarnOnly == true)  {
-          LOG.warn("TrxRegionEndpoint coprocessor: performScan - performing memoryPercentage " + memoryPercentage + ", warning memory usage exceeds indicated percentage");
+          LOG.warn("performScan - performing memoryPercentage " + memoryPercentage + ", warning memory usage exceeds indicated percentage");
         }
         else {
-          if (LOG.isTraceEnabled()) LOG.trace("TrxRegionEndpoint coprocessor: performScan - performing memoryPercentage " + memoryPercentage + ", generating memory usage exceeds indicated percentage");
+          if (LOG.isTraceEnabled()) LOG.trace("performScan - performing memoryPercentage " + memoryPercentage + ", generating memory usage exceeds indicated percentage");
           mue = new MemoryUsageException("performScan memory usage exceeds " + memoryUsageThreshold + " percent, trxId is " + transId);
         }
       }
@@ -2291,9 +2069,8 @@ CoprocessorService, Coprocessor {
 
         if (keyFound != true)
         {
-          if (LOG.isTraceEnabled()) LOG.trace("TrxRegionEndpoint coprocessor: performScan - Unknown transaction [" + transId
-                 + "] in region ["
-                 + m_Region.getRegionInfo().getRegionNameAsString()
+          if (LOG.isTraceEnabled()) LOG.trace("performScan - Unknown transaction [" + transId
+                 + "] in region [" + m_regionDetails
                  + "], will create an OutOfOrderProtocol exception ");
           oop = new OutOfOrderProtocolException("performScan does not have an active transaction with an open scanner, txId: " + transId);
          }
@@ -2305,7 +2082,7 @@ CoprocessorService, Coprocessor {
 
           if (scanner != null)
           {
-            if (LOG.isTraceEnabled()) LOG.trace("TrxRegionEndpoint coprocessor: performScan - txId " + transId + ", scanner id " + scannerId + ", scanner is not null"); 
+            if (LOG.isTraceEnabled()) LOG.trace("performScan - txId " + transId + ", scanner id " + scannerId + ", scanner is not null");
             while (shouldContinue) {
               hasMore = scanner.next(cellResults);
               result = Result.create(cellResults);
@@ -2319,22 +2096,23 @@ CoprocessorService, Coprocessor {
               if (count == numberOfRows || !hasMore)
                 shouldContinue = false;
             }
-            if (LOG.isTraceEnabled()) LOG.trace("TrxRegionEndpoint coprocessor: performScan - txId " + transId + ", scanner id " + scannerId + ", count is " + count + ", hasMore is " + hasMore + ", result " + result.isEmpty());
+            if (LOG.isTraceEnabled()) LOG.trace("performScan - txId " + transId + ", scanner id " + scannerId + ", row count is " + count
+                    + ", hasMore is " + hasMore + ", result " + result.isEmpty() + " region " + m_regionDetails);
           }
           else
           {
-            if (LOG.isTraceEnabled()) LOG.trace("TrxRegionEndpoint coprocessor: performScan - txId " + transId + ", scanner id " + scannerId + transId + ", scanner is null"); 
+            if (LOG.isTraceEnabled()) LOG.trace("performScan - txId " + transId + ", scanner id " + scannerId + transId + ", scanner is null");
           }
        } catch(OutOfOrderScannerNextException ooone) {
-         if (LOG.isTraceEnabled()) LOG.trace("TrxRegionEndpoint coprocessor: performScan - txId " + transId + ", scanner id " + scannerId + " Caught OutOfOrderScannerNextException  " + ooone.getMessage() + " " + stackTraceToString(ooone));
+         LOG.error("performScan - txId " + transId + ", scanner id " + scannerId + " Caught OutOfOrderScannerNextException  " + ooone.getMessage() + " " + stackTraceToString(ooone));
          ooo = ooone;
          exceptionThrown = true;
        } catch(ScannerTimeoutException cste) {
-         if (LOG.isTraceEnabled()) LOG.trace("TrxRegionEndpoint coprocessor: performScan - txId " + transId + ", scanner id " + scannerId + " Caught ScannerTimeoutException  " + cste.getMessage() + " " + stackTraceToString(cste));
+         LOG.error("performScan - txId " + transId + ", scanner id " + scannerId + " Caught ScannerTimeoutException  " + cste.getMessage() + " " + stackTraceToString(cste));
          ste = cste;
          exceptionThrown = true;
        } catch(Throwable e) {
-         if (LOG.isTraceEnabled()) LOG.trace("TrxRegionEndpoint coprocessor: performScan - txId " + transId + ", scanner id " + scannerId + " Caught throwable exception " + e.getMessage() + " " + stackTraceToString(e));
+         LOG.error("performScan - txId " + transId + ", scanner id " + scannerId + " Caught throwable exception " + e.getMessage() + " " + stackTraceToString(e));
          t = e;
          exceptionThrown = true;
        }
@@ -2342,7 +2120,7 @@ CoprocessorService, Coprocessor {
          if (scanner != null) {
            try {
              if (closeScanner) {
-               if (LOG.isTraceEnabled()) LOG.trace("TrxRegionEndpoint coprocessor: performScan - txId " + transId + ", scanner id " + scannerId + ", close scanner was true, closing the scanner" + ", closeScanner is " + closeScanner + ", region is " + regionInfo.getRegionNameAsString());
+               if (LOG.isTraceEnabled()) LOG.trace("performScan - txId " + transId + ", scanner id " + scannerId + ", close scanner was true, closing the scanner" + ", closeScanner is " + closeScanner + ", region is " + m_regionDetails);
                removeScanner(scannerId);
                scanner.close();
 /*
@@ -2350,12 +2128,12 @@ CoprocessorService, Coprocessor {
                  scannerLeases.cancelLease(getScannerLeaseId(scannerId));
                } catch (LeaseException le) {
                  // ignore
-                 if (LOG.isTraceEnabled()) LOG.trace("TrxRegionEndpoint coprocessor: performScan failed to get a lease " + scannerId);
+                 if (LOG.isTraceEnabled()) LOG.trace("performScan failed to get a lease " + scannerId);
                }
 */
              }
            } catch(Exception e) {
-             if (LOG.isTraceEnabled()) LOG.trace("TrxRegionEndpoint coprocessor: performScan -  transaction id " + transId + ", Caught general exception " + e.getMessage() + " " + stackTraceToString(e));
+             LOG.error("performScan -  transaction id " + transId + ", Caught general exception " + e.getMessage() + " " + stackTraceToString(e));
              ne = e;
              exceptionThrown = true;
            }
@@ -2370,7 +2148,7 @@ CoprocessorService, Coprocessor {
  
            if (rsh == null)
            {
-            if (LOG.isTraceEnabled()) LOG.trace("TrxRegionEndpoint coprocessor: performScan rsh is null");
+            if (LOG.isTraceEnabled()) LOG.trace("performScan rsh is null");
               use =  new UnknownScannerException(
                 "ScannerId: " + scannerId + ", already closed?");
            }
@@ -2380,16 +2158,16 @@ CoprocessorService, Coprocessor {
 
            if (rsh == null)
            {
-            if (LOG.isTraceEnabled()) LOG.trace("TrxRegionEndpoint coprocessor: performScan - txId " + transId + ", performScan rsh is null, UnknownScannerException for scannerId: " + scannerId + ", nextCallSeq was " + nextCallSeq + ", for region " + regionInfo.getRegionNameAsString());
+            if (LOG.isTraceEnabled()) LOG.trace("performScan - txId " + transId + ", performScan rsh is null, UnknownScannerException for scannerId: " + scannerId + ", nextCallSeq was " + nextCallSeq + ", for region " + m_regionDetails);
               use =  new UnknownScannerException(
-                 "ScannerId: " + scannerId + ", was scanner already closed?, transaction id " + transId + ", nextCallSeq was " + nextCallSeq + ", for region " + regionInfo.getRegionNameAsString());
+                 "ScannerId: " + scannerId + ", was scanner already closed?, transaction id " + transId + ", nextCallSeq was " + nextCallSeq + ", for region " + m_regionDetails);
            }
            else
            {
              rsh.nextCallSeq = nextCallSeq;
 
-             if (LOG.isTraceEnabled()) LOG.trace("TrxRegionEndpoint coprocessor: performScan - txId " + transId + ", scanner id " + scannerId + ", regionName " + regionInfo.getRegionNameAsString() +
-", nextCallSeq " + nextCallSeq + ", rsh.nextCallSeq " + rsh.nextCallSeq + ", close scanner is " + closeScanner);
+             if (LOG.isTraceEnabled()) LOG.trace("performScan exit - txId " + transId + ", scanner id " + scannerId + " row count " + count + ", region " + m_regionDetails
+                    + ", nextCallSeq " + nextCallSeq + ", rsh.nextCallSeq " + rsh.nextCallSeq + ", close scanner is " + closeScanner);
 
           }
          }
@@ -2426,13 +2204,6 @@ CoprocessorService, Coprocessor {
       performResponseBuilder.setException(ste.toString());
     }
 
-    if (wre != null)
-    {
-      performResponseBuilder.setHasMore(false);
-      performResponseBuilder.setHasException(true);
-      performResponseBuilder.setException(wre.toString());
-    }
-
     if (ne != null)
     {
       performResponseBuilder.setHasMore(false);
@@ -2461,15 +2232,15 @@ CoprocessorService, Coprocessor {
       {
         performResponseBuilder.setHasException(true);
         performResponseBuilder.setException(oop.toString());
-        LOG.warn("TrxRegionEndpoint coprocessor: performScan - OutOfOrderProtocolException, transaction was not found, txId: " + transId + ", return exception" + ", regionName " + regionInfo.getRegionNameAsString());
+        LOG.warn("performScan - OutOfOrderProtocolException, transaction was not found, txId: " + transId + ", return exception" + ", region " + m_regionDetails);
       }
       else
-        LOG.warn("TrxRegionEndpoint coprocessor: performScan - suppressing OutOfOrderProtocolException, transaction was not found, txId: " + transId + ", regionName " + regionInfo.getRegionNameAsString());
+        LOG.warn("performScan - suppressing OutOfOrderProtocolException, transaction was not found, txId: " + transId + ", region " + m_regionDetails);
     }
 
     if (mue != null)
     {
-      if (LOG.isTraceEnabled()) LOG.trace("TrxRegionEndpoint coprocessor: performScan - performing memoryPercentage " + memoryPercentage + ", posting memory usage exceeds indicated percentage");
+      if (LOG.isTraceEnabled()) LOG.trace("performScan - performing memoryPercentage " + memoryPercentage + ", posting memory usage exceeds indicated percentage");
       performResponseBuilder.setHasMore(false);
       performResponseBuilder.setHasException(true);
       performResponseBuilder.setException(mue.toString());
@@ -2480,549 +2251,374 @@ CoprocessorService, Coprocessor {
   }
 
   public void deleteTlogEntries(RpcController controller,
-           TlogDeleteRequest request, RpcCallback<TlogDeleteResponse> done) {
+          TlogDeleteRequest request, RpcCallback<TlogDeleteResponse> done) {
+    boolean hasMore = true;
+    InternalScanner scanner = null;
+    Throwable t = null;
+    ScannerTimeoutException ste = null;
+    OutOfOrderProtocolException oop = null;
+    OutOfOrderScannerNextException ooo = null;
+    UnknownScannerException use = null;
+    MemoryUsageException mue = null;
+    Exception ne = null;
+    Scan scan = null;
+    List<Cell> cellResults = new ArrayList<Cell>();
+    org.apache.hadoop.hbase.client.Result result = null;
+    long lvAsn = request.getAuditSeqNum();
+    boolean lvAgeCommitted = request.getAgeCommitted();
+
+    try{
+       scan = ProtobufUtil.toScan(request.getScan());
+       // use an internal scanner to perform scanning.
+       scanner = m_Region.getScanner(scan);
+    }
+    catch (Exception e){
+       if (LOG.isErrorEnabled()) LOG.error("deleteTlogEntries Exception in region: "
+          + m_regionDetails + " getting scanner ", e );
+    }
+
+    long count = 0L;
+    boolean shouldContinue = true;
+
+    if (LOG.isTraceEnabled()) LOG.trace("deleteTlogEntries ENTRY.  Records older than " + lvAsn
+        + " will be deleted in region: " + m_regionDetails);
+
+    try {
+       if (scanner != null){
+          if (LOG.isTraceEnabled()) LOG.trace("deleteTlogEntries - scanner is not null");
+          while (shouldContinue) {
+             hasMore = scanner.next(cellResults);
+             result = Result.create(cellResults);
+             if (!result.isEmpty()) {
+                for (Cell cell : result.rawCells()) {
+                   String valueString = new String(CellUtil.cloneValue(cell));
+                   StringTokenizer st = new StringTokenizer(valueString, ",");
+                   if (st.hasMoreElements()) {
+                      String asnToken = st.nextElement().toString();
+                      String transidToken = st.nextElement().toString();
+                      String stateToken = st.nextElement().toString();
+                      if (LOG.isTraceEnabled()) LOG.trace("deleteTlogEntries transidToken: "
+                                   + transidToken + " asnToken: " + asnToken);
+                      if (Long.parseLong(asnToken) < lvAsn) {
+                         if ( (stateToken.contains(TransState.STATE_FORGOTTEN.toString())) ||
+                              (stateToken.equals(TransState.STATE_COMMITTED.toString()) && (lvAgeCommitted)) ||
+                              (stateToken.equals(TransState.STATE_ABORTED.toString()) && (lvAgeCommitted))) {
+
+                            if (LOG.isTraceEnabled()) LOG.trace("Deleting transid: " + transidToken
+                                  + " from region: " + m_regionDetails + " with state: " + stateToken);
+                            try {
+                               Delete d = new Delete(result.getRow());
+                               d.setDurability(Durability.SKIP_WAL);
+                               m_Region.delete(d);
+                            }
+                            catch (Exception e) {
+                               LOG.warn("deleteTlogEntries -"
+                                   + " txId " + transidToken + ", Executing delete caught an exception ", e);
+                               throw new IOException("deleteTlogEntries -"
+                                   + " txId " + transidToken + ", Executing delete caught an exception " + e.toString());
+                            }
+                            count++;
+                         }
+                      } else {
+                         if (LOG.isTraceEnabled()) LOG.trace("deleteTlogEntries Ending scan at asn: " + asnToken
+                                    + ", transid: " + transidToken +
+                                    " because it is not less than the comparator: " + lvAsn +
+                                    " in region: " + m_regionDetails);
+                         shouldContinue = false;
+                         break;
+                      }
+                   } // if (st.hasMoreElements()
+                } // for (Cell cell : result.rawCells()
+             } // if (!result.isEmpty()
+             cellResults.clear();
+
+             if (!hasMore){
+                shouldContinue = false;
+             }
+          } // while (shouldContinue)
+          if (LOG.isTraceEnabled()) LOG.trace("deleteTlogEntries - count is " + count + ", hasMore is " + hasMore
+                    + ", result " + result.isEmpty());
+       }
+       else {
+          if (LOG.isTraceEnabled()) LOG.trace("deleteTlogEntries - scanner is null");
+       }
+    } catch(OutOfOrderScannerNextException ooone) {
+       LOG.warn("deleteTlogEntries - Caught OutOfOrderScannerNextException ", ooone);
+       ooo = ooone;
+    } catch(ScannerTimeoutException cste) {
+       LOG.warn("deleteTlogEntries - Caught ScannerTimeoutException ", cste);
+       ste = cste;
+    } catch(Throwable e) {
+       LOG.warn("deleteTlogEntries - Caught throwable exception ", e);
+       t = e;
+    } finally {
+       if (LOG.isTraceEnabled()) LOG.trace("deleteTlogEntries - closing the scanner, region is " + m_regionDetails);
+       try{
+          scanner.close();
+       }
+       catch(IOException ioe){
+          LOG.warn("deleteTlogEntries - except

<TRUNCATED>


[14/22] incubator-trafodion git commit: Merge branch 'master' of github.com:apache/incubator-trafodion into sean_traf

Posted by sa...@apache.org.
Merge branch 'master' of github.com:apache/incubator-trafodion into sean_traf


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

Branch: refs/heads/master
Commit: 5cfd92de2939511fb67b477c872a61cb0a344f18
Parents: cd75b42 d0c4d73
Author: Sean Broeder <sb...@edev05.novalocal>
Authored: Tue May 2 19:15:26 2017 +0000
Committer: Sean Broeder <sb...@edev05.novalocal>
Committed: Tue May 2 19:15:26 2017 +0000

----------------------------------------------------------------------
 core/conn/odbc/src/odbc/nsksrvr/SrvrMain.cpp    |  16 +-
 core/sqf/conf/log4cxx.trafodion.lob.config      |  49 --
 core/sqf/conf/log4cxx.trafodion.sql.config      |  51 ++
 core/sqf/conf/log4cxx.trafodion.udr.config      |  50 --
 core/sqf/conf/log4j.hdfs.config                 |  56 ---
 core/sqf/conf/log4j.sql.config                  |  51 ++
 core/sqf/tools/sqtools.sh                       |   2 +-
 core/sql/arkcmp/CmpStatement.cpp                |  23 +-
 core/sql/bin/SqlciMain.cpp                      |   5 +-
 core/sql/bin/arkcmp.cpp                         |  12 +-
 core/sql/bin/clitest.cpp                        |   2 +-
 core/sql/bin/ex_esp_main.cpp                    |   9 +-
 core/sql/cli/CliExtern.cpp                      |   2 +-
 core/sql/cli/Statement.cpp                      |   3 +
 core/sql/cli/sqlcli.h                           |   5 +-
 core/sql/clitest/blobtest.cpp                   |   4 +-
 core/sql/comexe/CmpMessage.h                    |  10 +-
 core/sql/common/ComSmallDefs.h                  |   1 +
 core/sql/executor/ExExeUtilCli.cpp              |  25 +-
 core/sql/executor/ExExeUtilCli.h                |   3 +-
 core/sql/executor/ExExeUtilExplain.cpp          |  20 +-
 core/sql/executor/ExExeUtilLoad.cpp             |   2 +-
 core/sql/executor/HBaseClient_JNI.cpp           |   3 +-
 core/sql/exp/ExpLOB.cpp                         | 103 ++--
 core/sql/exp/ExpLOB.h                           |  21 +-
 core/sql/exp/ExpLOBaccess.cpp                   | 215 +++++++--
 core/sql/exp/ExpLOBaccess.h                     |  22 +-
 core/sql/exp/ExpLOBenums.h                      |  17 +-
 core/sql/exp/ExpLOBinterface.cpp                |  75 +--
 core/sql/exp/ExpLOBinterface.h                  |  34 +-
 core/sql/exp/ExpLOBprocess.cpp                  |   8 +-
 core/sql/generator/GenItemFunc.cpp              |   7 +-
 core/sql/generator/GenPreCode.cpp               |  52 +-
 core/sql/optimizer/ItemFunc.h                   |   2 +-
 core/sql/parser/SqlParserGlobalsCmn.h           |  84 +---
 core/sql/parser/SqlParserGlobalsEnum.h          | 119 +++++
 core/sql/qmscommon/QRLogger.cpp                 |  53 ++-
 core/sql/qmscommon/QRLogger.h                   |   2 +
 core/sql/regress/executor/EXPECTED130           | 469 ++++++++-----------
 core/sql/regress/executor/TEST130               |  37 +-
 core/sql/regress/privs1/EXPECTED120             |  16 +
 core/sql/regress/privs1/TEST120                 |   3 +
 core/sql/sqlcomp/CmpMain.cpp                    |  10 +-
 core/sql/sqlcomp/CmpMain.h                      |   3 +-
 core/sql/sqlcomp/nadefaults.cpp                 |   2 +-
 core/sql/sqludr/SqlUdrPredefLogReader.cpp       |   1 +
 .../java/org/trafodion/sql/HBaseClient.java     |  30 +-
 .../main/java/org/trafodion/sql/HiveClient.java |   7 +-
 core/sql/udrserv/udrserv.cpp                    |   8 +-
 licenses/lic-pyinstall-src                      |   4 +-
 50 files changed, 1007 insertions(+), 801 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-trafodion/blob/5cfd92de/core/sql/src/main/java/org/trafodion/sql/HBaseClient.java
----------------------------------------------------------------------


[15/22] incubator-trafodion git commit: Updated expected seabase/expected022

Posted by sa...@apache.org.
Updated expected seabase/expected022


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

Branch: refs/heads/master
Commit: bb531e6ab5f4e00517628f78833d8d55ff5d79f0
Parents: 5cfd92d
Author: Sean Broeder <sb...@edev05.novalocal>
Authored: Tue May 2 20:12:51 2017 +0000
Committer: Sean Broeder <sb...@edev05.novalocal>
Committed: Tue May 2 20:12:51 2017 +0000

----------------------------------------------------------------------
 core/sql/regress/seabase/EXPECTED022 | 52 +++++++++++++++----------------
 1 file changed, 26 insertions(+), 26 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-trafodion/blob/bb531e6a/core/sql/regress/seabase/EXPECTED022
----------------------------------------------------------------------
diff --git a/core/sql/regress/seabase/EXPECTED022 b/core/sql/regress/seabase/EXPECTED022
index fa3a3af..e57c4d4 100644
--- a/core/sql/regress/seabase/EXPECTED022
+++ b/core/sql/regress/seabase/EXPECTED022
@@ -426,7 +426,7 @@ ROW_ID      (EXPR)
 >>invoke hbase."_CELL_".t022hbt2;
 
 -- Definition of Trafodion table HBASE."_CELL_".T022HBT2
--- Definition current  Mon Dec 12 23:03:22 2016
+-- Definition current  Tue May  2 16:52:50 2017
 
   (
     ROW_ID                           VARCHAR(100) CHARACTER SET ISO88591
@@ -445,7 +445,7 @@ ROW_ID      (EXPR)
 >>invoke hbase."_ROW_".t022hbt2;
 
 -- Definition of Trafodion table HBASE."_ROW_".T022HBT2
--- Definition current  Mon Dec 12 23:03:23 2016
+-- Definition current  Tue May  2 16:52:50 2017
 
   (
     ROW_ID                           VARCHAR(100) CHARACTER SET ISO88591
@@ -465,7 +465,7 @@ ROW_ID      (EXPR)
 >>invoke hbase."_CELL_".t022hbt2;
 
 -- Definition of Trafodion table HBASE."_CELL_".T022HBT2
--- Definition current  Mon Dec 12 23:03:23 2016
+-- Definition current  Tue May  2 16:52:50 2017
 
   (
     ROW_ID                           VARCHAR(100) CHARACTER SET ISO88591
@@ -484,7 +484,7 @@ ROW_ID      (EXPR)
 >>invoke hbase."_ROW_".t022hbt2;
 
 -- Definition of Trafodion table HBASE."_ROW_".T022HBT2
--- Definition current  Mon Dec 12 23:03:24 2016
+-- Definition current  Tue May  2 16:52:51 2017
 
   (
     ROW_ID                           VARCHAR(100) CHARACTER SET ISO88591
@@ -603,7 +603,7 @@ ROW_ID      (EXPR)
 >>invoke t022hbm1;
 
 -- Definition of Trafodion HBase mapped table T022HBM1
--- Definition current  Mon Dec 12 23:03:37 2016
+-- Definition current  Tue May  2 16:53:06 2017
 
   (
     "cf".A                           VARCHAR(4) CHARACTER SET ISO88591 COLLATE
@@ -619,7 +619,7 @@ ROW_ID      (EXPR)
 >>invoke t022hbm1;
 
 -- Definition of Trafodion HBase mapped table T022HBM1
--- Definition current  Mon Dec 12 23:03:38 2016
+-- Definition current  Tue May  2 16:53:07 2017
 
   (
     "cf".A                           VARCHAR(4) CHARACTER SET ISO88591 COLLATE
@@ -635,8 +635,8 @@ ROW_ID      (EXPR)
 --- SQL operation complete.
 >>invoke t022hbm1;
 
--- Definition of Trafodion table TRAFODION.SCH.T022HBM1
--- Definition current  Mon Dec 12 23:03:40 2016
+-- Definition of Trafodion table TRAFODION.SEABASE.T022HBM1
+-- Definition current  Tue May  2 16:53:11 2017
 
   (
     SYSKEY                           LARGEINT NO DEFAULT NOT NULL NOT DROPPABLE
@@ -650,7 +650,7 @@ ROW_ID      (EXPR)
 >>invoke t022hbm1;
 
 -- Definition of Trafodion HBase mapped table T022HBM1
--- Definition current  Mon Dec 12 23:03:49 2016
+-- Definition current  Tue May  2 16:53:20 2017
 
   (
     "cf".A                           VARCHAR(4) CHARACTER SET ISO88591 COLLATE
@@ -698,7 +698,7 @@ a2              ?
 >>invoke t022hbm1;
 
 -- Definition of Trafodion HBase mapped table T022HBM1
--- Definition current  Mon Dec 12 23:03:53 2016
+-- Definition current  Tue May  2 16:53:25 2017
 
   (
     "cf".A                           VARCHAR(4) CHARACTER SET ISO88591 COLLATE
@@ -720,8 +720,8 @@ a2              ?
 --- SQL operation complete.
 >>invoke t022hbm1_like;
 
--- Definition of Trafodion table TRAFODION.SCH.T022HBM1_LIKE
--- Definition current  Mon Dec 12 23:03:58 2016
+-- Definition of Trafodion table TRAFODION.SEABASE.T022HBM1_LIKE
+-- Definition current  Tue May  2 16:53:33 2017
 
   (
     "cf".A                           VARCHAR(4) CHARACTER SET ISO88591 COLLATE
@@ -770,7 +770,7 @@ a2    ?               ?
 --- SQL operation complete.
 >>invoke t022hbm1;
 
-*** ERROR[4082] Object TRAFODION.SCH.T022HBM1 does not exist or is inaccessible.
+*** ERROR[4082] Object TRAFODION.SEABASE.T022HBM1 does not exist or is inaccessible.
 
 --- SQL operation failed with errors.
 >>create external table t022hbm1 ("cf".a varchar(4) not null,
@@ -782,7 +782,7 @@ a2    ?               ?
 >>invoke t022hbm1;
 
 -- Definition of Trafodion HBase mapped table T022HBM1
--- Definition current  Mon Dec 12 23:04:14 2016
+-- Definition current  Tue May  2 16:53:49 2017
 
   (
     "cf".A                           VARCHAR(4) CHARACTER SET ISO88591 COLLATE
@@ -799,7 +799,7 @@ a2    ?               ?
 >>invoke t022hbm1;
 
 -- Definition of Trafodion HBase mapped table T022HBM1
--- Definition current  Mon Dec 12 23:04:17 2016
+-- Definition current  Tue May  2 16:53:54 2017
 
   (
     "cf".A                           VARCHAR(4) CHARACTER SET ISO88591 COLLATE
@@ -815,7 +815,7 @@ a2    ?               ?
 >>invoke t022hbm1;
 
 -- Definition of Trafodion HBase mapped table T022HBM1
--- Definition current  Mon Dec 12 23:04:20 2016
+-- Definition current  Tue May  2 16:53:56 2017
 
   (
     "cf".A                           VARCHAR(4) CHARACTER SET ISO88591 COLLATE
@@ -941,7 +941,7 @@ b               ?
 >>invoke t022hbm1;
 
 -- Definition of Trafodion HBase mapped table T022HBM1
--- Definition current  Mon Dec 12 23:04:35 2016
+-- Definition current  Tue May  2 16:54:08 2017
 
   (
     "cf".A                           INT NO DEFAULT NOT NULL NOT DROPPABLE
@@ -1001,7 +1001,7 @@ A            B            C
 >>invoke t022hbm1;
 
 -- Definition of Trafodion HBase mapped table T022HBM1
--- Definition current  Mon Dec 12 23:04:41 2016
+-- Definition current  Tue May  2 16:54:12 2017
 
   (
     "cf".A                           VARCHAR(4) CHARACTER SET ISO88591 COLLATE
@@ -1024,7 +1024,7 @@ A            B            C
 >>invoke t022hbm1;
 
 -- Definition of Trafodion HBase mapped table T022HBM1
--- Definition current  Mon Dec 12 23:04:49 2016
+-- Definition current  Tue May  2 16:54:18 2017
 
   (
     "cf".A                           CHAR(4) CHARACTER SET ISO88591 COLLATE
@@ -1046,7 +1046,7 @@ A            B            C
 >>invoke t022hbm1;
 
 -- Definition of Trafodion HBase mapped table T022HBM1
--- Definition current  Mon Dec 12 23:04:54 2016
+-- Definition current  Tue May  2 16:54:24 2017
 
   (
     "cf".A                           VARCHAR(4) CHARACTER SET ISO88591 COLLATE
@@ -1068,7 +1068,7 @@ A            B            C
 >>invoke t022hbm1;
 
 -- Definition of Trafodion HBase mapped table T022HBM1
--- Definition current  Mon Dec 12 23:04:58 2016
+-- Definition current  Tue May  2 16:54:28 2017
 
   (
     "cf".A                           VARCHAR(4) CHARACTER SET ISO88591 COLLATE
@@ -1091,7 +1091,7 @@ A            B            C
 >>invoke t022hbm1;
 
 -- Definition of Trafodion HBase mapped table T022HBM1
--- Definition current  Mon Dec 12 23:05:03 2016
+-- Definition current  Tue May  2 16:54:34 2017
 
   (
     "cf".A                           VARCHAR(4) CHARACTER SET ISO88591 COLLATE
@@ -1115,7 +1115,7 @@ A            B            C
 >>invoke t022hbm1;
 
 -- Definition of Trafodion HBase mapped table T022HBM1
--- Definition current  Mon Dec 12 23:05:07 2016
+-- Definition current  Tue May  2 16:54:40 2017
 
   (
     "cf".A                           CHAR(4) CHARACTER SET ISO88591 COLLATE
@@ -1138,7 +1138,7 @@ A            B            C
 >>invoke t022hbm1;
 
 -- Definition of Trafodion HBase mapped table T022HBM1
--- Definition current  Mon Dec 12 23:05:15 2016
+-- Definition current  Tue May  2 16:54:43 2017
 
   (
     "cf".A                           VARCHAR(4) CHARACTER SET ISO88591 COLLATE
@@ -1190,7 +1190,7 @@ A            B            C
 >>invoke t022hbm2;
 
 -- Definition of Trafodion HBase mapped table T022HBM2
--- Definition current  Mon Dec 12 23:05:29 2016
+-- Definition current  Tue May  2 16:54:58 2017
 
   (
     "cf1".A                          INT DEFAULT NULL
@@ -1231,7 +1231,7 @@ a1             10
 >>invoke t022hbm1;
 
 -- Definition of Trafodion HBase mapped table T022HBM1
--- Definition current  Mon Dec 12 23:05:32 2016
+-- Definition current  Tue May  2 16:55:00 2017
 
   (
     "cf".A                           VARCHAR(4) CHARACTER SET ISO88591 COLLATE


[16/22] incubator-trafodion git commit: Updated seabase/EXPECTED022

Posted by sa...@apache.org.
Updated seabase/EXPECTED022


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

Branch: refs/heads/master
Commit: 7a7cfa3fd373ad653bc139a6b4184b8552df94c4
Parents: bb531e6
Author: Sean Broeder <sb...@edev05.novalocal>
Authored: Wed May 3 21:27:41 2017 +0000
Committer: Sean Broeder <sb...@edev05.novalocal>
Committed: Wed May 3 21:27:41 2017 +0000

----------------------------------------------------------------------
 core/sql/regress/seabase/EXPECTED022 | 1058 ++++++++++++++---------------
 1 file changed, 529 insertions(+), 529 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-trafodion/blob/7a7cfa3f/core/sql/regress/seabase/EXPECTED022
----------------------------------------------------------------------
diff --git a/core/sql/regress/seabase/EXPECTED022 b/core/sql/regress/seabase/EXPECTED022
index e57c4d4..399deac 100644
--- a/core/sql/regress/seabase/EXPECTED022
+++ b/core/sql/regress/seabase/EXPECTED022
@@ -31,11 +31,11 @@
 
 --- 1 row(s) inserted.
 >>insert into hbase."_ROW_".t022hbt1 values ('2', column_create('cf:2b', '201')),
-+>                              ('1', column_create('cf:1b', '102'));
++> ('1', column_create('cf:1b', '102'));
 
 --- 2 row(s) inserted.
->>prepare s from insert into hbase."_ROW_".t022hbt1 values 
-+>         (?, column_create((?, ?), ('cf:3b', '302')));
+>>prepare s from insert into hbase."_ROW_".t022hbt1 values
++> (?, column_create((?, ?), ('cf:3b', '302')));
 
 --- SQL command prepared.
 >>execute s using '3', 'cf:3a', '301';
@@ -43,49 +43,49 @@
 --- 1 row(s) inserted.
 >>
 >>select left(row_id, 10) row_id, column_display(column_details, 40) column_details
-+>           from hbase."_ROW_".t022hbt1;
++> from hbase."_ROW_".t022hbt1;
 
-ROW_ID      COLUMN_DETAILS                          
-----------  ----------------------------------------
+ROW_ID COLUMN_DETAILS
+----------
 
-1           cf:1a => 101, cf:1b => 102              
-2           cf:2b => 201                            
-3           cf:3a => 301, cf:3b => 302              
+1 cf:1a => 101, cf:1b => 102
+2 cf:2b => 201
+3 cf:3a => 301, cf:3b => 302
 
 --- 3 row(s) selected.
->>select left(row_id, 10) row_id, left(column_display(column_details), 40) 
-+>           from hbase."_ROW_".t022hbt1;
+>>select left(row_id, 10) row_id, left(column_display(column_details), 40)
++> from hbase."_ROW_".t022hbt1;
 
-ROW_ID      (EXPR)                                  
-----------  ----------------------------------------
+ROW_ID (EXPR)
+----------
 
-1           cf:1a => 101, cf:1b => 102              
-2           cf:2b => 201                            
-3           cf:3a => 301, cf:3b => 302              
+1 cf:1a => 101, cf:1b => 102
+2 cf:2b => 201
+3 cf:3a => 301, cf:3b => 302
 
 --- 3 row(s) selected.
->>select left(row_id, 10) row_id, left(column_display(column_details, ('cf:2b', 'cf:1b')), 40) 
-+>           from hbase."_ROW_".t022hbt1;
+>>select left(row_id, 10) row_id, left(column_display(column_details, ('cf:2b', 'cf:1b')), 40)
++> from hbase."_ROW_".t022hbt1;
 
-ROW_ID      (EXPR)                                  
-----------  ----------------------------------------
+ROW_ID (EXPR)
+----------
 
-1           cf:1b => 102                            
-2           cf:2b => 201                            
+1 cf:1b => 102
+2 cf:2b => 201
 
 --- 2 row(s) selected.
 >>
 >>-- no rows updated. where pred fails.
->>update  hbase."_ROW_".t022hbt1 set column_details = column_create ('cf:a', 
-+>          (cast(column_lookup(column_details, 'cf:a') as varchar(10)) || '0103'))
-+>   where row_id = '3' and column_lookup(column_details, 'cf:3b') = '3021';
+>>update hbase."_ROW_".t022hbt1 set column_details = column_create ('cf:a',
++> (cast(column_lookup(column_details, 'cf:a') as varchar(10)) || '0103'))
++> where row_id = '3' and column_lookup(column_details, 'cf:3b') = '3021';
 
 --- 0 row(s) updated.
 >>
 >>-- no rows updated. column not found in set clause.
->>update  hbase."_ROW_".t022hbt1 set column_details = column_create ('cf:a', 
-+>          (cast(column_lookup(column_details, 'cf:a') as varchar(10)) || '0103'))
-+>   where row_id = '3' and column_lookup(column_details, 'cf:3b') = '302';
+>>update hbase."_ROW_".t022hbt1 set column_details = column_create ('cf:a',
++> (cast(column_lookup(column_details, 'cf:a') as varchar(10)) || '0103'))
++> where row_id = '3' and column_lookup(column_details, 'cf:3b') = '302';
 
 --- 0 row(s) updated.
 >>
@@ -93,36 +93,36 @@ ROW_ID      (EXPR)
 
 --- SQL operation complete.
 >>-- one row updated
->>update  hbase."_ROW_".t022hbt1 set column_details = column_create ('cf:3a', 
-+>          (cast(column_lookup(column_details, 'cf:3a') as varchar(10)) || '0103'))
-+>   where row_id = '3' and column_lookup(column_details, 'cf:3b') = '302';
+>>update hbase."_ROW_".t022hbt1 set column_details = column_create ('cf:3a',
++> (cast(column_lookup(column_details, 'cf:3a') as varchar(10)) || '0103'))
++> where row_id = '3' and column_lookup(column_details, 'cf:3b') = '302';
 
 --- 1 row(s) updated.
 >>
->>select left(row_id, 10) row_id, left(column_display(column_details), 50) 
-+>           from hbase."_ROW_".t022hbt1;
+>>select left(row_id, 10) row_id, left(column_display(column_details), 50)
++> from hbase."_ROW_".t022hbt1;
 
-ROW_ID      (EXPR)                                            
-----------  --------------------------------------------------
+ROW_ID (EXPR)
+----------
 
-1           cf:1a => 101, cf:1b => 102                        
-2           cf:2b => 201                                      
-3           cf:3a => 3010103, cf:3a => 301, cf:3b => 302      
+1 cf:1a => 101, cf:1b => 102
+2 cf:2b => 201
+3 cf:3a => 3010103, cf:3a => 301, cf:3b => 302
 
 --- 3 row(s) selected.
 >>rollback work;
 
 --- SQL operation complete.
 >>
->>select left(row_id, 10) row_id, left(column_display(column_details), 50) 
-+>           from hbase."_ROW_".t022hbt1;
+>>select left(row_id, 10) row_id, left(column_display(column_details), 50)
++> from hbase."_ROW_".t022hbt1;
 
-ROW_ID      (EXPR)                                            
-----------  --------------------------------------------------
+ROW_ID (EXPR)
+----------
 
-1           cf:1a => 101, cf:1b => 102                        
-2           cf:2b => 201                                      
-3           cf:3a => 301, cf:3b => 302                        
+1 cf:1a => 101, cf:1b => 102
+2 cf:2b => 201
+3 cf:3a => 301, cf:3b => 302
 
 --- 3 row(s) selected.
 >>
@@ -130,53 +130,53 @@ ROW_ID      (EXPR)
 
 --- SQL operation complete.
 >>-- one row updated
->>update  hbase."_ROW_".t022hbt1 set column_details = column_create ('cf:3a', 
-+>          (cast(column_lookup(column_details, 'cf:3a') as varchar(10)) || '0103'))
-+>   where row_id = '3' and column_lookup(column_details, 'cf:3b') = '302';
+>>update hbase."_ROW_".t022hbt1 set column_details = column_create ('cf:3a',
++> (cast(column_lookup(column_details, 'cf:3a') as varchar(10)) || '0103'))
++> where row_id = '3' and column_lookup(column_details, 'cf:3b') = '302';
 
 --- 1 row(s) updated.
 >>
->>select left(row_id, 10) row_id, left(column_display(column_details), 50) 
-+>           from hbase."_ROW_".t022hbt1;
+>>select left(row_id, 10) row_id, left(column_display(column_details), 50)
++> from hbase."_ROW_".t022hbt1;
 
-ROW_ID      (EXPR)                                            
-----------  --------------------------------------------------
+ROW_ID (EXPR)
+----------
 
-1           cf:1a => 101, cf:1b => 102                        
-2           cf:2b => 201                                      
-3           cf:3a => 3010103, cf:3a => 301, cf:3b => 302      
+1 cf:1a => 101, cf:1b => 102
+2 cf:2b => 201
+3 cf:3a => 3010103, cf:3a => 301, cf:3b => 302
 
 --- 3 row(s) selected.
 >>commit work;
 
 --- SQL operation complete.
 >>
->>select left(row_id, 10) row_id, left(column_display(column_details), 50) 
-+>           from hbase."_ROW_".t022hbt1;
+>>select left(row_id, 10) row_id, left(column_display(column_details), 50)
++> from hbase."_ROW_".t022hbt1;
 
-ROW_ID      (EXPR)                                            
-----------  --------------------------------------------------
+ROW_ID (EXPR)
+----------
 
-1           cf:1a => 101, cf:1b => 102                        
-2           cf:2b => 201                                      
-3           cf:3a => 3010103, cf:3b => 302                    
+1 cf:1a => 101, cf:1b => 102
+2 cf:2b => 201
+3 cf:3a => 3010103, cf:3b => 302
 
 --- 3 row(s) selected.
 >>
 >>begin work;
 
 --- SQL operation complete.
->>insert into hbase."_ROW_".t022hbt1 values 
-+>       ('4', column_create(('cf:4a', '301'), ('cf:4b', '302')));
+>>insert into hbase."_ROW_".t022hbt1 values
++> ('4', column_create(('cf:4a', '301'), ('cf:4b', '302')));
 
 --- 1 row(s) inserted.
->>select left(row_id, 10) row_id, left(column_display(column_details), 40) 
-+>           from hbase."_ROW_".t022hbt1 where row_id = '4';
+>>select left(row_id, 10) row_id, left(column_display(column_details), 40)
++> from hbase."_ROW_".t022hbt1 where row_id = '4';
 
-ROW_ID      (EXPR)                                  
-----------  ----------------------------------------
+ROW_ID (EXPR)
+----------
 
-4           cf:4a => 301, cf:4b => 302              
+4 cf:4a => 301, cf:4b => 302
 
 --- 1 row(s) selected.
 >>rollback work;
@@ -185,93 +185,93 @@ ROW_ID      (EXPR)
 >>
 >>select col_family, col_name, left(col_value, 20) from hbase."_CELL_".t022hbt1;
 
-COL_FAMILY                                                                                            COL_NAME                                                                                              (EXPR)
-----------------------------------------------------------------------------------------------------  ----------------------------------------------------------------------------------------------------  --------------------
+COL_FAMILY COL_NAME (EXPR)
+----------
 
-cf                                                                                                    1a                                                                                                    101                 
-cf                                                                                                    1b                                                                                                    102                 
-cf                                                                                                    2b                                                                                                    201                 
-cf                                                                                                    3a                                                                                                    3010103             
-cf                                                                                                    3b                                                                                                    302                 
+cf 1a 101
+cf 1b 102
+cf 2b 201
+cf 3a 3010103
+cf 3b 302
 
 --- 5 row(s) selected.
->>select left(row_id, 10) row_id, left(column_display(column_details), 40) 
-+>           from hbase."_ROW_".t022hbt1 where row_id = '4';
+>>select left(row_id, 10) row_id, left(column_display(column_details), 40)
++> from hbase."_ROW_".t022hbt1 where row_id = '4';
 
 --- 0 row(s) selected.
 >>
 >>select left(column_lookup (column_details, 'cf:1a'), 20) from hbase."_ROW_".t022hbt1;
 
-(EXPR)              
---------------------
+(EXPR)
+----------
 
-101                 
+101
 
 --- 1 row(s) selected.
 >>
 >>select left(column_lookup (column_details, 'cf:3a'), 20) from hbase."_ROW_".t022hbt1
-+>  where column_lookup (column_details, 'cf:3b') = '302';
++> where column_lookup (column_details, 'cf:3b') = '302';
 
-(EXPR)              
---------------------
+(EXPR)
+----------
 
-3010103             
+3010103
 
 --- 1 row(s) selected.
 >>
 >>select left(column_lookup (column_details, 'cf:3a'), 20) from hbase."_ROW_".t022hbt1
-+>  where column_lookup (column_details, 'cf:3a') = '301' 
-+>     and column_lookup (column_details, 'cf:3b') = '302';
++> where column_lookup (column_details, 'cf:3a') = '301'
++> and column_lookup (column_details, 'cf:3b') = '302';
 
 --- 0 row(s) selected.
 >>
 >>select left(column_lookup (column_details, 'cf:3a'), 20) from hbase."_ROW_".t022hbt1
-+>  where column_lookup (column_details, 'cf:3a') = '3010103' 
-+>     and column_lookup (column_details, 'cf:3b') = '302';
++> where column_lookup (column_details, 'cf:3a') = '3010103'
++> and column_lookup (column_details, 'cf:3b') = '302';
 
-(EXPR)              
---------------------
+(EXPR)
+----------
 
-3010103             
+3010103
 
 --- 1 row(s) selected.
 >>
 >>select left(column_lookup (column_details, 'cf:3c'), 20) from hbase."_ROW_".t022hbt1
-+>  where column_lookup (column_details, 'cf:3b') = '3020';
++> where column_lookup (column_details, 'cf:3b') = '3020';
 
 --- 0 row(s) selected.
 >>
 >>select left(column_lookup (column_details, 'cf:3c'), 20) from hbase."_ROW_".t022hbt1
-+>  where column_lookup (column_details, 'cf:3b') = '302';
++> where column_lookup (column_details, 'cf:3b') = '302';
 
-(EXPR)              
---------------------
+(EXPR)
+----------
 
-?                   
+?
 
 --- 1 row(s) selected.
 >>
 >>select column_lookup (column_details, 'cf:3a', cast as int),
-+>       column_lookup (column_details, 'cf:3a', cast as int) + 1 
-+>  from hbase."_ROW_".t022hbt1;
++> column_lookup (column_details, 'cf:3a', cast as int) + 1
++> from hbase."_ROW_".t022hbt1;
 
-(EXPR)       (EXPR)              
------------  --------------------
+(EXPR) (EXPR)
+----------
 
-    3010103               3010104
+ 3010103 3010104
 
 --- 1 row(s) selected.
 >>
 >>select column_lookup (column_details, 'cf:3a', cast as int),
-+>       column_lookup (column_details, 'cf:3a', cast as int) + 1 
-+>  from hbase."_ROW_".t022hbt1
-+>   where 
-+>       column_lookup (column_details, 'cf:3b', cast as int) = 302;
++> column_lookup (column_details, 'cf:3a', cast as int) + 1
++> from hbase."_ROW_".t022hbt1
++> where
++> column_lookup (column_details, 'cf:3b', cast as int) = 302;
 
-(EXPR)       (EXPR)              
------------  --------------------
+(EXPR) (EXPR)
+----------
 
-    3010103               3010104
+ 3010103 3010104
 
 --- 1 row(s) selected.
 >>
@@ -283,11 +283,11 @@ cf
 --- 1 row(s) deleted.
 >>select left(row_id, 10) from hbase."_ROW_".t022hbt1;
 
-(EXPR)    
+(EXPR)
 ----------
 
-1         
-2         
+1
+2
 
 --- 2 row(s) selected.
 >>commit work;
@@ -295,25 +295,25 @@ cf
 --- SQL operation complete.
 >>select left(row_id, 10) from hbase."_ROW_".t022hbt1;
 
-(EXPR)    
+(EXPR)
 ----------
 
-1         
-2         
+1
+2
 
 --- 2 row(s) selected.
 >>
->>delete columns ('cf:1a') from hbase."_ROW_".t022hbt1  where row_id  = '1';
+>>delete columns ('cf:1a') from hbase."_ROW_".t022hbt1 where row_id = '1';
 
 --- 1 row(s) deleted.
 >>select left(row_id, 10) row_id, column_display(column_details, 40) column_details
-+>           from hbase."_ROW_".t022hbt1;
++> from hbase."_ROW_".t022hbt1;
 
-ROW_ID      COLUMN_DETAILS                          
-----------  ----------------------------------------
+ROW_ID COLUMN_DETAILS
+----------
 
-1           cf:1b => 102                            
-2           cf:2b => 201                            
+1 cf:1b => 102
+2 cf:2b => 201
 
 --- 2 row(s) selected.
 >>
@@ -322,14 +322,14 @@ ROW_ID      COLUMN_DETAILS
 --- 2 row(s) deleted.
 >>select count(*) from hbase."_ROW_".t022hbt1;
 
-(EXPR)              
---------------------
+(EXPR)
+----------
 
-                   0
+ 0
 
 --- 1 row(s) selected.
 >>
->>-- operations between trafodion and hbase tables
+>>-- operations between TRAFODION and hbase tables
 >>begin work;
 
 --- SQL operation complete.
@@ -341,18 +341,18 @@ ROW_ID      COLUMN_DETAILS
 --- 1 row(s) inserted.
 >>select count(*) from t022t1;
 
-(EXPR)              
---------------------
+(EXPR)
+----------
 
-                   1
+ 1
 
 --- 1 row(s) selected.
 >>select count(*) from hbase."_ROW_".t022hbt1;
 
-(EXPR)              
---------------------
+(EXPR)
+----------
 
-                   1
+ 1
 
 --- 1 row(s) selected.
 >>rollback work;
@@ -360,40 +360,40 @@ ROW_ID      COLUMN_DETAILS
 --- SQL operation complete.
 >>select count(*) from t022t1;
 
-(EXPR)              
---------------------
+(EXPR)
+----------
 
-                   0
+ 0
 
 --- 1 row(s) selected.
 >>select count(*) from hbase."_ROW_".t022hbt1;
 
-(EXPR)              
---------------------
+(EXPR)
+----------
 
-                   0
+ 0
 
 --- 1 row(s) selected.
 >>
->>insert into hbase."_ROW_".t022hbt1 values 
-+>       ('3', column_create(('cf:3a', '301'), ('cf:3b', '302')));
+>>insert into hbase."_ROW_".t022hbt1 values
++> ('3', column_create(('cf:3a', '301'), ('cf:3b', '302')));
 
 --- 1 row(s) inserted.
 >>update hbase."_ROW_".t022hbt1 set column_details = column_create('cf:3b', '3022')
-+>  where row_id = '3' and column_lookup(column_details, 'cf:3a', cast as int) = 3011;
++> where row_id = '3' and column_lookup(column_details, 'cf:3a', cast as int) = 3011;
 
 --- 0 row(s) updated.
 >>update hbase."_ROW_".t022hbt1 set column_details = column_create('cf:3b', '3022')
-+>  where row_id = '3' and column_lookup(column_details, 'cf:3a', cast as int) = 301;
++> where row_id = '3' and column_lookup(column_details, 'cf:3a', cast as int) = 301;
 
 --- 1 row(s) updated.
->>select left(row_id, 10) row_id, left(column_display(column_details), 40) 
-+>           from hbase."_ROW_".t022hbt1;
+>>select left(row_id, 10) row_id, left(column_display(column_details), 40)
++> from hbase."_ROW_".t022hbt1;
 
-ROW_ID      (EXPR)                                  
-----------  ----------------------------------------
+ROW_ID (EXPR)
+----------
 
-3           cf:3a => 301, cf:3b => 3022             
+3 cf:3a => 301, cf:3b => 3022
 
 --- 1 row(s) selected.
 >>
@@ -406,7 +406,7 @@ ROW_ID      (EXPR)
 
 >>insert into hbase."_ROW_".t022hbt1 values ('2', column_create(':b', '201'));
 
-*** ERROR[1426] An invalid HBase column name :b was specified. A valid name must be of the format:   <ColumnFamily>:<ColumnName>
+*** ERROR[1426] An invalid HBase column name :b was specified. A valid name must be of the format: <ColumnFamily>:<ColumnName>
 
 --- 0 row(s) inserted.
 >>insert into hbase."_ROW_".t022hbt1 values ('2', '100');
@@ -425,35 +425,35 @@ ROW_ID      (EXPR)
 >>-- tests with large rows
 >>invoke hbase."_CELL_".t022hbt2;
 
--- Definition of Trafodion table HBASE."_CELL_".T022HBT2
--- Definition current  Tue May  2 16:52:50 2017
+-- Definition of table HBASE."_CELL_".T022HBT2
+-- Definition current
 
-  (
-    ROW_ID                           VARCHAR(100) CHARACTER SET ISO88591
-      COLLATE DEFAULT NO DEFAULT NOT NULL NOT DROPPABLE
-  , COL_FAMILY                       VARCHAR(100) CHARACTER SET ISO88591
-      COLLATE DEFAULT NO DEFAULT NOT NULL NOT DROPPABLE
-  , COL_NAME                         VARCHAR(100) CHARACTER SET ISO88591
-      COLLATE DEFAULT NO DEFAULT NOT NULL NOT DROPPABLE
-  , COL_TIMESTAMP                    LARGEINT NO DEFAULT NOT NULL NOT DROPPABLE
-  , COL_VALUE                        VARCHAR(1000) CHARACTER SET ISO88591
-      COLLATE DEFAULT NO DEFAULT NOT NULL NOT DROPPABLE
-  )
-  PRIMARY KEY (ROW_ID ASC)
+ (
+ ROW_ID VARCHAR(100) CHARACTER SET ISO88591
+ COLLATE DEFAULT NO DEFAULT NOT NULL NOT DROPPABLE
+ , COL_FAMILY VARCHAR(100) CHARACTER SET ISO88591
+ COLLATE DEFAULT NO DEFAULT NOT NULL NOT DROPPABLE
+ , COL_NAME VARCHAR(100) CHARACTER SET ISO88591
+ COLLATE DEFAULT NO DEFAULT NOT NULL NOT DROPPABLE
+ , COL_TIMESTAMP LARGEINT NO DEFAULT NOT NULL NOT DROPPABLE
+ , COL_VALUE VARCHAR(1000) CHARACTER SET ISO88591
+ COLLATE DEFAULT NO DEFAULT NOT NULL NOT DROPPABLE
+ )
+ PRIMARY KEY (ROW_ID ASC)
 
 --- SQL operation complete.
 >>invoke hbase."_ROW_".t022hbt2;
 
--- Definition of Trafodion table HBASE."_ROW_".T022HBT2
--- Definition current  Tue May  2 16:52:50 2017
+-- Definition of table HBASE."_ROW_".T022HBT2
+-- Definition current
 
-  (
-    ROW_ID                           VARCHAR(100) CHARACTER SET ISO88591
-      COLLATE DEFAULT NO DEFAULT NOT NULL NOT DROPPABLE
-  , COLUMN_DETAILS                   VARCHAR(10000) CHARACTER SET ISO88591
-      COLLATE DEFAULT NO DEFAULT NOT NULL NOT DROPPABLE
-  )
-  PRIMARY KEY (ROW_ID ASC)
+ (
+ ROW_ID VARCHAR(100) CHARACTER SET ISO88591
+ COLLATE DEFAULT NO DEFAULT NOT NULL NOT DROPPABLE
+ , COLUMN_DETAILS VARCHAR(10000) CHARACTER SET ISO88591
+ COLLATE DEFAULT NO DEFAULT NOT NULL NOT DROPPABLE
+ )
+ PRIMARY KEY (ROW_ID ASC)
 
 --- SQL operation complete.
 >>cqd hbase_max_column_val_length '100000';
@@ -464,107 +464,107 @@ ROW_ID      (EXPR)
 --- SQL operation complete.
 >>invoke hbase."_CELL_".t022hbt2;
 
--- Definition of Trafodion table HBASE."_CELL_".T022HBT2
--- Definition current  Tue May  2 16:52:50 2017
+-- Definition of table HBASE."_CELL_".T022HBT2
+-- Definition current
 
-  (
-    ROW_ID                           VARCHAR(100) CHARACTER SET ISO88591
-      COLLATE DEFAULT NO DEFAULT NOT NULL NOT DROPPABLE
-  , COL_FAMILY                       VARCHAR(100) CHARACTER SET ISO88591
-      COLLATE DEFAULT NO DEFAULT NOT NULL NOT DROPPABLE
-  , COL_NAME                         VARCHAR(100) CHARACTER SET ISO88591
-      COLLATE DEFAULT NO DEFAULT NOT NULL NOT DROPPABLE
-  , COL_TIMESTAMP                    LARGEINT NO DEFAULT NOT NULL NOT DROPPABLE
-  , COL_VALUE                        VARCHAR(100000) CHARACTER SET ISO88591
-      COLLATE DEFAULT NO DEFAULT NOT NULL NOT DROPPABLE
-  )
-  PRIMARY KEY (ROW_ID ASC)
+ (
+ ROW_ID VARCHAR(100) CHARACTER SET ISO88591
+ COLLATE DEFAULT NO DEFAULT NOT NULL NOT DROPPABLE
+ , COL_FAMILY VARCHAR(100) CHARACTER SET ISO88591
+ COLLATE DEFAULT NO DEFAULT NOT NULL NOT DROPPABLE
+ , COL_NAME VARCHAR(100) CHARACTER SET ISO88591
+ COLLATE DEFAULT NO DEFAULT NOT NULL NOT DROPPABLE
+ , COL_TIMESTAMP LARGEINT NO DEFAULT NOT NULL NOT DROPPABLE
+ , COL_VALUE VARCHAR(100000) CHARACTER SET ISO88591
+ COLLATE DEFAULT NO DEFAULT NOT NULL NOT DROPPABLE
+ )
+ PRIMARY KEY (ROW_ID ASC)
 
 --- SQL operation complete.
 >>invoke hbase."_ROW_".t022hbt2;
 
--- Definition of Trafodion table HBASE."_ROW_".T022HBT2
--- Definition current  Tue May  2 16:52:51 2017
+-- Definition of table HBASE."_ROW_".T022HBT2
+-- Definition current
 
-  (
-    ROW_ID                           VARCHAR(100) CHARACTER SET ISO88591
-      COLLATE DEFAULT NO DEFAULT NOT NULL NOT DROPPABLE
-  , COLUMN_DETAILS                   VARCHAR(100000) CHARACTER SET ISO88591
-      COLLATE DEFAULT NO DEFAULT NOT NULL NOT DROPPABLE
-  )
-  PRIMARY KEY (ROW_ID ASC)
+ (
+ ROW_ID VARCHAR(100) CHARACTER SET ISO88591
+ COLLATE DEFAULT NO DEFAULT NOT NULL NOT DROPPABLE
+ , COLUMN_DETAILS VARCHAR(100000) CHARACTER SET ISO88591
+ COLLATE DEFAULT NO DEFAULT NOT NULL NOT DROPPABLE
+ )
+ PRIMARY KEY (ROW_ID ASC)
 
 --- SQL operation complete.
 >>insert into hbase."_CELL_".t022hbt2 values ('1', 'cf', 'a', -1, repeat('a', 100000));
 
 --- 1 row(s) inserted.
->>insert into hbase."_ROW_".t022hbt2 values 
-+>                              ('2', column_create(
-+>                                       ('cf:1a', repeat('a', 40000)),
-+>                                       ('cf:1b', repeat('z', 40000))));
+>>insert into hbase."_ROW_".t022hbt2 values
++> ('2', column_create(
++> ('cf:1a', repeat('a', 40000)),
++> ('cf:1b', repeat('z', 40000))));
 
 --- 1 row(s) inserted.
 >>select count(*) from hbase."_CELL_".t022hbt2;
 
-(EXPR)              
---------------------
+(EXPR)
+----------
 
-                   3
+ 3
 
 --- 1 row(s) selected.
->>select char_length(col_value) from hbase."_CELL_".t022hbt2;
+>>select char_length(col_value) from hbase."_CELL_".t022hbt2 order by 1;
 
-(EXPR)    
+(EXPR)
 ----------
 
-    100000
-     40000
-     40000
+ 40000
+ 40000
+ 100000
 
 --- 3 row(s) selected.
 >>select count(*) from hbase."_CELL_".t022hbt2;
 
-(EXPR)              
---------------------
+(EXPR)
+----------
 
-                   3
+ 3
 
 --- 1 row(s) selected.
->>select char_length(col_value) from hbase."_CELL_".t022hbt2;
+>>select char_length(col_value) from hbase."_CELL_".t022hbt2 order by 1;
 
-(EXPR)    
+(EXPR)
 ----------
 
-    100000
-     40000
-     40000
+ 40000
+ 40000
+ 100000
 
 --- 3 row(s) selected.
 >>select count(*) from hbase."_ROW_".t022hbt2;
 
-(EXPR)              
---------------------
+(EXPR)
+----------
 
-                   2
+ 2
 
 --- 1 row(s) selected.
->>select char_length(column_details) from hbase."_ROW_".t022hbt2;
+>>select char_length(column_details) from hbase."_ROW_".t022hbt2 order by 1;
 
-(EXPR)    
+(EXPR)
 ----------
 
-    100000
-     80022
+ 80022
+ 100000
 
 --- 2 row(s) selected.
->>select left(row_id, 10) row_id, left(column_display(column_details), 40) 
-+>           from hbase."_ROW_".t022hbt2;
+>>select left(row_id, 10) row_id, left(column_display(column_details), 40)
++> from hbase."_ROW_".t022hbt2;
 
-ROW_ID      (EXPR)                                  
-----------  ----------------------------------------
+ROW_ID (EXPR)
+----------
 
-1           cf:a => aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa
-2           cf:1a => aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa
+1 cf:a => aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa
+2 cf:1a => aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa
 
 --- 2 row(s) selected.
 >>
@@ -580,8 +580,8 @@ ROW_ID      (EXPR)
 >>create hbase table t022hbm1 (column family 'cf');
 
 --- SQL operation complete.
->>insert into hbase."_ROW_".t022hbm1 values ('a1', 
-+>                 column_create(('cf:B', '100 ')));
+>>insert into hbase."_ROW_".t022hbm1 values ('a1',
++> column_create(('cf:B', '100 ')));
 
 --- 1 row(s) inserted.
 >>insert into hbase."_ROW_".t022hbm1 values ('a2', column_create(('cf:A', 'a2')));
@@ -595,39 +595,39 @@ ROW_ID      (EXPR)
 
 --- SQL operation complete.
 >>create external table t022hbm1 (a varchar(4) not null, b char(4))
-+>        primary key (a)
-+>        attribute default column family 'cf'
-+>        map to hbase table t022hbm1;
++> primary key (a)
++> attribute default column family 'cf'
++> map to hbase table t022hbm1;
 
 --- SQL operation complete.
 >>invoke t022hbm1;
 
--- Definition of Trafodion HBase mapped table T022HBM1
--- Definition current  Tue May  2 16:53:06 2017
+-- Definition of TRAFODION HBase mapped table T022HBM1
+-- Definition current
 
-  (
-    "cf".A                           VARCHAR(4) CHARACTER SET ISO88591 COLLATE
-      DEFAULT NO DEFAULT NOT NULL NOT DROPPABLE
-  , "cf".B                           CHAR(4) CHARACTER SET ISO88591 COLLATE
-      DEFAULT DEFAULT NULL
-  )
-  PRIMARY KEY NOT SERIALIZED (A ASC)
+ (
+ "cf".A VARCHAR(4) CHARACTER SET ISO88591 COLLATE
+ DEFAULT NO @eof@ NULL NOT DROPPABLE
+ , "cf".B CHAR(4) CHARACTER SET ISO88591 COLLATE
+ DEFAULT DEFAULT NULL
+ )
+ PRIMARY KEY NOT SERIALIZED (A ASC)
 
 --- SQL operation complete.
 >>
 >>-- if no schema is specified, table is looked in regular and then mapped schema
 >>invoke t022hbm1;
 
--- Definition of Trafodion HBase mapped table T022HBM1
--- Definition current  Tue May  2 16:53:07 2017
+-- Definition of TRAFODION HBase mapped table T022HBM1
+-- Definition current
 
-  (
-    "cf".A                           VARCHAR(4) CHARACTER SET ISO88591 COLLATE
-      DEFAULT NO DEFAULT NOT NULL NOT DROPPABLE
-  , "cf".B                           CHAR(4) CHARACTER SET ISO88591 COLLATE
-      DEFAULT DEFAULT NULL
-  )
-  PRIMARY KEY NOT SERIALIZED (A ASC)
+ (
+ "cf".A VARCHAR(4) CHARACTER SET ISO88591 COLLATE
+ DEFAULT NO @eof@ NULL NOT DROPPABLE
+ , "cf".B CHAR(4) CHARACTER SET ISO88591 COLLATE
+ DEFAULT DEFAULT NULL
+ )
+ PRIMARY KEY NOT SERIALIZED (A ASC)
 
 --- SQL operation complete.
 >>create table t022hbm1 (a int);
@@ -635,13 +635,13 @@ ROW_ID      (EXPR)
 --- SQL operation complete.
 >>invoke t022hbm1;
 
--- Definition of Trafodion table TRAFODION.SEABASE.T022HBM1
--- Definition current  Tue May  2 16:53:11 2017
+-- Definition of table #CAT.#SCH.T022HBM1
+-- Definition current
 
-  (
-    SYSKEY                           LARGEINT NO DEFAULT NOT NULL NOT DROPPABLE
-  , A                                INT DEFAULT NULL
-  )
+ (
+ SYSKEY LARGEINT NO DEFAULT NOT NULL NOT DROPPABLE
+ , A INT DEFAULT NULL
+ )
 
 --- SQL operation complete.
 >>drop table t022hbm1;
@@ -649,16 +649,16 @@ ROW_ID      (EXPR)
 --- SQL operation complete.
 >>invoke t022hbm1;
 
--- Definition of Trafodion HBase mapped table T022HBM1
--- Definition current  Tue May  2 16:53:20 2017
+-- Definition of TRAFODION HBase mapped table T022HBM1
+-- Definition current
 
-  (
-    "cf".A                           VARCHAR(4) CHARACTER SET ISO88591 COLLATE
-      DEFAULT NO DEFAULT NOT NULL NOT DROPPABLE
-  , "cf".B                           CHAR(4) CHARACTER SET ISO88591 COLLATE
-      DEFAULT DEFAULT NULL
-  )
-  PRIMARY KEY NOT SERIALIZED (A ASC)
+ (
+ "cf".A VARCHAR(4) CHARACTER SET ISO88591 COLLATE
+ DEFAULT NO @eof@ NULL NOT DROPPABLE
+ , "cf".B CHAR(4) CHARACTER SET ISO88591 COLLATE
+ DEFAULT DEFAULT NULL
+ )
+ PRIMARY KEY NOT SERIALIZED (A ASC)
 
 --- SQL operation complete.
 >>
@@ -667,28 +667,28 @@ ROW_ID      (EXPR)
 --- SQL command prepared.
 >>execute s;
 
-A     B   
-----  ----
+A B
+---- ----
 
-a1    100 
-a2    ?   
+a1 100
+a2 ?
 
 --- 2 row(s) selected.
 >>select * from t022hbm1 where a = 'a1';
 
-A     B   
-----  ----
+A B
+---- ----
 
-a1    100 
+a1 100
 
 --- 1 row(s) selected.
 >>select a, cast(b as int) from t022hbm1;
 
-A     (EXPR)     
-----  -----------
+A (EXPR)
+---- -----------
 
-a1            100
-a2              ?
+a1 100
+a2 ?
 
 --- 2 row(s) selected.
 >>
@@ -697,17 +697,17 @@ a2              ?
 --- SQL operation complete.
 >>invoke t022hbm1;
 
--- Definition of Trafodion HBase mapped table T022HBM1
--- Definition current  Tue May  2 16:53:25 2017
+-- Definition of TRAFODION HBase mapped table T022HBM1
+-- Definition current
 
-  (
-    "cf".A                           VARCHAR(4) CHARACTER SET ISO88591 COLLATE
-      DEFAULT NO DEFAULT NOT NULL NOT DROPPABLE
-  , "cf".B                           CHAR(4) CHARACTER SET ISO88591 COLLATE
-      DEFAULT DEFAULT NULL
-  , "cf".C                           INT DEFAULT NULL /*added_col*/
-  )
-  PRIMARY KEY NOT SERIALIZED (A ASC)
+ (
+ "cf".A VARCHAR(4) CHARACTER SET ISO88591 COLLATE
+ DEFAULT NO @eof@ NULL NOT DROPPABLE
+ , "cf".B CHAR(4) CHARACTER SET ISO88591 COLLATE
+ DEFAULT DEFAULT NULL
+ , "cf".C INT DEFAULT NULL /*added_col*/
+ )
+ PRIMARY KEY NOT SERIALIZED (A ASC)
 
 --- SQL operation complete.
 >>
@@ -720,17 +720,17 @@ a2              ?
 --- SQL operation complete.
 >>invoke t022hbm1_like;
 
--- Definition of Trafodion table TRAFODION.SEABASE.T022HBM1_LIKE
--- Definition current  Tue May  2 16:53:33 2017
+-- Definition of table #CAT.#SCH.T022HBM1_LIKE
+-- Definition current
 
-  (
-    "cf".A                           VARCHAR(4) CHARACTER SET ISO88591 COLLATE
-      DEFAULT NO DEFAULT NOT NULL NOT DROPPABLE
-  , "cf".B                           CHAR(4) CHARACTER SET ISO88591 COLLATE
-      DEFAULT DEFAULT NULL
-  , "cf".C                           INT DEFAULT NULL
-  )
-  PRIMARY KEY (A ASC)
+ (
+ "cf".A VARCHAR(4) CHARACTER SET ISO88591 COLLATE
+ DEFAULT NO @eof@ NULL NOT DROPPABLE
+ , "cf".B CHAR(4) CHARACTER SET ISO88591 COLLATE
+ DEFAULT DEFAULT NULL
+ , "cf".C INT DEFAULT NULL
+ )
+ PRIMARY KEY (A ASC)
 
 --- SQL operation complete.
 >>
@@ -739,11 +739,11 @@ a2              ?
 --- 2 row(s) inserted.
 >>select * from t022hbm1_like;
 
-A     B     C          
-----  ----  -----------
+A B C
+---- ---- -----------
 
-a1    100             ?
-a2    ?               ?
+a1 100 ?
+a2 ? ?
 
 --- 2 row(s) selected.
 >>
@@ -752,11 +752,11 @@ a2    ?               ?
 --- 2 row(s) inserted.
 >>select * from t022hbm1_ctas;
 
-A     B     C          
-----  ----  -----------
+A B C
+---- ---- -----------
 
-a1    100             ?
-a2    ?               ?
+a1 100 ?
+a2 ? ?
 
 --- 2 row(s) selected.
 >>
@@ -770,26 +770,26 @@ a2    ?               ?
 --- SQL operation complete.
 >>invoke t022hbm1;
 
-*** ERROR[4082] Object TRAFODION.SEABASE.T022HBM1 does not exist or is inaccessible.
+*** ERROR[4082] Object #CAT.#SCH.T022HBM1 does not exist or is inaccessible.
 
 --- SQL operation failed with errors.
 >>create external table t022hbm1 ("cf".a varchar(4) not null,
-+>            b int)
-+>        primary key (a)
-+>        map to hbase table t022hbm1;
++> b int)
++> primary key (a)
++> map to hbase table t022hbm1;
 
 --- SQL operation complete.
 >>invoke t022hbm1;
 
--- Definition of Trafodion HBase mapped table T022HBM1
--- Definition current  Tue May  2 16:53:49 2017
+-- Definition of TRAFODION HBase mapped table T022HBM1
+-- Definition current
 
-  (
-    "cf".A                           VARCHAR(4) CHARACTER SET ISO88591 COLLATE
-      DEFAULT NO DEFAULT NOT NULL NOT DROPPABLE
-  , B                                INT DEFAULT NULL
-  )
-  PRIMARY KEY NOT SERIALIZED (A ASC)
+ (
+ "cf".A VARCHAR(4) CHARACTER SET ISO88591 COLLATE
+ DEFAULT NO @eof@ NULL NOT DROPPABLE
+ , B INT DEFAULT NULL
+ )
+ PRIMARY KEY NOT SERIALIZED (A ASC)
 
 --- SQL operation complete.
 >>
@@ -798,14 +798,14 @@ a2    ?               ?
 --- SQL operation complete.
 >>invoke t022hbm1;
 
--- Definition of Trafodion HBase mapped table T022HBM1
--- Definition current  Tue May  2 16:53:54 2017
+-- Definition of TRAFODION HBase mapped table T022HBM1
+-- Definition current
 
-  (
-    "cf".A                           VARCHAR(4) CHARACTER SET ISO88591 COLLATE
-      DEFAULT NO DEFAULT NOT NULL NOT DROPPABLE
-  )
-  PRIMARY KEY NOT SERIALIZED (A ASC)
+ (
+ "cf".A VARCHAR(4) CHARACTER SET ISO88591 COLLATE
+ DEFAULT NO @eof@ NULL NOT DROPPABLE
+ )
+ PRIMARY KEY NOT SERIALIZED (A ASC)
 
 --- SQL operation complete.
 >>
@@ -814,15 +814,15 @@ a2    ?               ?
 --- SQL operation complete.
 >>invoke t022hbm1;
 
--- Definition of Trafodion HBase mapped table T022HBM1
--- Definition current  Tue May  2 16:53:56 2017
+-- Definition of TRAFODION HBase mapped table T022HBM1
+-- Definition current
 
-  (
-    "cf".A                           VARCHAR(4) CHARACTER SET ISO88591 COLLATE
-      DEFAULT NO DEFAULT NOT NULL NOT DROPPABLE
-  , "cf".B                           INT DEFAULT NULL /*added_col*/
-  )
-  PRIMARY KEY NOT SERIALIZED (A ASC)
+ (
+ "cf".A VARCHAR(4) CHARACTER SET ISO88591 COLLATE
+ DEFAULT NO @eof@ NULL NOT DROPPABLE
+ , "cf".B INT DEFAULT NULL /*added_col*/
+ )
+ PRIMARY KEY NOT SERIALIZED (A ASC)
 
 --- SQL operation complete.
 >>
@@ -840,11 +840,11 @@ a2    ?               ?
 >>drop external table t022hbm1;
 
 --- SQL operation complete.
->>create external table t022hbm1 (a varchar(4) not null, b int) 
-+>        primary key (a)
-+>        attribute default column family 'cf'
-+>        map to hbase table t022hbm1
-+>        data format native;
+>>create external table t022hbm1 (a varchar(4) not null, b int)
++> primary key (a)
++> attribute default column family 'cf'
++> map to hbase table t022hbm1
++> data format native;
 
 --- SQL operation complete.
 >>
@@ -853,10 +853,10 @@ a2    ?               ?
 --- 1 row(s) inserted.
 >>select * from t022hbm1;
 
-A     B          
-----  -----------
+A B
+---- -----------
 
-a               1
+a 1
 
 --- 1 row(s) selected.
 >>update t022hbm1 set b = b + 1;
@@ -864,10 +864,10 @@ a               1
 --- 1 row(s) updated.
 >>select * from t022hbm1;
 
-A     B          
-----  -----------
+A B
+---- -----------
 
-a               2
+a 2
 
 --- 1 row(s) selected.
 >>insert into t022hbm1 values ('a', 1);
@@ -881,11 +881,11 @@ a               2
 --- 1 row(s) inserted.
 >>select * from t022hbm1;
 
-A     B          
-----  -----------
+A B
+---- -----------
 
-a               2
-b               ?
+a 2
+b ?
 
 --- 2 row(s) selected.
 >>delete from t022hbm1 where a = 'a';
@@ -893,10 +893,10 @@ b               ?
 --- 1 row(s) deleted.
 >>select * from t022hbm1;
 
-A     B          
-----  -----------
+A B
+---- -----------
 
-b               ?
+b ?
 
 --- 1 row(s) selected.
 >>update t022hbm1 set b = 10;
@@ -904,10 +904,10 @@ b               ?
 --- 1 row(s) updated.
 >>select * from t022hbm1;
 
-A     B          
-----  -----------
+A B
+---- -----------
 
-b              10
+b 10
 
 --- 1 row(s) selected.
 >>update t022hbm1 set b = null;
@@ -915,10 +915,10 @@ b              10
 --- 1 row(s) updated.
 >>select * from t022hbm1;
 
-A     B          
-----  -----------
+A B
+---- -----------
 
-b               ?
+b ?
 
 --- 1 row(s) selected.
 >>delete from t022hbm1;
@@ -931,24 +931,24 @@ b               ?
 >>drop external table t022hbm1;
 
 --- SQL operation complete.
->>create external table t022hbm1 (a int not null, b int not null, c int) 
-+>        primary key (a, b)
-+>        attribute default column family 'cf'
-+>        map to hbase table t022hbm1
-+>        data format native;
+>>create external table t022hbm1 (a int not null, b int not null, c int)
++> primary key (a, b)
++> attribute default column family 'cf'
++> map to hbase table t022hbm1
++> data format native;
 
 --- SQL operation complete.
 >>invoke t022hbm1;
 
--- Definition of Trafodion HBase mapped table T022HBM1
--- Definition current  Tue May  2 16:54:08 2017
+-- Definition of TRAFODION HBase mapped table T022HBM1
+-- Definition current
 
-  (
-    "cf".A                           INT NO DEFAULT NOT NULL NOT DROPPABLE
-  , "cf".B                           INT NO DEFAULT NOT NULL NOT DROPPABLE
-  , "cf".C                           INT DEFAULT NULL
-  )
-  PRIMARY KEY NOT SERIALIZED (A ASC, B ASC)
+ (
+ "cf".A INT NO DEFAULT NOT NULL NOT DROPPABLE
+ , "cf".B INT NO DEFAULT NOT NULL NOT DROPPABLE
+ , "cf".C INT DEFAULT NULL
+ )
+ PRIMARY KEY NOT SERIALIZED (A ASC, B ASC)
 
 --- SQL operation complete.
 >>insert into t022hbm1 values (1,2,3);
@@ -965,27 +965,27 @@ b               ?
 >>insert into t022hbm1 values (-1, -2, -3);
 
 --- 1 row(s) inserted.
->>select * from t022hbm1;
+>>select * from t022hbm1 order by 1;
 
-A            B            C          
------------  -----------  -----------
+A B C
+----------
 
-          1            1            1
-          1            2            3
-         -1           -2           -3
+ 1 1 1
+ 1 2 3
+ -1 -2 -3
 
 --- 3 row(s) selected.
 >>upsert into t022hbm1 values (1,2,4);
 
 --- 1 row(s) inserted.
->>select * from t022hbm1;
+>>select * from t022hbm1 order by 1;
 
-A            B            C          
------------  -----------  -----------
+A B C
+----------
 
-          1            1            1
-          1            2            4
-         -1           -2           -3
+ 1 1 1
+ 1 2 4
+ -1 -2 -3
 
 --- 3 row(s) selected.
 >>
@@ -993,159 +993,159 @@ A            B            C
 >>drop external table t022hbm1;
 
 --- SQL operation complete.
->>create external table t022hbm1 (a varchar(4) not null, primary key not serialized (a), b int) 
-+>        attribute default column family 'cf'
-+>        map to hbase table t022hbm1;
+>>create external table t022hbm1 (a varchar(4) not null, primary key not serialized (a), b int)
++> attribute default column family 'cf'
++> map to hbase table t022hbm1;
 
 --- SQL operation complete.
 >>invoke t022hbm1;
 
--- Definition of Trafodion HBase mapped table T022HBM1
--- Definition current  Tue May  2 16:54:12 2017
+-- Definition of TRAFODION HBase mapped table T022HBM1
+-- Definition current
 
-  (
-    "cf".A                           VARCHAR(4) CHARACTER SET ISO88591 COLLATE
-      DEFAULT NO DEFAULT NOT NULL NOT DROPPABLE
-  , "cf".B                           INT DEFAULT NULL
-  )
-  PRIMARY KEY NOT SERIALIZED (A ASC)
+ (
+ "cf".A VARCHAR(4) CHARACTER SET ISO88591 COLLATE
+ DEFAULT NO @eof@ NULL NOT DROPPABLE
+ , "cf".B INT DEFAULT NULL
+ )
+ PRIMARY KEY NOT SERIALIZED (A ASC)
 
 --- SQL operation complete.
 >>
 >>drop external table t022hbm1;
 
 --- SQL operation complete.
->>create external table t022hbm1 (a char(4) not null, primary key serialized (a), b int) 
-+>        attribute default column family 'cf'
-+>        map to hbase table t022hbm1
-+>        data format native;
+>>create external table t022hbm1 (a char(4) not null, primary key serialized (a), b int)
++> attribute default column family 'cf'
++> map to hbase table t022hbm1
++> data format native;
 
 --- SQL operation complete.
 >>invoke t022hbm1;
 
--- Definition of Trafodion HBase mapped table T022HBM1
--- Definition current  Tue May  2 16:54:18 2017
+-- Definition of TRAFODION HBase mapped table T022HBM1
+-- Definition current
 
-  (
-    "cf".A                           CHAR(4) CHARACTER SET ISO88591 COLLATE
-      DEFAULT NO DEFAULT NOT NULL NOT DROPPABLE
-  , "cf".B                           INT DEFAULT NULL
-  )
-  PRIMARY KEY (A ASC)
+ (
+ "cf".A CHAR(4) CHARACTER SET ISO88591 COLLATE
+ DEFAULT NO @eof@ NULL NOT DROPPABLE
+ , "cf".B INT DEFAULT NULL
+ )
+ PRIMARY KEY (A ASC)
 
 --- SQL operation complete.
 >>
 >>drop external table t022hbm1;
 
 --- SQL operation complete.
->>create external table t022hbm1 (a varchar(4) not null, primary key(a), b int) 
-+>        attribute default column family 'cf'
-+>        map to hbase table t022hbm1;
+>>create external table t022hbm1 (a varchar(4) not null, primary key(a), b int)
++> attribute default column family 'cf'
++> map to hbase table t022hbm1;
 
 --- SQL operation complete.
 >>invoke t022hbm1;
 
--- Definition of Trafodion HBase mapped table T022HBM1
--- Definition current  Tue May  2 16:54:24 2017
+-- Definition of TRAFODION HBase mapped table T022HBM1
+-- Definition current
 
-  (
-    "cf".A                           VARCHAR(4) CHARACTER SET ISO88591 COLLATE
-      DEFAULT NO DEFAULT NOT NULL NOT DROPPABLE
-  , "cf".B                           INT DEFAULT NULL
-  )
-  PRIMARY KEY NOT SERIALIZED (A ASC)
+ (
+ "cf".A VARCHAR(4) CHARACTER SET ISO88591 COLLATE
+ DEFAULT NO @eof@ NULL NOT DROPPABLE
+ , "cf".B INT DEFAULT NULL
+ )
+ PRIMARY KEY NOT SERIALIZED (A ASC)
 
 --- SQL operation complete.
 >>
 >>drop external table t022hbm1;
 
 --- SQL operation complete.
->>create external table t022hbm1 (a varchar(4) not null primary key, b int) 
-+>        attribute default column family 'cf'
-+>        map to hbase table t022hbm1;
+>>create external table t022hbm1 (a varchar(4) not null primary key, b int)
++> attribute default column family 'cf'
++> map to hbase table t022hbm1;
 
 --- SQL operation complete.
 >>invoke t022hbm1;
 
--- Definition of Trafodion HBase mapped table T022HBM1
--- Definition current  Tue May  2 16:54:28 2017
+-- Definition of TRAFODION HBase mapped table T022HBM1
+-- Definition current
 
-  (
-    "cf".A                           VARCHAR(4) CHARACTER SET ISO88591 COLLATE
-      DEFAULT NO DEFAULT NOT NULL NOT DROPPABLE
-  , "cf".B                           INT DEFAULT NULL
-  )
-  PRIMARY KEY NOT SERIALIZED (A ASC)
+ (
+ "cf".A VARCHAR(4) CHARACTER SET ISO88591 COLLATE
+ DEFAULT NO @eof@ NULL NOT DROPPABLE
+ , "cf".B INT DEFAULT NULL
+ )
+ PRIMARY KEY NOT SERIALIZED (A ASC)
 
 --- SQL operation complete.
 >>
 >>drop external table t022hbm1;
 
 --- SQL operation complete.
->>create external table t022hbm1 (a varchar(4) not null, b int) 
-+>        primary key not serialized (a)
-+>        attribute default column family 'cf'
-+>        map to hbase table t022hbm1;
+>>create external table t022hbm1 (a varchar(4) not null, b int)
++> primary key not serialized (a)
++> attribute default column family 'cf'
++> map to hbase table t022hbm1;
 
 --- SQL operation complete.
 >>invoke t022hbm1;
 
--- Definition of Trafodion HBase mapped table T022HBM1
--- Definition current  Tue May  2 16:54:34 2017
+-- Definition of TRAFODION HBase mapped table T022HBM1
+-- Definition current
 
-  (
-    "cf".A                           VARCHAR(4) CHARACTER SET ISO88591 COLLATE
-      DEFAULT NO DEFAULT NOT NULL NOT DROPPABLE
-  , "cf".B                           INT DEFAULT NULL
-  )
-  PRIMARY KEY NOT SERIALIZED (A ASC)
+ (
+ "cf".A VARCHAR(4) CHARACTER SET ISO88591 COLLATE
+ DEFAULT NO @eof@ NULL NOT DROPPABLE
+ , "cf".B INT DEFAULT NULL
+ )
+ PRIMARY KEY NOT SERIALIZED (A ASC)
 
 --- SQL operation complete.
 >>
 >>drop external table t022hbm1;
 
 --- SQL operation complete.
->>create external table t022hbm1 (a char(4) not null, b int) 
-+>        primary key serialized (a) 
-+>        attribute default column family 'cf'
-+>        map to hbase table t022hbm1
-+>        data format native;
+>>create external table t022hbm1 (a char(4) not null, b int)
++> primary key serialized (a)
++> attribute default column family 'cf'
++> map to hbase table t022hbm1
++> data format native;
 
 --- SQL operation complete.
 >>invoke t022hbm1;
 
--- Definition of Trafodion HBase mapped table T022HBM1
--- Definition current  Tue May  2 16:54:40 2017
+-- Definition of TRAFODION HBase mapped table T022HBM1
+-- Definition current
 
-  (
-    "cf".A                           CHAR(4) CHARACTER SET ISO88591 COLLATE
-      DEFAULT NO DEFAULT NOT NULL NOT DROPPABLE
-  , "cf".B                           INT DEFAULT NULL
-  )
-  PRIMARY KEY (A ASC)
+ (
+ "cf".A CHAR(4) CHARACTER SET ISO88591 COLLATE
+ DEFAULT NO @eof@ NULL NOT DROPPABLE
+ , "cf".B INT DEFAULT NULL
+ )
+ PRIMARY KEY (A ASC)
 
 --- SQL operation complete.
 >>
 >>drop external table t022hbm1;
 
 --- SQL operation complete.
->>create external table t022hbm1 (a varchar(4) not null, b int) 
-+>        primary key (a)
-+>        attribute default column family 'cf'
-+>        map to hbase table t022hbm1;
+>>create external table t022hbm1 (a varchar(4) not null, b int)
++> primary key (a)
++> attribute default column family 'cf'
++> map to hbase table t022hbm1;
 
 --- SQL operation complete.
 >>invoke t022hbm1;
 
--- Definition of Trafodion HBase mapped table T022HBM1
--- Definition current  Tue May  2 16:54:43 2017
+-- Definition of TRAFODION HBase mapped table T022HBM1
+-- Definition current
 
-  (
-    "cf".A                           VARCHAR(4) CHARACTER SET ISO88591 COLLATE
-      DEFAULT NO DEFAULT NOT NULL NOT DROPPABLE
-  , "cf".B                           INT DEFAULT NULL
-  )
-  PRIMARY KEY NOT SERIALIZED (A ASC)
+ (
+ "cf".A VARCHAR(4) CHARACTER SET ISO88591 COLLATE
+ DEFAULT NO @eof@ NULL NOT DROPPABLE
+ , "cf".B INT DEFAULT NULL
+ )
+ PRIMARY KEY NOT SERIALIZED (A ASC)
 
 --- SQL operation complete.
 >>
@@ -1158,10 +1158,10 @@ A            B            C
 >>drop external table if exists t022hbm1;
 
 --- SQL operation complete.
->>create external table t022hbm1 (a varchar(4) not null, b int) 
-+>        primary key (a) 
-+>        attribute default column family 'cf'
-+>        map to hbase table t022hbm1;
+>>create external table t022hbm1 (a varchar(4) not null, b int)
++> primary key (a)
++> attribute default column family 'cf'
++> map to hbase table t022hbm1;
 
 --- SQL operation complete.
 >>insert into hbase."_ROW_".t022hbm1 values ('a1', column_create(('cf:B', '10')));
@@ -1178,35 +1178,35 @@ A            B            C
 >>create hbase table t022hbm2 (column family 'cf1', column family 'cf2');
 
 --- SQL operation complete.
->>create external table t022hbm2 ("cf1".A int, "cf2".B int, 
-+>                           "cf1".Z varchar(4) not null primary key) 
-+>     map to hbase table t022hbm2;
+>>create external table t022hbm2 ("cf1".A int, "cf2".B int,
++> "cf1".Z varchar(4) not null primary key)
++> map to hbase table t022hbm2;
 
 --- SQL operation complete.
->>insert into hbase."_ROW_".t022hbm2 values ('a11', 
-+>           column_create(('cf1:A', '10'), ('cf2:B', '20')));
+>>insert into hbase."_ROW_".t022hbm2 values ('a11',
++> column_create(('cf1:A', '10'), ('cf2:B', '20')));
 
 --- 1 row(s) inserted.
 >>invoke t022hbm2;
 
--- Definition of Trafodion HBase mapped table T022HBM2
--- Definition current  Tue May  2 16:54:58 2017
+-- Definition of TRAFODION HBase mapped table T022HBM2
+-- Definition current
 
-  (
-    "cf1".A                          INT DEFAULT NULL
-  , "cf2".B                          INT DEFAULT NULL
-  , "cf1".Z                          VARCHAR(4) CHARACTER SET ISO88591 COLLATE
-      DEFAULT NO DEFAULT NOT NULL NOT DROPPABLE
-  )
-  PRIMARY KEY NOT SERIALIZED (Z ASC)
+ (
+ "cf1".A INT DEFAULT NULL
+ , "cf2".B INT DEFAULT NULL
+ , "cf1".Z VARCHAR(4) CHARACTER SET ISO88591 COLLATE
+ DEFAULT NO @eof@ NULL NOT DROPPABLE
+ )
+ PRIMARY KEY NOT SERIALIZED (Z ASC)
 
 --- SQL operation complete.
 >>select * from t022hbm2;
 
-A            B            Z   
------------  -----------  ----
+A B Z
+----------
 
-         10           20  a11 
+ 10 20 a11
 
 --- 1 row(s) selected.
 >>
@@ -1215,10 +1215,10 @@ A            B            Z
 >>-- primary key cannot be missing
 >>select * from t022hbm1;
 
-A     B          
-----  -----------
+A B
+---- -----------
 
-a1             10
+a1 10
 
 --- 1 row(s) selected.
 >>
@@ -1230,19 +1230,19 @@ a1             10
 --- SQL operation failed with errors.
 >>invoke t022hbm1;
 
--- Definition of Trafodion HBase mapped table T022HBM1
--- Definition current  Tue May  2 16:55:00 2017
+-- Definition of TRAFODION HBase mapped table T022HBM1
+-- Definition current
 
-  (
-    "cf".A                           VARCHAR(4) CHARACTER SET ISO88591 COLLATE
-      DEFAULT NO DEFAULT NOT NULL NOT DROPPABLE
-  , "cf".B                           INT DEFAULT NULL
-  )
-  PRIMARY KEY NOT SERIALIZED (A ASC)
+ (
+ "cf".A VARCHAR(4) CHARACTER SET ISO88591 COLLATE
+ DEFAULT NO @eof@ NULL NOT DROPPABLE
+ , "cf".B INT DEFAULT NULL
+ )
+ PRIMARY KEY NOT SERIALIZED (A ASC)
 
 --- SQL operation complete.
 >>
->>set schema trafodion."_HB_MAP_";
+>>set schema TRAFODION."_HB_MAP_";
 
 --- SQL operation complete.
 >>
@@ -1273,13 +1273,13 @@ a1             10
 *** ERROR[4261] The schema name _HB_MAP_ is reserved for HBase mapped tables and cannot be explicitly specified.
 
 --- SQL operation failed with errors.
->>alter table trafodion."_HB_MAP_".t022hbm1 drop column b;
+>>alter table TRAFODION."_HB_MAP_".t022hbm1 drop column b;
 
 *** ERROR[4261] The schema name _HB_MAP_ is reserved for HBase mapped tables and cannot be explicitly specified.
 
 --- SQL operation failed with errors.
 >>
->>set schema trafodion.sch;
+>>set schema #CAT.#SCH;
 
 --- SQL operation complete.
 >>
@@ -1288,18 +1288,18 @@ a1             10
 --- SQL operation complete.
 >>
 >>-- cannot specify serialized option
->>create external table t022hbm1 (a varchar(4) not null, b int) 
-+>        primary key serialized (a) 
-+>        attribute default column family 'cf'
-+>        map to hbase table t022hbm1;
+>>create external table t022hbm1 (a varchar(4) not null, b int)
++> primary key serialized (a)
++> attribute default column family 'cf'
++> map to hbase table t022hbm1;
 
 *** ERROR[3242] This statement is not supported. Reason: SERIALIZED option cannot be specified for primary key of this table.
 
 --- SQL operation failed with errors.
 >>
 >>-- cannot be aligned format
->>create external table t022hbm1 (a varchar(4) not null primary key) 
-+>  attribute aligned format map to hbase table t022hbm1;
+>>create external table t022hbm1 (a varchar(4) not null primary key)
++> attribute aligned format map to hbase table t022hbm1;
 
 *** ERROR[3242] This statement is not supported. Reason: Aligned format cannot be specified for an HBase mapped table.
 
@@ -1313,17 +1313,17 @@ a1             10
 --- SQL operation failed with errors.
 >>
 >>-- external and hbase table names must be the same
->>create external table t022hbm11 (a char(4) not null primary key) 
-+>        map to hbase table t022hbm1;
+>>create external table t022hbm11 (a char(4) not null primary key)
++> map to hbase table t022hbm1;
 
-*** ERROR[1180] Trying to create an external HBASE table with a different schema or table name (T022HBM11) than the source table (T022HBM1).  The external schema and table name must be the same as the source.
+*** ERROR[1180] Trying to create an external HBASE table with a different schema or table name (T022HBM11) than the source table (T022HBM1). The external schema and table name must be the same as the source.
 
 --- SQL operation failed with errors.
 >>
 >>-- all non-pkey columns must be nullable
 >>create external table t022hbm1 (a varchar(4) not null primary key,
-+>              b int not null)
-+>              map to hbase table t022hbm1;
++> b int not null)
++> map to hbase table t022hbm1;
 
 *** ERROR[3242] This statement is not supported. Reason: Non-key columns of an HBase mapped table must be nullable with default value of NULL.
 
@@ -1331,28 +1331,28 @@ a1             10
 >>
 >>-- all non-pkey columns must have default value of null
 >>create external table t022hbm1 (a varchar(4) not null primary key,
-+>              b int default 10)
-+>              map to hbase table t022hbm1;
++> b int default 10)
++> map to hbase table t022hbm1;
 
 *** ERROR[3242] This statement is not supported. Reason: Non-key columns of an HBase mapped table must be nullable with default value of NULL.
 
 --- SQL operation failed with errors.
 >>
 >>-- mapped table already exist
->>create external table t022hbm1 (a varchar(4) not null primary key) 
-+>              map to hbase table t022hbm1;
+>>create external table t022hbm1 (a varchar(4) not null primary key)
++> map to hbase table t022hbm1;
 
 --- SQL operation complete.
->>create external table t022hbm1 (a varchar(4) not null primary key) 
-+>              map to hbase table t022hbm1;
+>>create external table t022hbm1 (a varchar(4) not null primary key)
++> map to hbase table t022hbm1;
 
-*** ERROR[1390] Object TRAFODION."_HB_MAP_".T022HBM1 already exists in Trafodion.
+*** ERROR[1390] Object TRAFODION."_HB_MAP_".T022HBM1 already exists in TRAFODION.
 
 --- SQL operation failed with errors.
 >>
 >>-- hbase table doesn't exist
->>create external table t022hbm11 (a char(4) not null primary key) 
-+>                map to hbase table t022hbm11;
+>>create external table t022hbm11 (a char(4) not null primary key)
++> map to hbase table t022hbm11;
 
 *** ERROR[4260] HBase table(T022HBM11) to be mapped as an external table does not exist.
 
@@ -1381,20 +1381,20 @@ a1             10
 >>create hbase table t022hbm1 (column family 'cf');
 
 --- SQL operation complete.
->>create external table t022hbm1 (a varchar(4) not null, b int) 
-+>        primary key (a) 
-+>        attribute default column family 'cf'
-+>        map to hbase table t022hbm1;
+>>create external table t022hbm1 (a varchar(4) not null, b int)
++> primary key (a)
++> attribute default column family 'cf'
++> map to hbase table t022hbm1;
 
 --- SQL operation complete.
->>insert into hbase."_ROW_".t022hbm1 values ('a1', 
-+>                               column_create(('cf:A', '10')));
+>>insert into hbase."_ROW_".t022hbm1 values ('a1',
++> column_create(('cf:A', '10')));
 
 --- 1 row(s) inserted.
 >>-- rowID must match pkey col contents
 >>select * from t022hbm1;
 
-*** ERROR[8556] An error occurred while accessing HBase table T022HBM1.  Details: HBase rowID content must match the primary key column content.
+*** ERROR[8556] An error occurred while accessing HBase table T022HBM1. Details: HBase rowID content must match the primary key column content.
 
 --- 0 row(s) selected.
 >>
@@ -1404,14 +1404,14 @@ a1             10
 >>create hbase table t022hbm1 (column family 'cf');
 
 --- SQL operation complete.
->>insert into hbase."_ROW_".t022hbm1 values ('a1234567', 
-+>                               column_create(('cf:B', '10')));
+>>insert into hbase."_ROW_".t022hbm1 values ('a1234567',
++> column_create(('cf:B', '10')));
 
 --- 1 row(s) inserted.
 >>-- primary key col length must be big enough to hold rowID
 >>select * from t022hbm1;
 
-*** ERROR[8556] An error occurred while accessing HBase table T022HBM1.  Details: retrieved rowID of length 8 is larger than the specified key size of 4.
+*** ERROR[8556] An error occurred while accessing HBase table T022HBM1. Details: retrieved rowID of length 8 is larger than the specified key size of 4.
 
 --- 0 row(s) selected.
 >>
@@ -1421,8 +1421,8 @@ a1             10
 >>create hbase table t022hbm1 (column family 'cf');
 
 --- SQL operation complete.
->>insert into hbase."_ROW_".t022hbm1 values ('a1', 
-+>                               column_create(('cf:B', '1000000')));
+>>insert into hbase."_ROW_".t022hbm1 values ('a1',
++> column_create(('cf:B', '1000000')));
 
 --- 1 row(s) inserted.
 >>-- buffer to retrieve column value must be big enough
@@ -1431,7 +1431,7 @@ a1             10
 --- SQL operation complete.
 >>select * from t022hbm1;
 
-*** ERROR[8556] An error occurred while accessing HBase table T022HBM1.  Details: actual column value length of 7 is greater than the expected max buffer size of 5.
+*** ERROR[8556] An error occurred while accessing HBase table T022HBM1. Details: actual column value length of 7 is greater than the expected max buffer size of 5.
 
 --- 0 row(s) selected.
 >>


[17/22] incubator-trafodion git commit: Updated seabase/TEST022

Posted by sa...@apache.org.
Updated seabase/TEST022


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

Branch: refs/heads/master
Commit: befb8fd33a300636b517ea22b2e648df4ae875cc
Parents: 7a7cfa3
Author: Sean Broeder <sb...@edev05.novalocal>
Authored: Wed May 3 22:21:04 2017 +0000
Committer: Sean Broeder <sb...@edev05.novalocal>
Committed: Wed May 3 22:21:04 2017 +0000

----------------------------------------------------------------------
 core/sql/regress/seabase/TEST022 | 10 +++++-----
 1 file changed, 5 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-trafodion/blob/befb8fd3/core/sql/regress/seabase/TEST022
----------------------------------------------------------------------
diff --git a/core/sql/regress/seabase/TEST022 b/core/sql/regress/seabase/TEST022
index e8e9d13..684def5 100644
--- a/core/sql/regress/seabase/TEST022
+++ b/core/sql/regress/seabase/TEST022
@@ -178,11 +178,11 @@ insert into hbase."_ROW_".t022hbt2 values
                                        ('cf:1a', repeat('a', 40000)),
                                        ('cf:1b', repeat('z', 40000))));
 select count(*) from hbase."_CELL_".t022hbt2;
-select char_length(col_value) from hbase."_CELL_".t022hbt2;
+select char_length(col_value) from hbase."_CELL_".t022hbt2 order by 1;
 select count(*) from hbase."_CELL_".t022hbt2;
-select char_length(col_value) from hbase."_CELL_".t022hbt2;
+select char_length(col_value) from hbase."_CELL_".t022hbt2 order by 1;
 select count(*) from hbase."_ROW_".t022hbt2;
-select char_length(column_details) from hbase."_ROW_".t022hbt2;
+select char_length(column_details) from hbase."_ROW_".t022hbt2 order by 1;
 select left(row_id, 10) row_id, left(column_display(column_details), 40) 
            from hbase."_ROW_".t022hbt2;
 
@@ -286,9 +286,9 @@ insert into t022hbm1 values (1,2,3);
 insert into t022hbm1 values (1,2,3);
 insert into t022hbm1 values (1, 1, 1);
 insert into t022hbm1 values (-1, -2, -3);
-select * from t022hbm1;
+select * from t022hbm1 order by 1;
 upsert into t022hbm1 values (1,2,4);
-select * from t022hbm1;
+select * from t022hbm1 order by 1;
 
 -- various serialization options
 drop external table t022hbm1;


[12/22] incubator-trafodion git commit: Added missed template file

Posted by sa...@apache.org.
Added missed template file


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

Branch: refs/heads/master
Commit: d9c2ec5517043eb5b74f598ddbdc3e324ac14dee
Parents: 4d378a6
Author: Sean Broeder <sb...@edev05.novalocal>
Authored: Mon May 1 21:49:34 2017 +0000
Committer: Sean Broeder <sb...@edev05.novalocal>
Committed: Mon May 1 21:49:34 2017 +0000

----------------------------------------------------------------------
 .../TransactionalRegionScannerHolder.java.tmpl  | 79 ++++++++++++++++++++
 1 file changed, 79 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-trafodion/blob/d9c2ec55/core/sqf/src/seatrans/hbase-trx/src/main/java/org/apache/hadoop/hbase/regionserver/transactional/TransactionalRegionScannerHolder.java.tmpl
----------------------------------------------------------------------
diff --git a/core/sqf/src/seatrans/hbase-trx/src/main/java/org/apache/hadoop/hbase/regionserver/transactional/TransactionalRegionScannerHolder.java.tmpl b/core/sqf/src/seatrans/hbase-trx/src/main/java/org/apache/hadoop/hbase/regionserver/transactional/TransactionalRegionScannerHolder.java.tmpl
new file mode 100644
index 0000000..30d83c0
--- /dev/null
+++ b/core/sqf/src/seatrans/hbase-trx/src/main/java/org/apache/hadoop/hbase/regionserver/transactional/TransactionalRegionScannerHolder.java.tmpl
@@ -0,0 +1,79 @@
+/**
+* @@@ START COPYRIGHT @@@
+*
+* Licensed to the Apache Software Foundation (ASF) under one
+* or more contributor license agreements.  See the NOTICE file
+* distributed with this work for additional information
+* regarding copyright ownership.  The ASF licenses this file
+* to you under the Apache License, Version 2.0 (the
+* "License"); you may not use this file except in compliance
+* with the License.  You may obtain a copy of the License at
+*
+*   http://www.apache.org/licenses/LICENSE-2.0
+*
+* Unless required by applicable law or agreed to in writing,
+* software distributed under the License is distributed on an
+* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+* KIND, either express or implied.  See the License for the
+* specific language governing permissions and limitations
+* under the License.
+*
+* @@@ END COPYRIGHT @@@
+**/
+
+package org.apache.hadoop.hbase.regionserver.transactional;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+#ifdef APACHE1.2 || CDH5.7
+import org.apache.hadoop.hbase.regionserver.Region;
+#else
+import org.apache.hadoop.hbase.regionserver.HRegion;
+#endif
+import org.apache.hadoop.hbase.regionserver.RegionScanner;
+
+/**
+ * Holds a RegionScanner
+ */
+ public class TransactionalRegionScannerHolder {
+    public RegionScanner s;
+#ifdef APACHE1.2 || CDH5.7
+    public Region r;
+#else
+    public HRegion r;
+#endif
+    public long nextCallSeq;
+    public long numberOfRows;
+    public long rowsRemaining;
+    public long transId;
+    public long scannerId;
+    public boolean hasMore;
+
+
+#ifdef APACHE1.2 || CDH5.7
+    public TransactionalRegionScannerHolder(long transId,
+                                            long scannerId,
+                                            RegionScanner s, 
+                                            Region r) {
+#else
+    public TransactionalRegionScannerHolder(long transId,
+                                            long scannerId,
+                                            RegionScanner s, 
+                                            HRegion r) {
+#endif
+      this.transId = transId;
+      this.scannerId = scannerId;
+      this.s = s;
+      this.r = r;
+      this.nextCallSeq = 0L;
+      this.numberOfRows = 0L;
+      this.rowsRemaining = 0L;
+      this.hasMore = false;
+    }
+
+    public void cleanHolder() {
+      this.r = null;
+      this.s = null;
+    }
+  }
+


[18/22] incubator-trafodion git commit: Manual merge fixes

Posted by sa...@apache.org.
Manual merge fixes


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

Branch: refs/heads/master
Commit: 41fcd8d48a2e8bbeccee741beea0b3d4cad9332f
Parents: befb8fd
Author: Sean Broeder <sb...@edev05.esgyn.local>
Authored: Wed May 10 21:37:59 2017 +0000
Committer: Sean Broeder <sb...@edev05.esgyn.local>
Committed: Wed May 10 21:37:59 2017 +0000

----------------------------------------------------------------------
 .../hadoop/hbase/client/transactional/RMInterface.java  |  6 +++---
 .../hbase/client/transactional/TransactionState.java    |  2 +-
 .../client/transactional/TransactionalScanner.java.tmpl |  8 ++++----
 .../transactional/TrxRegionEndpoint.java.tmpl           | 12 ++++++------
 .../src/main/java/org/trafodion/sql/HTableClient.java   |  3 ++-
 5 files changed, 16 insertions(+), 15 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-trafodion/blob/41fcd8d4/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 f783b13..8a543aa 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
@@ -346,7 +346,7 @@ public class RMInterface {
         else {
             if (LOG.isTraceEnabled()) LOG.trace("RMInterface:registerTransaction - Found TS in map for tx " + ts);
         }
-        HRegionLocation location = ttable.getRegionLocation(row, false /*reload*/);
+        HRegionLocation location = pv_table.getRegionLocation(row, false /*reload*/);
 
         if (LOG.isTraceEnabled()) LOG.trace("RMInterface:registerTransaction - retrieved location with startKey="
               + Hex.encodeHexString(location.getRegionInfo().getStartKey()) + ", endKey="
@@ -367,7 +367,7 @@ public class RMInterface {
 
         // register region with TM.
         if (register) {
-            ts.registerLocation(trLocation);
+            ts.registerLocation(location);
              if (LOG.isTraceEnabled()) LOG.trace("RMInterface:registerTransaction, called registerLocation TransactionRegionLocation [" + trLocation.getRegionInfo().getRegionNameAsString() +  "\nEncodedName: [" + trLocation.getRegionInfo().getEncodedName() + "], endKey: "
                   + Hex.encodeHexString(trLocation.getRegionInfo().getEndKey()) + " to transaction [" + transactionID + "]");
         }
@@ -426,7 +426,7 @@ public class RMInterface {
 
        // register region with TM.
        if (register) {
-          ts.registerLocation(trLocation);
+          ts.registerLocation(location);
           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 + "]");
        }

http://git-wip-us.apache.org/repos/asf/incubator-trafodion/blob/41fcd8d4/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 4b5fdf9..4ee4afb 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
@@ -291,7 +291,7 @@ public class TransactionState {
     }
 
     // Used at the client end - the one performing the mutation - e.g. the SQL process
-    public void registerLocation(final TransactionRegionLocation location) throws IOException {
+    public void registerLocation(final HRegionLocation location) throws IOException {
         byte [] lv_hostname = location.getHostname().getBytes();
         int lv_port = location.getPort();
         long lv_startcode = location.getServerName().getStartcode();

http://git-wip-us.apache.org/repos/asf/incubator-trafodion/blob/41fcd8d4/core/sqf/src/seatrans/hbase-trx/src/main/java/org/apache/hadoop/hbase/client/transactional/TransactionalScanner.java.tmpl
----------------------------------------------------------------------
diff --git a/core/sqf/src/seatrans/hbase-trx/src/main/java/org/apache/hadoop/hbase/client/transactional/TransactionalScanner.java.tmpl b/core/sqf/src/seatrans/hbase-trx/src/main/java/org/apache/hadoop/hbase/client/transactional/TransactionalScanner.java.tmpl
index 35dc84a..86853a7 100755
--- a/core/sqf/src/seatrans/hbase-trx/src/main/java/org/apache/hadoop/hbase/client/transactional/TransactionalScanner.java.tmpl
+++ b/core/sqf/src/seatrans/hbase-trx/src/main/java/org/apache/hadoop/hbase/client/transactional/TransactionalScanner.java.tmpl
@@ -159,8 +159,8 @@ public class TransactionalScanner extends AbstractClientScanner {
                 return false;
             }
             else
-                this.currentBeginKey = TransactionManager.binaryIncrementPos(this.currentEndKey,1);
-                //this.currentBeginKey = this.currentEndKey;
+                //this.currentBeginKey = TransactionManager.binaryIncrementPos(this.currentEndKey,1);
+                  this.currentBeginKey = this.currentEndKey;
         }
         else {
             // First call to nextScanner
@@ -175,8 +175,8 @@ public class TransactionalScanner extends AbstractClientScanner {
                       "INFINITE" : Hex.encodeHexString(this.currentBeginKey))
               + " currentEndKey: " + (Bytes.equals(this.currentEndKey, HConstants.EMPTY_END_ROW) ?
                       "INFINITE" : Hex.encodeHexString(this.currentEndKey)));
-        if ( ! (Bytes.equals(this.currentEndKey, HConstants.EMPTY_END_ROW)))
-           this.currentEndKey = TransactionManager.binaryIncrementPos(currentRegion.getEndKey(), -1);
+        //if(this.currentEndKey != HConstants.EMPTY_END_ROW)
+        //   this.currentEndKey = TransactionManager.binaryIncrementPos(currentRegion.getEndKey(), -1);
 
         this.closed = false;
 

http://git-wip-us.apache.org/repos/asf/incubator-trafodion/blob/41fcd8d4/core/sqf/src/seatrans/hbase-trx/src/main/java/org/apache/hadoop/hbase/coprocessor/transactional/TrxRegionEndpoint.java.tmpl
----------------------------------------------------------------------
diff --git a/core/sqf/src/seatrans/hbase-trx/src/main/java/org/apache/hadoop/hbase/coprocessor/transactional/TrxRegionEndpoint.java.tmpl b/core/sqf/src/seatrans/hbase-trx/src/main/java/org/apache/hadoop/hbase/coprocessor/transactional/TrxRegionEndpoint.java.tmpl
index b0f9d24..5a67afc 100644
--- a/core/sqf/src/seatrans/hbase-trx/src/main/java/org/apache/hadoop/hbase/coprocessor/transactional/TrxRegionEndpoint.java.tmpl
+++ b/core/sqf/src/seatrans/hbase-trx/src/main/java/org/apache/hadoop/hbase/coprocessor/transactional/TrxRegionEndpoint.java.tmpl
@@ -120,7 +120,7 @@ import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.CellUtil;
-#ifdef HDP2.3 HDP2.4 CDH5.5 CDH5.7 APACHE1.2
+#ifdef HDP2.3 HDP2.4 CDH5.7 APACHE1.2
 import org.apache.hadoop.hbase.ChoreService;
 import org.apache.hadoop.hbase.ScheduledChore;
 #endif
@@ -4891,7 +4891,7 @@ CoprocessorService, Coprocessor {
     throws IOException {
 
     if (LOG.isTraceEnabled()) LOG.trace("Enter checkAndPut, txid: "
-                + transactionId + ", on HRegion " + this);
+                + transactionId + ", on HRegion " + m_regionDetails);
     TrxTransactionState state = this.beginTransIfNotExist(transactionId, startId);
     if (!state.getStatus().equals(Status.PENDING)) { // Active
        throw new IOException("checkAndPut late checkin for transaction " + transactionId + " in region " + m_regionDetails);
@@ -4909,7 +4909,7 @@ CoprocessorService, Coprocessor {
       get.addColumn(family, qualifier);
 
       // this will call getScanner (and then add a state scan range), and later put will call addWrite and then addRead
-      // would also add a scan range, maybe leave it nnow and optimize later
+      // would also add a scan range, maybe leave it now and optimize later
       Result rs = this.get(transactionId, startId, get);
 
       boolean valueIsNull = value == null ||
@@ -4925,7 +4925,7 @@ CoprocessorService, Coprocessor {
           result = true;
         }
         else {
-          if (LOG.isTraceEnabled()) LOG.trace("checkAndPut - txid " + transactionId + ", row " + Bytes.toStringBinary(row) + ", row in hex " + Hex.encodeHexString(row) + ", first check setting result to false");
+          if (LOG.isTraceEnabled()) LOG.trace("checkAndPut - txid " + transactionId + ", row " + Bytes.toStringBinary(row) + ", row in hex " + Hex.encodeHexString(row) + ", first check setting result to false in region " + m_regionDetails);
           result = false;
         }
       } else if ((!rs.isEmpty()) && !valueIsNull   
@@ -4933,7 +4933,7 @@ CoprocessorService, Coprocessor {
          this.put(transactionId, startId, put);
          result = true;
       } else {
-          if (LOG.isTraceEnabled()) LOG.trace("checkAndPut - txid " + transactionId + ", row " + Bytes.toStringBinary(row) + ", row in hex " + Hex.encodeHexString(row) + ", second check setting result to false");
+          if (LOG.isTraceEnabled()) LOG.trace("checkAndPut - txid " + transactionId + ", row " + Bytes.toStringBinary(row) + ", row in hex " + Hex.encodeHexString(row) + ", second check setting result to false in region " + m_regionDetails);
         result = false;
       }
     } catch (Exception e) {
@@ -4941,7 +4941,7 @@ CoprocessorService, Coprocessor {
       throw new IOException("checkAndPut - " + e.toString());
     }
 
-    if (LOG.isTraceEnabled()) LOG.trace("checkAndPut EXIT - returns " + result + ", transId " + transactionId + ", row " + Bytes.toStringBinary(row) + ", row in hex " + Hex.encodeHexString(row));
+    if (LOG.isTraceEnabled()) LOG.trace("checkAndPut EXIT - returns " + result + ", transId " + transactionId + ", row " + Bytes.toStringBinary(row) + ", row in hex " + Hex.encodeHexString(row) + " in region " + m_regionDetails);
 
     return result;
   }

http://git-wip-us.apache.org/repos/asf/incubator-trafodion/blob/41fcd8d4/core/sql/src/main/java/org/trafodion/sql/HTableClient.java
----------------------------------------------------------------------
diff --git a/core/sql/src/main/java/org/trafodion/sql/HTableClient.java b/core/sql/src/main/java/org/trafodion/sql/HTableClient.java
index 0b4e475..48cc1fe 100644
--- a/core/sql/src/main/java/org/trafodion/sql/HTableClient.java
+++ b/core/sql/src/main/java/org/trafodion/sql/HTableClient.java
@@ -1388,9 +1388,10 @@ public class HTableClient {
                             else {
                                 table.delete(del);
                             }
-                            return true;
+                            return new Boolean(res);
                         }
                     });
+		    return true;
             }
             else {
                 if (useTRex && (transID != 0)) {


[03/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

Posted by sa...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-trafodion/blob/91794b53/core/sqf/src/seatrans/tm/hbasetmlib2/src/main/java/org/trafodion/dtm/HBaseTxClient.java
----------------------------------------------------------------------
diff --git a/core/sqf/src/seatrans/tm/hbasetmlib2/src/main/java/org/trafodion/dtm/HBaseTxClient.java b/core/sqf/src/seatrans/tm/hbasetmlib2/src/main/java/org/trafodion/dtm/HBaseTxClient.java
index d7be3ea..21ca319 100644
--- a/core/sqf/src/seatrans/tm/hbasetmlib2/src/main/java/org/trafodion/dtm/HBaseTxClient.java
+++ b/core/sqf/src/seatrans/tm/hbasetmlib2/src/main/java/org/trafodion/dtm/HBaseTxClient.java
@@ -94,6 +94,7 @@ public class HBaseTxClient {
    private int stallWhere;
    private IdTm idServer;
    private static final int ID_TM_SERVER_TIMEOUT = 1000;
+   private static int myClusterId = 0;
 
    public enum AlgorithmType{
      MVCC, SSCC
@@ -180,7 +181,7 @@ public class HBaseTxClient {
       try {
         trxManager = TransactionManager.getInstance(config, connection);
       } catch (IOException e ){
-          LOG.error("Unable to create TransactionManager, throwing exception", e);
+          LOG.error("trxManager Initialization failure throwing exception", e);
           throw e;
       }
 
@@ -300,7 +301,9 @@ public class HBaseTxClient {
                                            this,
                                            useForgotten,
                                            forceForgotten,
-                                           useTlog);
+                                           useTlog,
+                                           false,
+                                           false);
           recovThread.start();
       }
       if (LOG.isTraceEnabled()) LOG.trace("Exit init()");
@@ -328,7 +331,9 @@ public class HBaseTxClient {
                                                    this,
                                                    useForgotten,
                                                    forceForgotten,
-                                                   useTlog);
+                                                   useTlog,
+                                                   false,
+                                                   true);
                newRecovThread.start();
                mapRecoveryThreads.put(nodeID, recovThread);
                if(LOG.isTraceEnabled()) LOG.trace("nodeDown -- mapRecoveryThreads size: " + mapRecoveryThreads.size());
@@ -366,6 +371,10 @@ public class HBaseTxClient {
       return TransReturnCode.RET_OK.getShort();
    }
 
+   public static Map<Long, TransactionState> getMap() {
+     return mapTransactionStates;
+   }
+
    public long beginTransaction(final long transactionId) throws IOException {
 
       if (LOG.isTraceEnabled()) LOG.trace("Enter beginTransaction, txid: " + transactionId);
@@ -383,11 +392,11 @@ public class HBaseTxClient {
          if (tx2 != null) {
             // Some other thread added the transaction while we were creating one.  It's already in the
             // map, so we can use the existing one.
-            if (LOG.isDebugEnabled()) LOG.debug("HBaseTxClient:beginTransaction, found TransactionState object while creating a new one " + tx2);
+            if (LOG.isDebugEnabled()) LOG.debug("beginTransaction, found TransactionState object while creating a new one " + tx2);
             tx = tx2;
          }
          else {
-            if (LOG.isDebugEnabled()) LOG.debug("HBaseTxClient:beginTransaction, adding new TransactionState to map " + tx);
+            if (LOG.isDebugEnabled()) LOG.debug("beginTransaction, adding new TransactionState to map " + tx);
             mapTransactionStates.put(transactionId, tx);
          }
       }
@@ -397,7 +406,7 @@ public class HBaseTxClient {
    }
 
    public short abortTransaction(final long transactionID) throws IOException {
-      if (LOG.isDebugEnabled()) LOG.debug("Enter abortTransaction, txid: " + transactionID);
+      if (LOG.isDebugEnabled()) LOG.debug("Enter abortTransaction, txId: " + transactionID);
       TransactionState ts = mapTransactionStates.get(transactionID);
 
       if(ts == null) {
@@ -408,10 +417,11 @@ public class HBaseTxClient {
       try {
          ts.setStatus(TransState.STATE_ABORTED);
          if (useTlog) {
-            tLog.putSingleRecord(transactionID, -1, "ABORTED", ts.getParticipatingRegions(), false);
+            tLog.putSingleRecord(transactionID, ts.getStartId(), -1, TransState.STATE_ABORTED.toString(), ts.getParticipatingRegions(), ts.hasPlaceHolder(), false); //force flush
          }
-      } catch(IOException e) {
-         LOG.error("Returning from HBaseTxClient:abortTransaction, txid: " + transactionID + " tLog.putRecord: EXCEPTION", e);
+      }
+      catch(IOException e) {
+         LOG.error("Returning from HBaseTxClient:abortTransaction, txid: " + transactionID + " tLog.putRecord: EXCEPTION ", e);
          return TransReturnCode.RET_EXCEPTION.getShort();
       }
 
@@ -465,12 +475,7 @@ public class HBaseTxClient {
       }
 
       if (useTlog && useForgotten) {
-         if (forceForgotten) {
-            tLog.putSingleRecord(transactionID, -1, "FORGOTTEN", null, true);
-         }
-         else {
-            tLog.putSingleRecord(transactionID, -1, "FORGOTTEN", null, false);
-         }
+            tLog.putSingleRecord(transactionID, ts.getStartId(), -1, TransState.STATE_FORGOTTEN_ABORT.toString(), ts.getParticipatingRegions(), ts.hasPlaceHolder(), forceForgotten); // forced flush?
       }
  //     mapTransactionStates.remove(transactionID);
 
@@ -481,11 +486,17 @@ public class HBaseTxClient {
    public short prepareCommit(long transactionId) throws 
                                                  TransactionManagerException,
                                                  IOException{
-     if (LOG.isDebugEnabled()) LOG.debug("Enter prepareCommit, txid: " + transactionId);
-     if (LOG.isTraceEnabled()) LOG.trace("mapTransactionStates " + mapTransactionStates + " entries " + mapTransactionStates.size());
-        TransactionState ts = mapTransactionStates.get(transactionId);
-     if(ts == null) {
-       LOG.error("Returning from HBaseTxClient:prepareCommit, txid: " + transactionId + " retval: " + TransReturnCode.RET_NOTX.toString());
+     if (LOG.isDebugEnabled()) LOG.debug("Enter prepareCommit"
+					 + ", txId: " + transactionId
+					 + ", #txstate entries " + mapTransactionStates.size()
+					 );
+     TransactionState ts = mapTransactionStates.get(transactionId);
+     
+     if (ts == null) {
+       LOG.error("Returning from prepareCommit" 
+		 + ", txId: " + transactionId 
+		 + ", retval: " + TransReturnCode.RET_NOTX.toString()
+		 );
        return TransReturnCode.RET_NOTX.getShort();
      }
 
@@ -494,13 +505,13 @@ public class HBaseTxClient {
         if (LOG.isDebugEnabled()) LOG.debug("prepareCommit, [ " + ts + " ], result " + result + ((result == TransactionalReturn.COMMIT_OK_READ_ONLY)?", Read-Only":""));
         switch (result) {
           case TransactionalReturn.COMMIT_OK:
-             if (LOG.isTraceEnabled()) LOG.trace("Exit OK prepareCommit, txid: " + transactionId);
+             if (LOG.isTraceEnabled()) LOG.trace("Exit OK prepareCommit, txId: " + transactionId);
              return TransReturnCode.RET_OK.getShort();
           case TransactionalReturn.COMMIT_OK_READ_ONLY:
              synchronized(mapLock) {
                 mapTransactionStates.remove(transactionId);
              }
-             if (LOG.isTraceEnabled()) LOG.trace("Exit OK_READ_ONLY prepareCommit, txid: " + transactionId);
+             if (LOG.isTraceEnabled()) LOG.trace("Exit OK_READ_ONLY prepareCommit, txId: " + transactionId);
              return TransReturnCode.RET_READONLY.getShort();
           case TransactionalReturn.COMMIT_UNSUCCESSFUL:
              if(!ts.getRecordedException().isEmpty())
@@ -530,28 +541,32 @@ public class HBaseTxClient {
        throw t;
      } 
      catch (CommitUnsuccessfulException e) {
-       LOG.error("Returning from HBaseTxClient:prepareCommit, txid: " + transactionId + " retval: " + TransReturnCode.RET_NOCOMMITEX.toString() + " CommitUnsuccessfulException", e);
+       LOG.error("Returning from prepareCommit, txId: " + transactionId + " retval: " + TransReturnCode.RET_NOCOMMITEX.toString() + " CommitUnsuccessfulException");
        throw new TransactionManagerException(e,
                                    TransReturnCode.RET_NOCOMMITEX.getShort());
      }
      catch (IOException e) {
-       LOG.error("Returning from HBaseTxClient:prepareCommit, txid: " + transactionId + " retval: " + TransReturnCode.RET_IOEXCEPTION.toString() + " IOException", e);
+       LOG.error("Returning from prepareCommit, txId: " + transactionId + " retval: " + TransReturnCode.RET_IOEXCEPTION.toString() + " IOException");
        throw new TransactionManagerException(e,
                                    TransReturnCode.RET_IOEXCEPTION.getShort());
      }
    }
 
-   public short doCommit(long transactionId) throws IOException, CommitUnsuccessfulException {
-       if (LOG.isDebugEnabled()) LOG.debug("Enter doCommit, txid: " + transactionId);
-       TransactionState ts = mapTransactionStates.get(transactionId);
+   public short doCommit(long transactionId) throws IOException {
+      if (LOG.isDebugEnabled()) LOG.debug("Enter doCommit, txId: " + transactionId);
+      TransactionState ts = mapTransactionStates.get(transactionId);
 
-       if(ts == null) {
-      LOG.error("Returning from HBaseTxClient:doCommit, (null tx) retval: " + TransReturnCode.RET_NOTX.toString() + " txid: " + transactionId);
+      if(ts == null) {
+	      LOG.error("Returning from doCommit, (null tx) retval: " 
+			+ TransReturnCode.RET_NOTX.toString() 
+			+ ", txId: " + transactionId
+			);
           return TransReturnCode.RET_NOTX.getShort();
        }
 
        // Set the commitId
        IdTmId commitId = null;
+       long commitIdVal = -1;
        if (TRANSACTION_ALGORITHM == AlgorithmType.SSCC) {
           try {
              commitId = new IdTmId();
@@ -562,24 +577,39 @@ public class HBaseTxClient {
              LOG.error("doCommit: IdTm threw exception " ,  exc);
              throw new CommitUnsuccessfulException("doCommit: IdTm threw exception " ,  exc);
           }
+          commitIdVal = commitId.val;
        }
-
-       final long commitIdVal = (TRANSACTION_ALGORITHM == AlgorithmType.SSCC) ? commitId.val : -1;
        if (LOG.isTraceEnabled()) LOG.trace("doCommit setting commitId (" + commitIdVal + ") for tx: " + ts.getTransactionId());
        ts.setCommitId(commitIdVal);
 
-       try {
-          ts.setStatus(TransState.STATE_COMMITTED);
-          if (useTlog) {
-             tLog.putSingleRecord(transactionId, commitIdVal, "COMMITTED", ts.getParticipatingRegions(), true);
-          }
-       } catch(IOException e) {
-          LOG.error("Returning from HBaseTxClient:doCommit, txid: " + transactionId + " tLog.putRecord: EXCEPTION ", e);
-          return TransReturnCode.RET_EXCEPTION.getShort();
+       if (stallWhere == 4) {
+    	  if (LOG.isInfoEnabled())LOG.info("Stalling in phase 2a (before TLOG write) for doCommit for transaction: " + transactionId);
+          boolean loopBack = false;
+          do
+          {
+             try {
+                loopBack = false;
+                Thread.sleep(600000); // Initially set to run every 5 min
+             } catch (InterruptedException ie) {
+                loopBack = true;
+             }
+          } while (loopBack);
        }
 
+       //try {
+          ts.setStatus(TransState.STATE_COMMITTING);
+          if (useTlog) {
+             try {
+                tLog.putSingleRecord(transactionId, ts.getStartId(), commitIdVal, TransState.STATE_COMMITTED.toString(), ts.getParticipatingRegions(), ts.hasPlaceHolder(), true);
+                ts.setStatus(TransState.STATE_COMMITTED);
+             }
+             catch (IOException e) {
+                 LOG.error("doCommit: Local TLOG write threw exception during commit " , e);
+                 throw new RuntimeException(e);
+             }
+          }
        if ((stallWhere == 2) || (stallWhere == 3)) {
-          LOG.info("Stalling in phase 2 for doCommit");
+    	  if (LOG.isInfoEnabled())LOG.info("Stalling in phase 2 for doCommit for transaction: " + transactionId);
           boolean loopBack = false;
           do {
              try {
@@ -592,13 +622,15 @@ public class HBaseTxClient {
        }
 
        try {
+          if (LOG.isTraceEnabled()) LOG.trace("doCommit, calling trxManager.doCommit(" + ts.getTransactionId() + ")" );
           trxManager.doCommit(ts);
        } catch (CommitUnsuccessfulException e) {
-          LOG.error("Returning from HBaseTxClient:doCommit, retval: " + TransReturnCode.RET_EXCEPTION.toString() + " IOException" + " txid: " + transactionId, e);
+          LOG.error("Returning from doCommit, transaction: " + transactionId
+        		      + ", retval: " + TransReturnCode.RET_EXCEPTION.toString() + " IOException");
           return TransReturnCode.RET_EXCEPTION.getShort();
        }
        catch (UnsuccessfulDDLException ddle) {
-          LOG.error("FATAL DDL Exception from HBaseTxClient:doCommit, WAITING INDEFINETLY !! retval: " + TransReturnCode.RET_EXCEPTION.toString() + " UnsuccessfulDDLException" + " txid: " + transactionId, ddle);
+          LOG.error("FATAL DDL Exception from doCommit, WAITING INDEFINETLY !! retval: " + TransReturnCode.RET_EXCEPTION.toString() + " UnsuccessfulDDLException" + " txId: " + transactionId);
 
           //Reaching here means several attempts to perform the DDL operation has failed in commit phase.
           //Generally if only DML operation is involved, returning error causes TM to call completeRequest()
@@ -616,7 +648,9 @@ public class HBaseTxClient {
                    loopBack = false;
                    commitDDLLock.wait();
                 } catch(InterruptedException ie) {
-                    LOG.warn("Interrupting commitDDLLock.wait,  but retrying ", ie);
+                    LOG.warn("Interrupting commitDDLLock.wait" 
+			     + ", txId: " + transactionId
+			     + ", retrying ", ie);
                     loopBack = true;
                 }
              } while (loopBack);
@@ -624,38 +658,37 @@ public class HBaseTxClient {
           return TransReturnCode.RET_EXCEPTION.getShort();
        }
        if (useTlog && useForgotten) {
-          if (forceForgotten) {
-             tLog.putSingleRecord(transactionId, commitIdVal, "FORGOTTEN", null, true);
-          }
-          else {
-             tLog.putSingleRecord(transactionId, commitIdVal, "FORGOTTEN", null, false);
-          }
+          tLog.putSingleRecord(transactionId, ts.getStartId(), commitIdVal, TransState.STATE_FORGOTTEN_COMMITTED.toString(), ts.getParticipatingRegions(), ts.hasPlaceHolder(), forceForgotten); // forced flush?
        }
-//       mapTransactionStates.remove(transactionId);
-
        if (LOG.isTraceEnabled()) LOG.trace("Exit doCommit, retval(ok): " + TransReturnCode.RET_OK.toString() +
-                         " txid: " + transactionId + " mapsize: " + mapTransactionStates.size());
+                         " txId: " + transactionId + " mapsize: " + mapTransactionStates.size());
 
        return TransReturnCode.RET_OK.getShort();
    }
 
-   public short completeRequest(long transactionId) throws IOException, CommitUnsuccessfulException {
-     if (LOG.isDebugEnabled()) LOG.debug("Enter completeRequest, txid: " + transactionId);
+    public short completeRequest(long transactionId)
+	throws IOException, CommitUnsuccessfulException 
+    {
+     if (LOG.isDebugEnabled()) LOG.debug("Enter completeRequest" 
+					 + ", txId: " + transactionId
+					 );
      TransactionState ts = mapTransactionStates.get(transactionId);
 
-     if(ts == null) {
-          LOG.error("Returning from HBaseTxClient:completeRequest, (null tx) retval: " + TransReturnCode.RET_NOTX.toString() + " txid: " + transactionId);
-          return TransReturnCode.RET_NOTX.getShort();
-       }
-  
+     if (ts == null) {
+	     LOG.error("Returning from completeRequest, (null tx) retval: " 
+		       + TransReturnCode.RET_NOTX.toString() 
+		       + ", txId: " + transactionId
+		       );
+	 return TransReturnCode.RET_NOTX.getShort();
+     }
+
        boolean loopBack = false;
        do {
           try {
-             if (LOG.isTraceEnabled()) LOG.trace("TEMP completeRequest Calling CompleteRequest() Txid :" + transactionId);
              loopBack = false;
              ts.completeRequest();
           } catch(InterruptedException ie) {
-              LOG.warn("Interrupting HBaseTxClient:completeRequest but retrying, ts.completeRequest: txid: " + transactionId + ", EXCEPTION: ", ie);
+              LOG.warn("Interrupting completeRequest but retrying, ts.completeRequest: txid: " + transactionId + ", EXCEPTION: ", ie);
               loopBack = true;
           } 
        } while (loopBack);
@@ -681,11 +714,8 @@ public class HBaseTxClient {
        commitErr = doCommit(transactionId);
        if (commitErr != TransReturnCode.RET_OK.getShort()) {
          LOG.error("doCommit for committed transaction " + transactionId + " failed with error " + commitErr);
-         // It is a violation of 2 PC protocol to try to abort the transaction after prepare
+         // It is a violation of 2 PC protocol to try to abort the transaction after commit write
          return commitErr;
-//         abortErr = abortTransaction(transactionId);
-//         if (LOG.isDebugEnabled()) LOG.debug("tryCommit commit failed and was aborted. Commit error " +
-//                   commitErr + ", Abort error " + abortErr);
        }
 
        if (LOG.isTraceEnabled()) LOG.trace("TEMP tryCommit Calling CompleteRequest() Txid :" + transactionId);
@@ -704,7 +734,7 @@ public class HBaseTxClient {
        mapTransactionStates.remove(transactionId);
     }
 
-    if (LOG.isDebugEnabled()) LOG.debug("Exit completeRequest txid: " + transactionId + " mapsize: " + mapTransactionStates.size());
+    if (LOG.isDebugEnabled()) LOG.debug("Exit completeRequest txId: " + transactionId + " mapsize: " + mapTransactionStates.size());
     return TransReturnCode.RET_OK.getShort();
   }
 
@@ -713,11 +743,11 @@ public class HBaseTxClient {
       TransactionState ts;
       HTableDescriptor htdesc = null;
 
-      if (LOG.isTraceEnabled()) LOG.trace("Enter callCreateTable, txid: [" + transactionId + "],  htbldesc bytearray: " + pv_htbldesc + "desc in hex: " + Hex.encodeHexString(pv_htbldesc));
+      if (LOG.isTraceEnabled()) LOG.trace("Enter callCreateTable, txId: [" + transactionId + "],  htbldesc bytearray: " + pv_htbldesc + "desc in hex: " + Hex.encodeHexString(pv_htbldesc));
 
       ts = mapTransactionStates.get(transactionId);
       if(ts == null) {
-         LOG.error("Returning from HBaseTxClient:callCreateTable, (null tx) retval: " + TransReturnCode.RET_NOTX.getShort()  + " txid: " + transactionId);
+         LOG.error("Returning from callCreateTable, (null tx) retval: " + TransReturnCode.RET_NOTX.getShort()  + " txId: " + transactionId);
          return TransReturnCode.RET_NOTX.getShort();
       }
       try {
@@ -745,11 +775,11 @@ public class HBaseTxClient {
       TransactionState ts;
       String strTblName = new String(pv_tblname, "UTF-8");
 
-      if (LOG.isTraceEnabled()) LOG.trace("Enter callAlterTable, txid: [" + transactionId + "],  tableName: " + strTblName);
+      if (LOG.isTraceEnabled()) LOG.trace("Enter callAlterTable, txId: [" + transactionId + "],  tableName: " + strTblName);
 
       ts = mapTransactionStates.get(transactionId);
       if(ts == null) {
-         LOG.error("Returning from HBaseTxClient:callAlterTable, (null tx) retval: " + TransReturnCode.RET_NOTX.getShort()  + " txid: " + transactionId);
+         LOG.error("Returning from callAlterTable, (null tx) retval: " + TransReturnCode.RET_NOTX.getShort()  + " txId: " + transactionId);
          return TransReturnCode.RET_NOTX.getShort();
       }
 
@@ -762,11 +792,11 @@ public class HBaseTxClient {
       TransactionState ts;
       String strTblName = new String(pv_tblname, "UTF-8");
 
-      if (LOG.isTraceEnabled()) LOG.trace("Enter callRegisterTruncateOnAbort, txid: [" + transactionId + "],  tablename: " + strTblName);
+      if (LOG.isTraceEnabled()) LOG.trace("Enter callRegisterTruncateOnAbort, txId: [" + transactionId + "],  tablename: " + strTblName);
 
       ts = mapTransactionStates.get(transactionId);
       if(ts == null) {
-         LOG.error("Returning from HBaseTxClient:callRegisterTruncateOnAbort, (null tx) retval: " + TransReturnCode.RET_NOTX.getShort()  + " txid: " + transactionId);
+         LOG.error("Returning from callRegisterTruncateOnAbort, (null tx) retval: " + TransReturnCode.RET_NOTX.getShort()  + " txId: " + transactionId);
          return TransReturnCode.RET_NOTX.getShort();
       }
 
@@ -779,11 +809,11 @@ public class HBaseTxClient {
       TransactionState ts;
       String strTblName = new String(pv_tblname, "UTF-8");
 
-      if (LOG.isTraceEnabled()) LOG.trace("Enter callDropTable, txid: [" + transactionId + "],  tablename: " + strTblName);
+      if (LOG.isTraceEnabled()) LOG.trace("Enter callDropTable, txId: [" + transactionId + "],  tablename: " + strTblName);
 
       ts = mapTransactionStates.get(transactionId);
       if(ts == null) {
-         LOG.error("Returning from HBaseTxClient:callDropTable, (null tx) retval: " + TransReturnCode.RET_NOTX.getShort()  + " txid: " + transactionId);
+         LOG.error("Returning from callDropTable, (null tx) retval: " + TransReturnCode.RET_NOTX.getShort()  + " txId: " + transactionId);
          return TransReturnCode.RET_NOTX.getShort();
       }
 
@@ -798,8 +828,14 @@ public class HBaseTxClient {
                                     long pv_startcode,
                                     byte[] pv_regionInfo) throws IOException {
        String hostname    = new String(pv_hostname);
-       if (LOG.isTraceEnabled()) LOG.trace("Enter callRegisterRegion, txid: [" + transactionId + "], startId: " + startId + ", port: "
-           + pv_port + ", hostname: " + hostname + ", reg info len: " + pv_regionInfo.length + " " + new String(pv_regionInfo, "UTF-8"));
+       if (LOG.isTraceEnabled()) LOG.trace("Enter callRegisterRegion, "
+					   + "txId: [" + transactionId + "]" 
+					   + ", startId: " + startId 
+					   + ", port: " + pv_port 
+					   + ", hostname: " + hostname 
+					   + ", startcode: " + pv_startcode 
+					   + ", reg info len: " + pv_regionInfo.length 
+					   + " " + new String(pv_regionInfo, "UTF-8"));
 
        HRegionInfo lv_regionInfo;
        try {
@@ -818,7 +854,7 @@ public class HBaseTxClient {
 
        TransactionState ts = mapTransactionStates.get(transactionId);
        if(ts == null) {
-          if (LOG.isTraceEnabled()) LOG.trace("HBaseTxClient:callRegisterRegion transactionId (" + transactionId +
+          if (LOG.isTraceEnabled()) LOG.trace("callRegisterRegion transactionId (" + transactionId +
                    ") not found in mapTransactionStates of size: " + mapTransactionStates.size());
           try {
              ts = trxManager.beginTransaction(transactionId);
@@ -831,38 +867,35 @@ public class HBaseTxClient {
              if (ts2 != null) {
                 // Some other thread added the transaction while we were creating one.  It's already in the
                 // map, so we can use the existing one.
-                if (LOG.isTraceEnabled()) LOG.trace("HBaseTxClient:callRegisterRegion, found TransactionState object while creating a new one " + ts2);
+                if (LOG.isTraceEnabled()) LOG.trace("callRegisterRegion, found TransactionState object while creating a new one " + ts2);
                 ts = ts2;
              }
              else {
                 ts.setStartId(startId);
-                if (LOG.isTraceEnabled()) LOG.trace("HBaseTxClient:callRegisterRegion new transactionState created: " + ts );
+                if (LOG.isTraceEnabled()) LOG.trace("callRegisterRegion new transactionState created: " + ts );
              }
           }// end synchronized
        }
        else {
-          if (LOG.isTraceEnabled()) LOG.trace("HBaseTxClient:callRegisterRegion existing transactionState found: " + ts );
+          if (LOG.isTraceEnabled()) LOG.trace("callRegisterRegion existing transactionState found: " + ts );
           if (ts.getStartId() == -1) {
             ts.setStartId(startId);
-            if (LOG.isTraceEnabled()) LOG.trace("HBaseTxClient:callRegisterRegion reset startId for transactionState: " + ts );
+            if (LOG.isTraceEnabled()) LOG.trace("callRegisterRegion reset startId for transactionState: " + ts );
           }
        }
 
        try {
           trxManager.registerRegion(ts, regionLocation);
        } catch (IOException e) {
-          LOG.error("HBaseTxClient:callRegisterRegion exception in registerRegion call, txid: " + transactionId +
-            " retval: " + TransReturnCode.RET_EXCEPTION.toString() + " IOException " , e);
+          LOG.error("callRegisterRegion exception in registerRegion call, txId: " + transactionId +
+            " retval: " + TransReturnCode.RET_EXCEPTION.toString() + " IOException " + e);
           return TransReturnCode.RET_EXCEPTION.getShort();
        }
 
        if (LOG.isDebugEnabled()) LOG.debug("RegisterRegion adding table name " + regionTableName);
        ts.addTableName(regionTableName);
 
-       // Removing unnecessary put back into the map
-       // mapTransactionStates.put(ts.getTransactionId(), ts);
-
-       if (LOG.isTraceEnabled()) LOG.trace("Exit callRegisterRegion, txid: [" + transactionId + "] with mapsize: "
+       if (LOG.isTraceEnabled()) LOG.trace("Exit callRegisterRegion, txId: [" + transactionId + "] with mapsize: "
                   + mapTransactionStates.size());
        return TransReturnCode.RET_OK.getShort();
    }
@@ -893,7 +926,7 @@ public class HBaseTxClient {
       if (LOG.isTraceEnabled()) LOG.trace("Enter addControlPoint");
       long result = 0L;
       if (LOG.isTraceEnabled()) LOG.trace("HBaseTxClient calling tLog.addControlPoint with mapsize " + mapTransactionStates.size());
-      result = tLog.addControlPoint(mapTransactionStates);
+      result = tLog.addControlPoint(myClusterId, mapTransactionStates, true);
       Long lowestStartId = Long.MAX_VALUE;
       for(ConcurrentHashMap.Entry<Long, TransactionState> entry : mapTransactionStates.entrySet()){
           TransactionState value;
@@ -1019,6 +1052,8 @@ public class HBaseTxClient {
              private boolean useForgotten;
              private boolean forceForgotten;
              private boolean useTlog;
+             private boolean leadtm;
+             private boolean takeover;
              HBaseTxClient hbtx;
              private static int envSleepTimeInt;
              private boolean ddlOnlyRecoveryCheck = true;
@@ -1038,36 +1073,43 @@ public class HBaseTxClient {
                                HBaseTxClient hbtx,
                                boolean useForgotten,
                                boolean forceForgotten,
-                               boolean useTlog) {
+                               boolean useTlog,
+                               boolean leadtm,
+                               boolean takeover) {
              this(audit, zookeeper, txnManager);
              this.hbtx = hbtx;
              this.useForgotten = useForgotten;
              this.forceForgotten = forceForgotten;
              this.useTlog= useTlog;
-         }
+             this.leadtm = leadtm;
+             this.takeover = takeover;
+             if(LOG.isDebugEnabled()) LOG.debug("Traf Recovery Thread starts for DTM " + tmID +
+                             " LDTM " + leadtm + " Takeover " + takeover);
+
+            }
              /**
               *
               * @param audit
               * @param zookeeper
               * @param txnManager
               */
-             public RecoveryThread(TmAuditTlog audit,
+            public RecoveryThread(TmAuditTlog audit,
                                    HBaseTmZK zookeeper,
                                    TransactionManager txnManager)
-             {
+            {
                           this.audit = audit;
                           this.zookeeper = zookeeper;
                           this.txnManager = txnManager;
                           this.inDoubtList = new HashSet<Long> ();
                           this.tmID = zookeeper.getTMID();
                           this.sleepTimeInt = envSleepTimeInt;
-             }
+            }
 
-             public void stopThread() {
+            public void stopThread() {
                  this.continueThread = false;
-             }
+            }
 
-             private void addRegionToTS(String hostnamePort, byte[] regionInfo, TransactionState ts) throws IOException{
+            private void addRegionToTS(String hostnamePort, byte[] regionInfo, TransactionState ts) throws IOException{
                  HRegionInfo regionInfoLoc; // = new HRegionInfo();
                  final byte [] delimiter = ",".getBytes();
                  String[] result = hostnamePort.split(new String(delimiter), 3);
@@ -1141,10 +1183,10 @@ public class HBaseTxClient {
                             if(transactionStates != null)
                                 recoverTransactions(transactionStates);
 
-                        }
+                        } // region not null
                         else {
                             if (recoveryIterations > 0) {
-                                if(LOG.isInfoEnabled()) LOG.info("Recovery completed for TM" + tmID);
+                                if(LOG.isDebugEnabled()) LOG.debug("Recovery completed for TM" + tmID);
                             }
                             recoveryIterations = -1;
                         }
@@ -1156,7 +1198,7 @@ public class HBaseTxClient {
                             }
                             retryCount = 0;
                         } catch (InterruptedException e) {
-                            LOG.error("Error in recoveryThread: " + e);
+                            LOG.error("Error in recoveryThread: ", e);
                         }
                     } catch (IOException e) {
                         int possibleRetries = 4;
@@ -1351,6 +1393,7 @@ public class HBaseTxClient {
         
         for (Map.Entry<Long, TransactionState> tsEntry : transactionStates.entrySet()) {
             int isTransactionStillAlive = 0;
+           TransactionState ts1 = null;
             TransactionState ts = tsEntry.getValue();
             Long txID = ts.getTransactionId();
             // TransactionState ts = new TransactionState(txID);
@@ -1379,9 +1422,16 @@ public class HBaseTxClient {
                         LOG.debug("TRAF RCOV THREAD:Redriving commit for " + txID + " number of regions " + ts.getParticipatingRegions().size() +
                                 " and tolerating UnknownTransactionExceptions");
                     txnManager.doCommit(ts, true /*ignore UnknownTransactionException*/);
-                    if(useTlog && useForgotten) {
-                        long nextAsn = tLog.getNextAuditSeqNum((int)TransactionState.getNodeId(txID));
-                        tLog.putSingleRecord(txID, ts.getCommitId(), "FORGOTTEN", null, forceForgotten, nextAsn);
+                    if(useTlog) {
+                        long nextAsn = tLog.getNextAuditSeqNum(myClusterId, (int)TransactionState.getNodeId(txID));
+                        tLog.putSingleRecord(txID,
+                                             ts.getStartId(),
+                                             ts.getCommitId(),
+                                             TransState.STATE_RECOVERY_COMMITTED.toString(),
+                                             ts.getParticipatingRegions(),
+                                             false,
+                                             forceForgotten,
+                                             nextAsn);
                     }
                 } else if (ts.getStatus().equals(TransState.STATE_ABORTED.toString())) {
                     if (LOG.isDebugEnabled())
@@ -1436,7 +1486,7 @@ public class HBaseTxClient {
       TransactionState value;
       int tnum = 0; // Transaction number
 
-      if (LOG.isTraceEnabled()) LOG.trace("HBaseTxClient::callRequestRegionInfo:: start\n");
+      if (LOG.isTraceEnabled()) LOG.trace(":callRequestRegionInfo:: start\n");
 
       HashMapArray hm = new HashMapArray();
 
@@ -1448,40 +1498,64 @@ public class HBaseTxClient {
           TransactionState ts = mapTransactionStates.get(id);
           final Set<TransactionRegionLocation> regions = ts.getParticipatingRegions();
 
+          // TableName
           Iterator<TransactionRegionLocation> it = regions.iterator();
-          
-          while(it.hasNext()) {
-              TransactionRegionLocation trl = it.next();
-              tablename = trl.getRegionInfo().getTable().getNameAsString();
-              if(tablename.contains("TRAFODION._MD_."))
-                 continue;
-              encoded_region_name = trl.getRegionInfo().getEncodedName();
-              region_name = trl.getRegionInfo().getRegionNameAsString();
-              boolean is_offline_bool = trl.getRegionInfo().isOffline();
-              is_offline = String.valueOf(is_offline_bool);
-              region_id = String.valueOf(trl.getRegionInfo().getRegionId());
-              thn = String.valueOf(trl.getHostname());
-              hostname = thn.substring(0, thn.length()-1);
-              port = String.valueOf(trl.getPort());              
-              startkey = Bytes.equals(trl.getRegionInfo().getStartKey(), HConstants.EMPTY_START_ROW) ?
-                            "INFINITE" : Hex.encodeHexString(trl.getRegionInfo().getStartKey()); 
-              endkey   = Bytes.equals(trl.getRegionInfo().getEndKey(), HConstants.EMPTY_END_ROW) ?
-                            "INFINITE" : Hex.encodeHexString(trl.getRegionInfo().getEndKey()); 
-
-              StringBuilder inputStr = new StringBuilder();
-              inputStr.append(tablename).append(";");
-              inputStr.append(encoded_region_name).append(";");
-              inputStr.append(region_name).append(";");
-              inputStr.append(region_id).append(";");
-              inputStr.append(hostname).append(";");
-              inputStr.append(port).append(";");
-              inputStr.append(startkey).append(";");
-              inputStr.append(endkey);
-              hm.appendRegionInfo(id,  inputStr.toString());
+          tablename = it.next().getRegionInfo().getTable().getNameAsString();
+          while(it.hasNext()){
+              tablename = tablename + ";" + it.next().getRegionInfo().getTable().getNameAsString();
+          }
+          hm.addElement(tnum, "TableName", tablename);
 
-         }
-         tnum = tnum + 1;
-      }
+          // Encoded Region Name
+          Iterator<TransactionRegionLocation> it2 = regions.iterator();
+          encoded_region_name = it2.next().getRegionInfo().getEncodedName();
+          while(it2.hasNext()){
+              encoded_region_name = encoded_region_name + ";" + it2.next().getRegionInfo().getTable().getNameAsString();
+          }
+          hm.addElement(tnum, "EncodedRegionName", encoded_region_name);
+
+          // Region Name
+          Iterator<TransactionRegionLocation> it3 = regions.iterator();
+          region_name = it3.next().getRegionInfo().getRegionNameAsString();
+          while(it3.hasNext()){
+              region_name = region_name + ";" + it3.next().getRegionInfo().getTable().getNameAsString();
+          }
+          hm.addElement(tnum, "RegionName", region_name);
+
+          // Region Offline
+          Iterator<TransactionRegionLocation> it4 = regions.iterator();
+          boolean is_offline_bool = it4.next().getRegionInfo().isOffline();
+          is_offline = String.valueOf(is_offline_bool);
+          hm.addElement(tnum, "RegionOffline", is_offline);
+
+          // Region ID
+          Iterator<TransactionRegionLocation> it5 = regions.iterator();
+          region_id = String.valueOf(it5.next().getRegionInfo().getRegionId());
+          while(it5.hasNext()){
+              region_id = region_id + ";" + it5.next().getRegionInfo().getRegionId();
+          }
+          hm.addElement(tnum, "RegionID", region_id);
+
+          // Hostname
+          Iterator<TransactionRegionLocation> it6 = regions.iterator();
+          thn = String.valueOf(it6.next().getHostname());
+          hostname = thn.substring(0, thn.length()-1);
+          while(it6.hasNext()){
+              thn = String.valueOf(it6.next().getHostname());
+              hostname = hostname + ";" + thn.substring(0, thn.length()-1);
+          }
+          hm.addElement(tnum, "Hostname", hostname);
+
+          // Port
+          Iterator<TransactionRegionLocation> it7 = regions.iterator();
+          port = String.valueOf(it7.next().getPort());
+          while(it7.hasNext()){
+              port = port + ";" + String.valueOf(it7.next().getPort());
+          }
+          hm.addElement(tnum, "Port", port);
+
+          tnum = tnum + 1;
+        }
       if (LOG.isTraceEnabled()) LOG.trace("HBaseTxClient::callRequestRegionInfo:: end size: " + hm.getSize());
       return hm;
    }



[04/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

Posted by sa...@apache.org.
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/regionserver/transactional/SsccTransactionState.java
----------------------------------------------------------------------
diff --git a/core/sqf/src/seatrans/hbase-trx/src/main/java/org/apache/hadoop/hbase/regionserver/transactional/SsccTransactionState.java b/core/sqf/src/seatrans/hbase-trx/src/main/java/org/apache/hadoop/hbase/regionserver/transactional/SsccTransactionState.java
index 31b35d4..dfd06a9 100644
--- a/core/sqf/src/seatrans/hbase-trx/src/main/java/org/apache/hadoop/hbase/regionserver/transactional/SsccTransactionState.java
+++ b/core/sqf/src/seatrans/hbase-trx/src/main/java/org/apache/hadoop/hbase/regionserver/transactional/SsccTransactionState.java
@@ -84,7 +84,7 @@ public class SsccTransactionState extends TransactionState{
     public SsccTransactionState(final long transactionId, final long rLogStartSequenceId, AtomicLong hlogSeqId,
     		final HRegionInfo regionInfo, HTableDescriptor htd, WAL hLog, boolean logging, long SsccSequenceId) {
 
-       super(transactionId,rLogStartSequenceId,hlogSeqId,regionInfo,htd,hLog,logging, /* region TX */ false);
+       super(transactionId,rLogStartSequenceId,hlogSeqId,regionInfo,htd,hLog,logging,SsccSequenceId, /* region TX */ false);
        setStartId(SsccSequenceId);
        if(LOG.isTraceEnabled()) LOG.trace("SsccTransactionState : new state object for transid: " + transactionId + " with sequence: " + SsccSequenceId + " complete");
     }
@@ -93,7 +93,7 @@ public class SsccTransactionState extends TransactionState{
             final HRegionInfo regionInfo, HTableDescriptor htd, WAL hLog, boolean logging, long SsccSequenceId,
             boolean regionTx) {
 
-        super(transactionId,rLogStartSequenceId,hlogSeqId,regionInfo,htd,hLog,logging, regionTx);
+        super(transactionId,rLogStartSequenceId,hlogSeqId,regionInfo,htd,hLog,logging,SsccSequenceId,regionTx);
         setStartId(SsccSequenceId);
         if(LOG.isTraceEnabled()) LOG.trace("SsccTransactionState : new state object for transid: " + transactionId + " with sequence: " + SsccSequenceId + " complete");
     }

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/regionserver/transactional/TransactionState.java
----------------------------------------------------------------------
diff --git a/core/sqf/src/seatrans/hbase-trx/src/main/java/org/apache/hadoop/hbase/regionserver/transactional/TransactionState.java b/core/sqf/src/seatrans/hbase-trx/src/main/java/org/apache/hadoop/hbase/regionserver/transactional/TransactionState.java
index a7eae1f..e48062a 100644
--- a/core/sqf/src/seatrans/hbase-trx/src/main/java/org/apache/hadoop/hbase/regionserver/transactional/TransactionState.java
+++ b/core/sqf/src/seatrans/hbase-trx/src/main/java/org/apache/hadoop/hbase/regionserver/transactional/TransactionState.java
@@ -72,6 +72,9 @@ public class TransactionState {
 
     protected static final Log LOG = LogFactory.getLog(TransactionState.class);
 
+    protected long startId_;
+    protected long commitSequenceId;
+
     /** Current commit progress */
     public enum CommitProgress {
         /** Initial status, still performing operations. */
@@ -88,6 +91,10 @@ public class TransactionState {
          * Checked if we can commit, and said yes. Still need to determine the global decision.
          */
         COMMIT_PENDING,
+        /**
+         * Checked if we can commit, and writeOrdering is empty.
+         */
+        COMMIT_READONLY,
         /** Committed. */
         COMMITED,
         /** Aborted. */
@@ -106,7 +113,10 @@ public class TransactionState {
     protected long controlPointEpochAtPrepare = 0;
     protected int reInstated = 0;
     protected long flushTxId = 0;
+    protected long nodeId;
+    protected long clusterId;
 
+    protected boolean neverReadOnly = false;
     protected boolean splitRetry = false;
     protected boolean earlyLogging = false;
     protected boolean commit_TS_CC = false;
@@ -124,7 +134,8 @@ public class TransactionState {
     public static final byte TS_TRAFODION_TXN_TAG_TYPE = 41;
 
     public TransactionState(final long transactionId, final long rLogStartSequenceId, AtomicLong hlogSeqId, final HRegionInfo regionInfo,
-                                                 HTableDescriptor htd, WAL hLog, boolean logging, boolean isRegionTx) {
+                                                 HTableDescriptor htd, WAL hLog, boolean logging, final long startId,
+                                                 boolean isRegionTx) {
         Tag transactionalTag = null;
         if (LOG.isTraceEnabled()) LOG.trace("Create TS object for " + transactionId + " early logging " + logging);
         this.transactionId = transactionId;
@@ -136,14 +147,18 @@ public class TransactionState {
         this.tabledescriptor = htd;
         this.earlyLogging = logging;
         this.tHLog = hLog;
+        setStartId(startId);
+        setNodeId();
+        setClusterId();
+
         if(isRegionTx){ // RegionTX takes precedence
-           transactionalTag = this.formTransactionalContextTag(TS_REGION_TX_COMMIT_REQUEST);
+           transactionalTag = this.formTransactionalContextTag(TS_REGION_TX_COMMIT_REQUEST, startId);
         }
         else if (earlyLogging) {
-           transactionalTag = this.formTransactionalContextTag(TS_ACTIVE);
+           transactionalTag = this.formTransactionalContextTag(TS_ACTIVE, startId);
         }
         else {
-           transactionalTag = this.formTransactionalContextTag(TS_COMMIT_REQUEST);
+           transactionalTag = this.formTransactionalContextTag(TS_COMMIT_REQUEST, startId);
         }
         tagList.add(transactionalTag);
     }
@@ -172,14 +187,15 @@ public class TransactionState {
        return result;
     }
 
-    public Tag formTransactionalContextTag(int transactionalOp) {
+    public Tag formTransactionalContextTag(int transactionalOp, long ts) {
         byte[] tid = Bytes.toBytes (this.transactionId);
         byte[] logSeqId = Bytes.toBytes(this.hLogStartSequenceId);
         byte[] type = Bytes.toBytes(transactionalOp);
         int vers = 1;
         byte[] version = Bytes.toBytes(vers);
+        byte[] tsId = Bytes.toBytes(ts);
 
-        byte[] tagBytes = concat(version, type, tid, logSeqId);
+        byte[] tagBytes = concat(version, type, tid, logSeqId, tsId);
         byte tagType = TS_TRAFODION_TXN_TAG_TYPE;
         Tag tag = new Tag(tagType, tagBytes);
         return tag;
@@ -197,6 +213,18 @@ public class TransactionState {
             }
         }
     }
+
+   // Same as updateLatestTimestamp except there is no test for isLatestTimestamp()
+   public  static void unconditionalUpdateLatestTimestamp(final Collection<List<Cell>> kvsCollection, final long time) {
+       byte[] timeBytes = Bytes.toBytes(time);
+       // HAVE to manually set the KV timestamps
+       for (List<Cell> kvs : kvsCollection) {
+           for (Cell cell : kvs) {
+             KeyValue kv = KeyValueUtil.ensureKeyValue(cell);
+             kv.updateLatestStamp(timeBytes);
+           }
+       }
+   }
    /**
     * Returns a boolean indicating whether or not this is a region transaction.
     *
@@ -214,7 +242,7 @@ public class TransactionState {
      */
     public long getNodeId() {
 
-        return ((transactionId >> 32) & 0xFFL);
+        return nodeId;
     }
 
     /**
@@ -226,6 +254,15 @@ public class TransactionState {
 
         return ((transId >> 32) & 0xFFL);
     }
+
+    /**
+     * Set the originating node of the transaction.
+     *
+     */
+    private void setNodeId() {
+       nodeId = ((transactionId >> 32) & 0xFFL);
+    }
+
     /**
      * Get the originating cluster of the passed in transaction.
      *
@@ -233,9 +270,27 @@ public class TransactionState {
      */
     public static long getClusterId(long transId) {
 
-        return ((transId >> 48) & 0xFFL);
+        return (transId >> 48);
     }
 
+    /**
+     * Get the originating cluster of the passed in transaction.
+     *
+     * @return Return the clusterId.
+     */
+    public long getClusterId() {
+
+        return clusterId;
+    }
+
+    /**
+     * Set the originating clusterId of the passed in transaction.
+     *
+     */
+    private void setClusterId() {
+
+        clusterId = (transactionId >> 48);
+    }
 
     /**
      * Get the status.
@@ -250,6 +305,14 @@ public class TransactionState {
       return logSeqId.get();
     }
 
+    public void setNeverReadOnly(boolean value) {
+      neverReadOnly = value;
+    }
+
+    public boolean getNeverReadOnly() {
+      return neverReadOnly;
+    }
+
     public void setSplitRetry(boolean value) {
       splitRetry = value;
     }
@@ -278,6 +341,33 @@ public class TransactionState {
        return xaOperation;
     }
 
+    public void setStartId(long startId)
+    {
+        startId_ = startId;
+    }
+
+    public long getStartId()
+    {
+        return startId_;
+    }
+
+    /**
+     * Get the commitId for this transaction.
+     * 
+     * @return Return the commitSequenceId.
+     */
+    public synchronized long getCommitId() {
+        return commitSequenceId;
+    }
+
+    /**
+     * Set the commitId for this transaction.
+     * 
+     */
+    public synchronized void setCommitId(final long Id) {
+        this.commitSequenceId = Id;
+    }
+
     /**
      * Get the CP epoch at Prepare.
      * 
@@ -381,7 +471,7 @@ public class TransactionState {
     @Override
     public String toString() {
         return "transactionId: " + transactionId + ", regionTX: " + getIsRegionTx()
-                + ", status: " + status + ", regionInfo: " + regionInfo;
+                + ", status: " + status + ", neverReadOnly " + neverReadOnly + ", regionInfo: " + regionInfo;
     }
 
 

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/regionserver/transactional/TransactionalRegionScannerHolder.java
----------------------------------------------------------------------
diff --git a/core/sqf/src/seatrans/hbase-trx/src/main/java/org/apache/hadoop/hbase/regionserver/transactional/TransactionalRegionScannerHolder.java b/core/sqf/src/seatrans/hbase-trx/src/main/java/org/apache/hadoop/hbase/regionserver/transactional/TransactionalRegionScannerHolder.java
index c796c2c..b4096f0 100644
--- a/core/sqf/src/seatrans/hbase-trx/src/main/java/org/apache/hadoop/hbase/regionserver/transactional/TransactionalRegionScannerHolder.java
+++ b/core/sqf/src/seatrans/hbase-trx/src/main/java/org/apache/hadoop/hbase/regionserver/transactional/TransactionalRegionScannerHolder.java
@@ -25,7 +25,7 @@ package org.apache.hadoop.hbase.regionserver.transactional;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.hbase.regionserver.HRegion;
+import org.apache.hadoop.hbase.regionserver.Region;
 import org.apache.hadoop.hbase.regionserver.RegionScanner;
 
 /**
@@ -33,7 +33,7 @@ import org.apache.hadoop.hbase.regionserver.RegionScanner;
  */
  public class TransactionalRegionScannerHolder {
     public RegionScanner s;
-    public HRegion r;
+    public Region r;
     public long nextCallSeq;
     public long numberOfRows;
     public long rowsRemaining;
@@ -41,10 +41,11 @@ import org.apache.hadoop.hbase.regionserver.RegionScanner;
     public long scannerId;
     public boolean hasMore;
 
+
     public TransactionalRegionScannerHolder(long transId,
                                             long scannerId,
                                             RegionScanner s, 
-                                            HRegion r) {
+                                            Region r) {
       this.transId = transId;
       this.scannerId = scannerId;
       this.s = s;

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/regionserver/transactional/TrxTransactionState.java.tmpl
----------------------------------------------------------------------
diff --git a/core/sqf/src/seatrans/hbase-trx/src/main/java/org/apache/hadoop/hbase/regionserver/transactional/TrxTransactionState.java.tmpl b/core/sqf/src/seatrans/hbase-trx/src/main/java/org/apache/hadoop/hbase/regionserver/transactional/TrxTransactionState.java.tmpl
index 6ec061d..131d1ee 100644
--- a/core/sqf/src/seatrans/hbase-trx/src/main/java/org/apache/hadoop/hbase/regionserver/transactional/TrxTransactionState.java.tmpl
+++ b/core/sqf/src/seatrans/hbase-trx/src/main/java/org/apache/hadoop/hbase/regionserver/transactional/TrxTransactionState.java.tmpl
@@ -76,12 +76,18 @@ import org.apache.hadoop.hbase.regionserver.ScannerContext;
 #endif
 #ifdef CDH5.7 APACHE1.2
 import org.apache.hadoop.hbase.HBaseConfiguration;
+import org.apache.hadoop.conf.Configuration;
 #endif
+import org.apache.hadoop.hbase.regionserver.HRegion;
 import org.apache.hadoop.hbase.regionserver.wal.WALEdit;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
 import org.apache.hadoop.hbase.util.VersionInfo;
 import org.apache.hadoop.io.DataInputBuffer;
+#ifdef CDH5.7 APACHE1.2
+import org.apache.hadoop.hbase.regionserver.MultiVersionConcurrencyControl.WriteEntry;
+import org.apache.hadoop.hbase.regionserver.MultiVersionConcurrencyControl;
+#endif
 
 /**
  * Holds the state of a transaction. This includes a buffer of all writes, a record of all reads / scans, and
@@ -90,29 +96,35 @@ import org.apache.hadoop.io.DataInputBuffer;
 public class TrxTransactionState extends TransactionState {
 
     static java.lang.reflect.Constructor c1_0 = null;
+#ifdef CDH5.7 APACHE1.2
+    static Configuration config = HBaseConfiguration.create();
+#endif
 
     static {
         String version = VersionInfo.getVersion();// the hbase version string, eg. "0.6.3-dev"
         LOG.info("Got info of Class ScanQueryMatcher for HBase version :" + version);
-        try {
-            NavigableSet<byte[]> lv_nvg = (NavigableSet<byte[]>) null;
-            c1_0 = ScanQueryMatcher.class.getConstructor(new Class[] { Scan.class, 
-                                                                                                                  ScanInfo.class,
-                                                                                                                  java.util.NavigableSet.class,
-                                                                                                                  ScanType.class,
-                                                                                                                  long.class,
-                                                                                                                  long.class,
-                                                                                                                  long.class,
-                                                                                                                  long.class,
-                                                                                                                  RegionCoprocessorHost.class });
-            if (c1_0 != null)
-                LOG.info("Got info of Class ScanQueryMatcher for HBase 1.0");
-
-        } catch (NoSuchMethodException exc_nsm) {
-            LOG.info("TrxRegionEndpoint coprocessor, No matching ScanQueryMatcher : Threw an exception");
-        }
-    }
-
+         try {
+             NavigableSet<byte[]> lv_nvg = (NavigableSet<byte[]>) null;
+             c1_0 = ScanQueryMatcher.class.getConstructor(
+                                   new Class [] {
+                                       Scan.class,
+                                       ScanInfo.class,
+                                       java.util.NavigableSet.class,
+                                       ScanType.class,
+                                       long.class,
+                                       long.class,
+                                       long.class,
+                                       long.class,
+                                       RegionCoprocessorHost.class
+                                   });
+             if (c1_0 != null)
+                 LOG.info("Got info of Class ScanQueryMatcher for HBase 1.0");
+             
+             }
+             catch (NoSuchMethodException exc_nsm) {
+                 LOG.info("TrxRegionEndpoint coprocessor, No matching ScanQueryMatcher : Threw an exception");
+             }
+         }
     /**
      * Simple container of the range of the scanners we've opened. Used to check for conflicting writes.
      */
@@ -155,9 +167,19 @@ public class TrxTransactionState extends TransactionState {
     private Set<TrxTransactionState> transactionsToCheck = Collections.synchronizedSet(new HashSet<TrxTransactionState>());
     private WALEdit e;
     private boolean dropTableRecorded;
+    private HRegion h_Region = null;
+    public long prepareEditSize = 0;
+    public long endEditSize = 0;
 
     public TrxTransactionState(final long transactionId, final long rLogStartSequenceId, AtomicLong hlogSeqId,
-            final HRegionInfo regionInfo, HTableDescriptor htd, WAL hLog, boolean logging) {
+            final HRegionInfo regionInfo, HTableDescriptor htd, WAL hLog, 
+#ifdef CDH5.7 APACHE1.2
+									  boolean logging, HRegion hRegion,
+#else									  
+									  boolean logging,
+#endif									  
+			       final long startId) {
+
        super(transactionId,
              rLogStartSequenceId,
              hlogSeqId,
@@ -165,16 +187,29 @@ public class TrxTransactionState extends TransactionState {
              htd,
              hLog,
              logging,
+             startId,
              false);  // not a region transaction
         this.e = new WALEdit();
         dropTableRecorded = false;
+#ifdef CDH5.7 APACHE1.2	
+	this.h_Region = hRegion;
+#endif	
     }
 
     public TrxTransactionState(final long transactionId, final long rLogStartSequenceId, AtomicLong hlogSeqId,
-            final HRegionInfo regionInfo, HTableDescriptor htd, WAL hLog, boolean logging, boolean regionTx) {
-        super(transactionId, rLogStartSequenceId, hlogSeqId, regionInfo, htd, hLog, logging, regionTx);
+            final HRegionInfo regionInfo, HTableDescriptor htd, WAL hLog,
+#ifdef CDH5.7 APACHE1.2
+									  boolean logging, HRegion hRegion,
+#else									  
+									  boolean logging,
+#endif									  			       
+			       final long startId, boolean regionTx) {
+        super(transactionId, rLogStartSequenceId, hlogSeqId, regionInfo, htd, hLog, logging, startId, regionTx);
         this.e = new WALEdit();
         dropTableRecorded = false;
+#ifdef CDH5.7 APACHE1.2	
+	this.h_Region = hRegion;
+#endif	
     }
 
     public void setDropTableRecorded(boolean dropTableRecord) {
@@ -195,53 +230,97 @@ public class TrxTransactionState extends TransactionState {
         scans.add(new ScanRange(rowKey, rowKey));
     }
 
-    public synchronized void addWrite(final Put write) {
-        if (LOG.isTraceEnabled())
-            LOG.trace("addWrite -- ENTRY: write: " + write.toString());
+    public synchronized void addWrite(final Put put, final boolean useStartId) {
+        if (!this.getStatus().equals(Status.PENDING)) { // Active
+            LOG.error("addWrite late checkin for transaction " + this);
+        }
+
+#ifdef CDH5.7 APACHE1.2	    
+        MultiVersionConcurrencyControl.WriteEntry writeEntry = null;
+        long mvccNum = 0;
+#endif
+        if (LOG.isTraceEnabled()) LOG.trace("addWrite -- ENTRY: useStartId: " + useStartId + " put: " + put.toString());
         WriteAction waction;
         KeyValue kv;
         WALEdit e1 = new WALEdit();
-        updateLatestTimestamp(write.getFamilyCellMap().values(), EnvironmentEdgeManager.currentTime());
+        // If useStartId is true we use the startId as the timestamp,
+        // otherwise we use the current timestamp
+        long now;
+        if (useStartId == true) {
+           now = getStartId();
+           unconditionalUpdateLatestTimestamp(put.getFamilyCellMap().values(), now);
+        }
+        else {
+           now = EnvironmentEdgeManager.currentTime();
+           updateLatestTimestamp(put.getFamilyCellMap().values(), now);
+        }
         // Adding read scan on a write action
-        addRead(new WriteAction(write).getRow());
+        addRead(new WriteAction(put).getRow());
 
         ListIterator<WriteAction> writeOrderIter = writeOrdering.listIterator(writeOrdering.size());
-        writeOrderIter.add(waction = new WriteAction(write));
+        writeOrderIter.add(waction = new WriteAction(put));
 
         if (this.earlyLogging) { // immediately write edit out to HLOG during DML (active transaction state)
-            for (Cell value : waction.getCells()) {
-                // KeyValue kv = KeyValueUtil.ensureKeyValue(value);
-                kv = KeyValue.cloneAndAddTags(value, tagList);
-                // if (LOG.isTraceEnabled()) LOG.trace("KV hex dump " + Hex.encodeHexString(kv.getValueArray()
-                // /*kv.getBuffer()*/));
-                e1.add(kv);
-                e.add(kv);
+           for (Cell value : waction.getCells()) {
+             //KeyValue kv = KeyValueUtil.ensureKeyValue(value);
+             kv = KeyValue.cloneAndAddTags(value, tagList);
+             if (LOG.isTraceEnabled()) LOG.trace("addWrite kv hex dump " + Hex.encodeHexString(kv.getValueArray() /*kv.getBuffer()*/));
+             e1.add(kv);
+             e.add(kv);
             }
-            try {
-                // long txid = this.tHLog.appendNoSync(this.regionInfo, this.regionInfo.getTable(),
-                // e1, new ArrayList<UUID>(), EnvironmentEdgeManager.currentTimeMillis(), this.tabledescriptor,
-                // this.logSeqId, false, HConstants.NO_NONCE, HConstants.NO_NONCE);
-                final WALKey wk = new WALKey(this.regionInfo.getEncodedNameAsBytes(), this.regionInfo.getTable(),
-                        EnvironmentEdgeManager.currentTime());
+           try {
+           //long txid = this.tHLog.appendNoSync(this.regionInfo, this.regionInfo.getTable(),
+           //         e1, new ArrayList<UUID>(), EnvironmentEdgeManager.currentTimeMillis(), this.tabledescriptor,
+           //         this.logSeqId, false, HConstants.NO_NONCE, HConstants.NO_NONCE);
+
 #ifdef CDH5.7 APACHE1.2
-                long txid = this.tHLog.append(this.tabledescriptor, this.regionInfo, wk, e1, false);
+		WALKey wk = new WALKey(this.regionInfo.getEncodedNameAsBytes(), 
+					     this.regionInfo.getTable(), 
+					     WALKey.NO_SEQUENCE_ID,
+					     EnvironmentEdgeManager.currentTime(),
+					     WALKey.EMPTY_UUIDS,
+					     HConstants.NO_NONCE,
+					     HConstants.NO_NONCE,
+					     this.h_Region.getMVCC());		
+		long txid = this.tHLog.append(this.tabledescriptor,this.regionInfo, wk, e1, false);
+		
+		writeEntry = wk.getWriteEntry();
+		mvccNum = writeEntry.getWriteNumber();
+		
+		if (writeEntry != null) {
+		    this.h_Region.getMVCC().completeAndWait(writeEntry);
+		    writeEntry = null;
+		}
+
 #else
-                long txid = this.tHLog.append(this.tabledescriptor, this.regionInfo, wk, e1, this.logSeqId, false, null);
+           	final WALKey wk = new WALKey(this.regionInfo.getEncodedNameAsBytes(), this.regionInfo.getTable(), now);
+		long txid = this.tHLog.append(this.tabledescriptor,this.regionInfo, wk , e1,	this.logSeqId, false, null);
 #endif
-                // if (LOG.isTraceEnabled()) LOG.trace("Trafodion Recovery: Y11 write edit to HLOG during put with txid
-                // " + txid + " ts flush id " + this.flushTxId);
-                if (txid > this.flushTxId)
-                    this.flushTxId = txid; // save the log txid into TS object, later sync on largestSeqid during phase
-                                           // 1
-            } catch (IOException exp1) {
-                LOG.info("TrxRegionEndpoint coprocessor addWrite writing to HLOG for early logging: Threw an exception");
-                // throw exp1;
-            }
-        } else { // edits are buffered in ts and written out to HLOG in phase 1
-            for (Cell value : waction.getCells()) {
-                kv = KeyValue.cloneAndAddTags(value, tagList);
-                e.add(kv);
-            }
+           //if (LOG.isTraceEnabled()) LOG.trace("Trafodion Recovery: Y11 write edit to HLOG during put with txid " + txid + " ts flush id " + this.flushTxId);
+           if (txid > this.flushTxId) this.flushTxId = txid; // save the log txid into TS object, later sync on largestSeqid during phase 1
+           }
+           catch (IOException exp1) {
+           LOG.info("TrxRegionEndpoint coprocessor addWrite writing to HLOG for early logging: Threw an exception");
+           //throw exp1;
+           }
+#ifdef CDH5.7 APACHE1.2    
+	    finally {
+	       if (writeEntry != null) {
+                   this.h_Region.getMVCC().completeAndWait(writeEntry);
+                   writeEntry = null;
+	       }	  
+	    } // finally
+#endif           
+        }
+        else { // edits are buffered in ts and written out to HLOG in phase 1
+               for (Cell value : waction.getCells()) {
+                    kv = KeyValue.cloneAndAddTags(value, tagList);
+                    if (LOG.isTraceEnabled()) LOG.trace("addWrite kv hex dump "
+                          + " key: " + Hex.encodeHexString(Bytes.toBytes(kv.getKeyString())) 
+                          + " timestamp: " + kv.getTimestamp() 
+                          + Hex.encodeHexString(kv.getValueArray() /*kv.getBuffer()*/));
+                    e.add(kv);
+                }
         }
         if (LOG.isTraceEnabled())
             LOG.trace("addWrite -- EXIT");
@@ -255,16 +334,32 @@ public class TrxTransactionState extends TransactionState {
         return writeOrdering.size();
     }
 
-    public synchronized void addDelete(final Delete delete) {
-        if (LOG.isTraceEnabled())
-            LOG.trace("addDelete -- ENTRY: delete: " + delete.toString());
+    public synchronized void addDelete(final Delete delete, final boolean useStartId) {
+       if (!this.getStatus().equals(Status.PENDING)) { // Active
+          LOG.error("addDelete late checkin for transaction " + this);
+       }
+#ifdef CDH5.7 APACHE1.2	    
+        MultiVersionConcurrencyControl.WriteEntry writeEntry = null;
+        long mvccNum = 0;
+#endif
+        if (LOG.isTraceEnabled()) LOG.trace("addDelete -- ENTRY: useStartId: " + useStartId + " delete: " + delete.toString());
 
         WriteAction waction;
-        WALEdit e1 = new WALEdit();
-        long now = EnvironmentEdgeManager.currentTime();
-        updateLatestTimestamp(delete.getFamilyCellMap().values(), now);
-        if (delete.getTimeStamp() == HConstants.LATEST_TIMESTAMP) {
-            delete.setTimestamp(now);
+	KeyValue kv;
+        WALEdit e1  = new WALEdit();
+        // If useStartId is true we use the startId as the timestamp,
+        // otherwise we use the current timestamp
+        long now;
+        if (useStartId == true) {
+           now = getStartId();
+           unconditionalUpdateLatestTimestamp(delete.getFamilyCellMap().values(), now);
+        }
+        else {
+           now = EnvironmentEdgeManager.currentTime();
+           updateLatestTimestamp(delete.getFamilyCellMap().values(), now);
+           if (delete.getTimeStamp() == HConstants.LATEST_TIMESTAMP) {
+              delete.setTimestamp(now);
+           }
         }
         deletes.add(delete);
 
@@ -272,40 +367,63 @@ public class TrxTransactionState extends TransactionState {
         writeOrderIter.add(waction = new WriteAction(delete));
 
         if (this.earlyLogging) {
-            for (Cell value : waction.getCells()) {
-                KeyValue kv = KeyValue.cloneAndAddTags(value, tagList);
-                e1.add(kv);
-                e.add(kv);
-            }
-            try {
-                // long txid = this.tHLog.appendNoSync(this.regionInfo, this.regionInfo.getTable(),
-                // e1, new ArrayList<UUID>(), EnvironmentEdgeManager.currentTimeMillis(), this.tabledescriptor,
-                // this.logSeqId, false, HConstants.NO_NONCE, HConstants.NO_NONCE);
-                final WALKey wk = new WALKey(this.regionInfo.getEncodedNameAsBytes(), this.regionInfo.getTable(),
-                        EnvironmentEdgeManager.currentTime());
+           for (Cell value : waction.getCells()) {
+               kv = KeyValue.cloneAndAddTags(value, tagList);
+               e1.add(kv);
+               e.add(kv);
+           }
+           try {
+           //long txid = this.tHLog.appendNoSync(this.regionInfo, this.regionInfo.getTable(),
+           //         e1, new ArrayList<UUID>(), EnvironmentEdgeManager.currentTimeMillis(), this.tabledescriptor,
+           //         this.logSeqId, false, HConstants.NO_NONCE, HConstants.NO_NONCE);
 
 #ifdef CDH5.7 APACHE1.2
-                long txid = this.tHLog.append(this.tabledescriptor, this.regionInfo, wk, e1, false);
+		WALKey wk = new WALKey(this.regionInfo.getEncodedNameAsBytes(), 
+					     this.regionInfo.getTable(), 
+					     WALKey.NO_SEQUENCE_ID,
+					     EnvironmentEdgeManager.currentTime(),
+					     WALKey.EMPTY_UUIDS,
+					     HConstants.NO_NONCE,
+					     HConstants.NO_NONCE,
+					     this.h_Region.getMVCC());		
+		long txid = this.tHLog.append(this.tabledescriptor,this.regionInfo, wk, e1, false);
+		
+		writeEntry = wk.getWriteEntry();
+		mvccNum = writeEntry.getWriteNumber();
+		
+		if (writeEntry != null) {
+		    this.h_Region.getMVCC().completeAndWait(writeEntry);
+		    writeEntry = null;
+		}
+
 #else
-                long txid = this.tHLog.append(this.tabledescriptor, this.regionInfo, wk, e1, this.logSeqId, false, null);
+           	final WALKey wk = new WALKey(this.regionInfo.getEncodedNameAsBytes(), this.regionInfo.getTable(), now);
+		long txid = this.tHLog.append(this.tabledescriptor,this.regionInfo, wk , e1,	this.logSeqId, false, null);
 #endif
-                // if (LOG.isTraceEnabled()) LOG.trace("Trafodion Recovery: Y00 write edit to HLOG during delete with
-                // txid " + txid + " ts flush id " + this.flushTxId);
-                if (txid > this.flushTxId)
-                    this.flushTxId = txid; // save the log txid into TS object, later sync on largestSeqid during phase
-                                           // 1
-            } catch (IOException exp1) {
-                LOG.info("TrxRegionEndpoint coprocessor addDelete writing to HLOG for early logging: Threw an exception");
-            }
-        } else {
-            for (Cell value : waction.getCells()) {
-                KeyValue kv = KeyValue.cloneAndAddTags(value, tagList);
-                e.add(kv);
-            }
-        }
 
-        if (LOG.isTraceEnabled())
-            LOG.trace("addDelete -- EXIT");
+                    //if (LOG.isTraceEnabled()) LOG.trace("Trafodion Recovery: Y00 write edit to HLOG during delete with txid " + txid + " ts flush id " + this.flushTxId);
+           if (txid > this.flushTxId) this.flushTxId = txid; // save the log txid into TS object, later sync on largestSeqid during phase 1
+           }
+           catch (IOException exp1) {
+           LOG.info("TrxRegionEndpoint coprocessor addDelete writing to HLOG for early logging: Threw an exception");
+           }
+#ifdef CDH5.7 APACHE1.2    
+	    finally {
+	       if (writeEntry != null) {
+                   this.h_Region.getMVCC().completeAndWait(writeEntry);
+                   writeEntry = null;
+	       }	  
+	    } // finally
+#endif           
+       }
+       else {
+               for (Cell value : waction.getCells()) {
+                    kv = KeyValue.cloneAndAddTags(value, tagList);
+                    e.add(kv);
+                } // all cells
+        } // not early logging
+    
+       if (LOG.isTraceEnabled()) LOG.trace("addDelete -- EXIT");
     }
 
     public synchronized void applyDeletes(final List<Cell> input, final long minTime, final long maxTime) {
@@ -512,12 +630,20 @@ public class TrxTransactionState extends TransactionState {
         result.append(getIsRegionTx());
         result.append(" status: ");
         result.append(status.name());
+        result.append(" neverReadOnly: ");
+        result.append(getNeverReadOnly());
         result.append(" scan Size: ");
         result.append(scans.size());
         result.append(" write Size: ");
         result.append(getWriteOrdering().size());
         result.append(" startSQ: ");
         result.append(startSequenceNumber);
+        result.append(" prepareEditSize: ");
+        result.append(prepareEditSize);
+        result.append(" endEditSize: ");
+        result.append(endEditSize);
+        result.append(" editSize: ");
+        result.append(getEdit().getCells().size());
         if (sequenceNumber != null) {
             result.append(" commitedSQ:");
             result.append(sequenceNumber);
@@ -541,9 +667,9 @@ public class TrxTransactionState extends TransactionState {
      * 
      * @return deletes
      */
-    public synchronized List<Delete> getDeletes() {
-        return deletes;
-    }
+    //public synchronized List<Delete> getDeletes() {
+    //    return deletes;
+    //}
 
     /**
      * Get a scanner to go through the puts and deletes from this transaction. Used to weave together the local trx puts
@@ -601,7 +727,8 @@ public class TrxTransactionState extends TransactionState {
 
     @SuppressWarnings("deprecation")
     private synchronized KeyValue[] getAllKVs(final Scan scan) {
-        // if (LOG.isTraceEnabled()) LOG.trace("getAllKVs -- ENTRY");
+        if (LOG.isTraceEnabled()) LOG.trace("getAllKVs -- ENTRY for transId " + this.getTransactionId()
+            + " with writeOrdering size " + this.getWriteOrdering().size());
         List<Cell> kvList = new ArrayList<Cell>();
 
         ListIterator<WriteAction> writeOrderIter = null;
@@ -694,7 +821,7 @@ public class TrxTransactionState extends TransactionState {
             
             //Store.ScanInfo scaninfo = new Store.ScanInfo(null, 0, 1, HConstants.FOREVER, false, 0, Cell.COMPARATOR);
 #ifdef CDH5.7 APACHE1.2
-            ScanInfo scaninfo = new ScanInfo(HBaseConfiguration.create(), null, 0, 1, HConstants.FOREVER,KeepDeletedCells.FALSE, 0, KeyValue.COMPARATOR);
+            ScanInfo scaninfo = new ScanInfo(getConfig(), null, 0, 1, HConstants.FOREVER,KeepDeletedCells.FALSE, 0, KeyValue.COMPARATOR);
 #else
             ScanInfo scaninfo = new ScanInfo(null, 0, 1, HConstants.FOREVER,KeepDeletedCells.FALSE, 0, KeyValue.COMPARATOR);
 #endif
@@ -1003,12 +1130,11 @@ public class TrxTransactionState extends TransactionState {
     }
 
     /**
-     * before adding a put to the writeOrdering list, check whether we have deleted a row with the same key. If true
-     * remove the delete before adding the put
+     * before adding a put to the writeOrdering list, check whether we have deleted a row with the
+     * same key.  If true remove the delete before adding the put
      */
     public void removeDelBeforePut(Put put) {
-        if (LOG.isTraceEnabled())
-            LOG.trace("removeDelBeforePut put : " + put);
+        if (LOG.isTraceEnabled()) LOG.trace("removeDelBeforePut put : " + put);
         byte[] putRow = put.getRow();
         KeyValue kv;
 
@@ -1032,4 +1158,9 @@ public class TrxTransactionState extends TransactionState {
             }
         }
     }
+#ifdef CDH5.7 APACHE1.2
+    public Configuration getConfig() {
+      return config;
+    }
+#endif
 }

http://git-wip-us.apache.org/repos/asf/incubator-trafodion/blob/91794b53/core/sqf/src/seatrans/hbase-trx/src/main/protobuf/TrxRegion.proto
----------------------------------------------------------------------
diff --git a/core/sqf/src/seatrans/hbase-trx/src/main/protobuf/TrxRegion.proto b/core/sqf/src/seatrans/hbase-trx/src/main/protobuf/TrxRegion.proto
index a0a1385..a69e6b8 100755
--- a/core/sqf/src/seatrans/hbase-trx/src/main/protobuf/TrxRegion.proto
+++ b/core/sqf/src/seatrans/hbase-trx/src/main/protobuf/TrxRegion.proto
@@ -36,6 +36,7 @@ message AbortTransactionRequest{
   required int64 transactionId = 2;
   required int32 participantNum = 3;
   required bool dropTableRecorded = 4;
+  optional bool ignoreUnknownTransactionException = 5;
 }
 
 message AbortTransactionResponse {
@@ -56,7 +57,8 @@ message AbortTransactionMultipleResponse {
 
 message BeginTransactionRequest{
   required int64 transactionId = 1;
-  required bytes regionName = 2;
+  required int64 startId = 2;
+  required bytes regionName = 3;
 }
 
 message BeginTransactionResponse {
@@ -67,8 +69,9 @@ message BeginTransactionResponse {
 message CommitRequest{
   required bytes regionName = 1;
   required int64 transactionId = 2;
-  required int32 participantNum = 3;
-  optional bool ignoreUnknownTransactionException = 4;
+  required int64 commitId = 3;
+  required int32 participantNum = 4;
+  optional bool ignoreUnknownTransactionException = 5;
 }
 
 message CommitResponse {
@@ -79,8 +82,9 @@ message CommitResponse {
 message CommitMultipleRequest{
   repeated bytes regionName = 1;
   required int64 transactionId = 2;
-  required int32 participantNum = 3;
-  optional bool ignoreUnknownTransactionException = 4;
+  required int64 commitId = 3;
+  required int32 participantNum = 4;
+  optional bool ignoreUnknownTransactionException = 5;
 }
 
 message CommitMultipleResponse {
@@ -130,12 +134,13 @@ message CommitIfPossibleResponse {
 
 message CheckAndDeleteRequest {
   required int64 transactionId = 1;
-  required bytes regionName = 2;
-  required bytes row = 3;
-  required bytes family = 4;
-  required bytes qualifier = 5;
-  required bytes value = 6;
-  required MutationProto delete = 7;
+  required int64 startId = 2;
+  required bytes regionName = 3;
+  required bytes row = 4;
+  required bytes family = 5;
+  required bytes qualifier = 6;
+  required bytes value = 7;
+  required MutationProto delete = 8;
 }
 
 message CheckAndDeleteResponse {
@@ -146,13 +151,14 @@ message CheckAndDeleteResponse {
 
 message CheckAndDeleteRegionTxRequest {
   required int64 tid = 1;
-  required bytes regionName = 2;
-  required bytes row = 3;
-  required bytes family = 4;
-  required bytes qualifier = 5;
-  required bytes value = 6;
-  required MutationProto delete = 7;
-  required bool autoCommit = 8;
+  required int64 commitId = 2;
+  required bytes regionName = 3;
+  required bytes row = 4;
+  required bytes family = 5;
+  required bytes qualifier = 6;
+  required bytes value = 7;
+  required MutationProto delete = 8;
+  required bool autoCommit = 9;
 }
 
 message CheckAndDeleteRegionTxResponse {
@@ -163,12 +169,13 @@ message CheckAndDeleteRegionTxResponse {
 
 message CheckAndPutRequest {
   required int64 transactionId = 1;
-  required bytes regionName = 2;
-  required bytes row = 3;
-  required bytes family = 4;
-  required bytes qualifier = 5;
-  required bytes value = 6;
-  required MutationProto put = 7;
+  required int64 startId = 2;
+  required bytes regionName = 3;
+  required bytes row = 4;
+  required bytes family = 5;
+  required bytes qualifier = 6;
+  required bytes value = 7;
+  required MutationProto put = 8;
 }
 
 message CheckAndPutResponse {
@@ -179,13 +186,14 @@ message CheckAndPutResponse {
 
 message CheckAndPutRegionTxRequest {
   required int64 tid = 1;
-  required bytes regionName = 2;
-  required bytes row = 3;
-  required bytes family = 4;
-  required bytes qualifier = 5;
-  required bytes value = 6;
-  required MutationProto put = 7;
-  required bool autoCommit = 8;
+  required int64 commitId = 2;
+  required bytes regionName = 3;
+  required bytes row = 4;
+  required bytes family = 5;
+  required bytes qualifier = 6;
+  required bytes value = 7;
+  required MutationProto put = 8;
+  required bool autoCommit = 9;
 }
 
 message CheckAndPutRegionTxResponse {
@@ -207,8 +215,9 @@ message CloseScannerResponse {
 
 message DeleteMultipleTransactionalRequest {
   required int64 transactionId = 1;
-  required bytes regionName = 2;
-  repeated MutationProto delete = 3;
+  required int64 startId = 2;
+  required bytes regionName = 3;
+  repeated MutationProto delete = 4;
 }
 
 message DeleteMultipleTransactionalResponse {
@@ -219,9 +228,10 @@ message DeleteMultipleTransactionalResponse {
 
 message DeleteRegionTxRequest {
   required int64 tid = 1;
-  required bytes regionName = 2;
-  required MutationProto delete = 3;
-  required bool autoCommit = 4;
+  required int64 commitId = 2;
+  required bytes regionName = 3;
+  required MutationProto delete = 4;
+  required bool autoCommit = 5;
 }
 
 message DeleteRegionTxResponse {
@@ -232,8 +242,9 @@ message DeleteRegionTxResponse {
 
 message DeleteTransactionalRequest {
   required int64 transactionId = 1;
-  required bytes regionName = 2;
-  required MutationProto delete = 3;
+  required int64 startId = 2;
+  required bytes regionName = 3;
+  required MutationProto delete = 4;
 }
 
 message DeleteTransactionalResponse {
@@ -244,8 +255,9 @@ message DeleteTransactionalResponse {
 
 message GetTransactionalRequest {
   required int64 transactionId = 1;
-  required bytes regionName = 2;
-  required Get get = 3;
+  required int64 startId = 2;
+  required bytes regionName = 3;
+  required Get get = 4;
 }
 
 message GetTransactionalResponse {
@@ -256,8 +268,9 @@ message GetTransactionalResponse {
 
 message OpenScannerRequest {
   required int64 transactionId = 1;
-  required bytes regionName = 2;
-  required Scan scan = 3;
+  required int64 startId = 2;
+  required bytes regionName = 3;
+  required Scan scan = 4;
 }
 
 message OpenScannerResponse {
@@ -268,11 +281,12 @@ message OpenScannerResponse {
 
 message PerformScanRequest {
   required int64 transactionId = 1;
-  required bytes regionName = 2;
-  required int64 scannerId = 3;
-  required int32 numberOfRows = 4;
-  required bool  closeScanner = 5;
-  required int64 nextCallSeq = 6;
+  required int64 startId = 2;
+  required bytes regionName = 3;
+  required int64 scannerId = 4;
+  required int32 numberOfRows = 5;
+  required bool  closeScanner = 6;
+  required int64 nextCallSeq = 7;
 }
 
 message PerformScanResponse {
@@ -286,9 +300,10 @@ message PerformScanResponse {
 
 message PutRegionTxRequest {
   required int64 tid = 1;
-  required bytes regionName = 2;
-  required MutationProto put = 3;
-  required bool  autoCommit = 4;
+  required int64 commitId = 2;
+  required bytes regionName = 3;
+  required MutationProto put = 4;
+  required bool  autoCommit = 5;
 }
 
 message PutRegionTxResponse {
@@ -299,8 +314,9 @@ message PutRegionTxResponse {
 
 message PutTransactionalRequest {
   required int64 transactionId = 1;
-  required bytes regionName = 2;
-  required MutationProto put = 3;
+  required int64 startId = 2;
+  required bytes regionName = 3;
+  required MutationProto put = 4;
 }
 
 message PutTransactionalResponse {
@@ -311,8 +327,9 @@ message PutTransactionalResponse {
 
 message PutMultipleTransactionalRequest {
   required int64 transactionId = 1;
-  required bytes regionName = 2;
-  repeated MutationProto put = 3;
+  required int64 startId = 2;
+  required bytes regionName = 3;
+  repeated MutationProto put = 4;
 }
 
 message PutMultipleTransactionalResponse {
@@ -346,55 +363,45 @@ message RecoveryRequestResponse {
 
 message TlogDeleteRequest{
   required bytes regionName = 1;
-  required int64 transactionId = 2;
-  required Scan scan = 3;
-  required int64 auditSeqNum = 4;
-  required bool ageCommitted = 5;
+  required Scan scan = 2;
+  required int64 auditSeqNum = 3;
+  required bool ageCommitted = 4;
 }
 
 message TlogDeleteResponse {
   repeated Result result = 1;
   required int64  count = 2;
-  optional string exception = 3;
-  optional bool hasException = 4;
+  optional bool hasException = 3;
+  optional string exception = 4;
 }
 
 message TlogWriteRequest{
   required bytes regionName = 1;
   required int64 transactionId = 2;
   required MutationProto put = 3;
-  required bytes row = 4;
-  required bytes family = 5;
-  required bytes qualifier = 6;
-  required bytes value = 7;
-  required int64 commitId = 8;
-  optional bool forced = 9;
+  required bytes family = 4;
+  required bytes qualifier = 5;
+  required int64 commitId = 6;
+  optional bool forced = 7;
 }
 
 message TlogWriteResponse {
   repeated int64 result = 1;
-  optional string exception = 2;
-  optional bool hasException = 3;
+  optional bool  hasException = 2;
+  optional string exception = 3;
 }
 
 message TlogTransactionStatesFromIntervalRequest{
-  required bytes regionName = 1;
-  required int64 transactionId = 2;
-  required int64 clusterId = 3;
-  required int64 auditSeqNum =4;
-  required int64 scannerId = 5;
-  required int32 numberOfRows = 6;
-  required int64 nextCallSeq = 7;
-  required bool  closeScanner = 8;
+  required int64 clusterId = 1;
+  required int64 auditSeqNum =2;
+  required Scan scan = 3;
 }
 
 message TlogTransactionStatesFromIntervalResponse {
   repeated Result result = 1;
   required int64  count = 2;
-  required int64  nextCallSeq = 3;
-  required bool   hasMore = 4;
-  optional string exception = 5;
-  optional bool hasException = 6;
+  optional string exception = 3;
+  optional bool hasException = 4;
 }
 
 message TransactionalAggregateRequest {
@@ -406,9 +413,11 @@ message TransactionalAggregateRequest {
    */
   required bytes regionName = 1;
   required int64 transactionId = 2;
-  required string interpreter_class_name = 3;
-  required Scan scan = 4;
-  optional bytes  interpreter_specific_bytes = 5;
+  required int64 startId = 3;
+  required string interpreter_class_name = 4;
+  required Scan scan = 5;
+  optional bytes  interpreter_specific_bytes = 6;
+  
 }
 
 message TransactionalAggregateResponse {

http://git-wip-us.apache.org/repos/asf/incubator-trafodion/blob/91794b53/core/sqf/src/seatrans/tm/hbasetmlib2/src/main/java/org/trafodion/dtm/HBaseAuditControlPoint.java
----------------------------------------------------------------------
diff --git a/core/sqf/src/seatrans/tm/hbasetmlib2/src/main/java/org/trafodion/dtm/HBaseAuditControlPoint.java b/core/sqf/src/seatrans/tm/hbasetmlib2/src/main/java/org/trafodion/dtm/HBaseAuditControlPoint.java
index ba81304..a595d42 100644
--- a/core/sqf/src/seatrans/tm/hbasetmlib2/src/main/java/org/trafodion/dtm/HBaseAuditControlPoint.java
+++ b/core/sqf/src/seatrans/tm/hbasetmlib2/src/main/java/org/trafodion/dtm/HBaseAuditControlPoint.java
@@ -83,10 +83,14 @@ public class HBaseAuditControlPoint {
     private Configuration config;
     private static String CONTROL_POINT_TABLE_NAME;
     private static final byte[] CONTROL_POINT_FAMILY = Bytes.toBytes("cpf");
-    private static final byte[] ASN_HIGH_WATER_MARK = Bytes.toBytes("hwm");
-    private static HTable table;
+    private static final byte[] CP_NUM_AND_ASN_HWM = Bytes.toBytes("hwm");
+    private HTable table;
     private boolean useAutoFlush;
     private boolean disableBlockCache;
+    private static final int versions = 10;
+    private static int myClusterId = 0;
+    private int TlogRetryDelay;
+    private int TlogRetryCount;
 
     public HBaseAuditControlPoint(Configuration config, Connection connection) throws IOException {
       if (LOG.isTraceEnabled()) LOG.trace("Enter HBaseAuditControlPoint constructor()");
@@ -96,17 +100,46 @@ public class HBaseAuditControlPoint {
       HTableDescriptor desc = new HTableDescriptor(TableName.valueOf(CONTROL_POINT_TABLE_NAME));
       HColumnDescriptor hcol = new HColumnDescriptor(CONTROL_POINT_FAMILY);
 
+      TlogRetryDelay = 3000; // 3 seconds
+      try {
+         String retryDelayS = System.getenv("TM_TLOG_RETRY_DELAY");
+         if (retryDelayS != null){
+            TlogRetryDelay = (Integer.parseInt(retryDelayS) > TlogRetryDelay ? Integer.parseInt(retryDelayS) : TlogRetryDelay);
+         }
+      }
+      catch (NumberFormatException e) {
+         if (LOG.isDebugEnabled()) LOG.debug("TM_TLOG_RETRY_DELAY is not valid in ms.env");
+      }
+
+      TlogRetryCount = 40;
+      try {
+         String retryCountS = System.getenv("TM_TLOG_RETRY_COUNT");
+         if (retryCountS != null){
+           TlogRetryCount = (Integer.parseInt(retryCountS) > TlogRetryCount ? Integer.parseInt(retryCountS) : TlogRetryCount);
+         }
+      }
+      catch (NumberFormatException e) {
+         if (LOG.isDebugEnabled()) LOG.debug("TM_TLOG_RETRY_COUNT is not valid in ms.env");
+      }
+
       disableBlockCache = false;
-      String blockCacheString = System.getenv("TM_TLOG_DISABLE_BLOCK_CACHE");
-      if (blockCacheString != null){
-         disableBlockCache = (Integer.parseInt(blockCacheString) != 0);
+      try {
+         String blockCacheString = System.getenv("TM_TLOG_DISABLE_BLOCK_CACHE");
+         if (blockCacheString != null){
+             disableBlockCache = (Integer.parseInt(blockCacheString) != 0);
          if (LOG.isDebugEnabled()) LOG.debug("disableBlockCache != null");
+         }
+      }
+      catch (NumberFormatException e) {
+         if (LOG.isDebugEnabled()) LOG.debug("TM_TLOG_DISABLE_BLOCK_CACHE is not valid in ms.env");
       }
       LOG.info("disableBlockCache is " + disableBlockCache);
       if (disableBlockCache) {
          hcol.setBlockCacheEnabled(false);
       }
 
+      hcol.setMaxVersions(versions);
+
       desc.addFamily(hcol);
 
       useAutoFlush = true;
@@ -120,7 +153,6 @@ public class HBaseAuditControlPoint {
       boolean lvControlPointExists = admin.tableExists(TableName.valueOf(CONTROL_POINT_TABLE_NAME));
       if (LOG.isDebugEnabled()) LOG.debug("HBaseAuditControlPoint lvControlPointExists " + lvControlPointExists);
       currControlPt = -1;
-      admin.close();
       if (lvControlPointExists == false) {
          try {
             if (LOG.isDebugEnabled()) LOG.debug("Creating the table " + CONTROL_POINT_TABLE_NAME);
@@ -128,7 +160,11 @@ public class HBaseAuditControlPoint {
             currControlPt = 1;
          }
          catch (TableExistsException e) {
-            LOG.error("Table " + CONTROL_POINT_TABLE_NAME + " already exists");
+            LOG.error("Table " + CONTROL_POINT_TABLE_NAME + " already exists", e);
+            throw new IOException(e);
+         }
+         finally{
+            admin.close();
          }
       }
       if (LOG.isDebugEnabled()) LOG.debug("try new HTable");
@@ -136,7 +172,8 @@ public class HBaseAuditControlPoint {
       table.setAutoFlushTo(this.useAutoFlush);
 
       if (currControlPt == -1){
-         currControlPt = getCurrControlPt();
+         if (LOG.isDebugEnabled()) LOG.debug("getting currControlPt for clusterId " + myClusterId);
+         currControlPt = getCurrControlPt(myClusterId);
       }
       if (LOG.isDebugEnabled()) LOG.debug("currControlPt is " + currControlPt);
 
@@ -144,167 +181,198 @@ public class HBaseAuditControlPoint {
       return;
     }
 
-   public long getCurrControlPt() throws IOException {
-      if (LOG.isTraceEnabled()) LOG.trace("getCurrControlPt:  start");
-      long highKey = -1;
-      if (LOG.isDebugEnabled()) LOG.debug("new Scan");
-      Scan s = new Scan();
-      s.setCaching(10);
-      s.setCacheBlocks(false);
-      if (LOG.isDebugEnabled()) LOG.debug("resultScanner");
-      ResultScanner ss = table.getScanner(s);
-      try {
-         long currKey;
-         String rowKey;
-         if (LOG.isDebugEnabled()) LOG.debug("entering for loop" );
-         for (Result r : ss) {
-            rowKey = new String(r.getRow());
-            if (LOG.isDebugEnabled()) LOG.debug("rowKey is " + rowKey );
-            currKey = Long.parseLong(rowKey);
-            if (LOG.isDebugEnabled()) LOG.debug("value is " + Long.parseLong(Bytes.toString(r.value())));
-            if (currKey > highKey) {
-               if (LOG.isDebugEnabled()) LOG.debug("Setting highKey to " + currKey);
-               highKey = currKey;
+    public long getCurrControlPt(final int clusterId) throws IOException {
+       if (LOG.isTraceEnabled()) LOG.trace("getCurrControlPt:  start, clusterId " + clusterId);
+       long lvCpNum = 1;
+
+       Get g = new Get(Bytes.toBytes(clusterId));
+       if (LOG.isDebugEnabled()) LOG.debug("getCurrControlPt attempting table.get");
+          Result r = table.get(g);
+          if (r.isEmpty())
+             return lvCpNum;
+          if (LOG.isDebugEnabled()) LOG.debug("getCurrControlPt Result: " + r);
+          String value = new String(Bytes.toString(r.getValue(CONTROL_POINT_FAMILY, CP_NUM_AND_ASN_HWM)));
+          // In theory the above value is the latestversion of the column
+          if (LOG.isDebugEnabled()) LOG.debug("getCurrControlPt for clusterId: " + clusterId + ", valueString is " + value);
+          StringTokenizer stok = new StringTokenizer(value, ",");
+          if (stok.hasMoreElements()) {
+             if (LOG.isTraceEnabled()) LOG.trace("Parsing record in getCurrControlPt");
+             String ctrlPtToken = stok.nextElement().toString();
+             lvCpNum = Long.parseLong(ctrlPtToken, 10);
+             if (LOG.isTraceEnabled()) LOG.trace("Value for getCurrControlPt and clusterId: "
+                               + clusterId + " is: " + lvCpNum);
+          }
+       return lvCpNum;
+    }
+
+   public long putRecord(final int clusterId, final long ControlPt, final long startingSequenceNumber) throws IOException {
+      if (LOG.isTraceEnabled()) LOG.trace("putRecord clusterId: " + clusterId + ", startingSequenceNumber (" + startingSequenceNumber + ")");
+      Put p = new Put(Bytes.toBytes(clusterId));
+      p.add(CONTROL_POINT_FAMILY, CP_NUM_AND_ASN_HWM, 
+    		  Bytes.toBytes(String.valueOf(ControlPt) + ","
+    	               + String.valueOf(startingSequenceNumber)));
+      boolean complete = false;
+      int retries = 0;
+      do {
+         try {
+       	    retries++;
+            if (LOG.isTraceEnabled()) LOG.trace("try table.put with cluster Id: " + clusterId + " and startingSequenceNumber " + startingSequenceNumber);
+            table.put(p);
+            if (useAutoFlush == false) {
+               if (LOG.isTraceEnabled()) LOG.trace("flushing controlpoint record");
+               table.flushCommits();
+            }
+            complete = true;
+            if (retries > 1){
+               if (LOG.isTraceEnabled()) LOG.trace("Retry successful in putRecord for cp: " + ControlPt + " on table "
+                        + table.getTableName().toString());                    	 
             }
          }
-      } finally {
-         ss.close();
-      }
-      if (LOG.isDebugEnabled()) LOG.debug("getCurrControlPt returning " + highKey);
-      return highKey;
-   }
-
-   public long putRecord(final long ControlPt, final long startingSequenceNumber) throws IOException {
-      if (LOG.isTraceEnabled()) LOG.trace("putRecord starting sequence number ("  + String.valueOf(startingSequenceNumber) + ")");
-      String controlPtString = new String(String.valueOf(ControlPt));
-      Put p = new Put(Bytes.toBytes(controlPtString));
-      p.add(CONTROL_POINT_FAMILY, ASN_HIGH_WATER_MARK, Bytes.toBytes(String.valueOf(startingSequenceNumber)));
-      if (LOG.isTraceEnabled()) LOG.trace("try table.put with starting sequence number " + startingSequenceNumber);
-      table.put(p);
-      if (useAutoFlush == false) {
-         if (LOG.isTraceEnabled()) LOG.trace("flushing controlpoint record");
-         table.flushCommits();
-      }
+         catch (IOException e){
+             LOG.error("Retrying putRecord on control point: " + ControlPt + " on control point table "
+                     + table.getTableName().toString() + " due to Exception " + e);
+//             locator.getRegionLocation(p.getRow(), true);
+             table.getRegionLocation(p.getRow(), true);
+             try {
+               Thread.sleep(TlogRetryDelay); // 3 second default
+             } catch (InterruptedException ie) {
+             }
+             if (retries == TlogRetryCount){
+                LOG.error("putRecord aborting due to excessive retries on on control point table : "
+                         + table.getTableName().toString() + " due to Exception; aborting ");
+                System.exit(1);
+             }
+         }
+      } while (! complete && retries < TlogRetryCount);  // default give up after 5 minutes
       if (LOG.isTraceEnabled()) LOG.trace("HBaseAuditControlPoint:putRecord returning " + ControlPt);
       return ControlPt;
    }
 
-   public ArrayList<String> getRecordList(String controlPt) throws IOException {
-      if (LOG.isTraceEnabled()) LOG.trace("getRecord");
-      ArrayList<String> transactionList = new ArrayList<String>();
-      Get g = new Get(Bytes.toBytes(controlPt));
-      Result r = table.get(g);
-      byte [] currValue = r.getValue(CONTROL_POINT_FAMILY, ASN_HIGH_WATER_MARK);
-      String recordString = new String(currValue);
-      if (LOG.isDebugEnabled()) LOG.debug("recordString is " + recordString);
-      StringTokenizer st = new StringTokenizer(recordString, ",");
-      while (st.hasMoreElements()) {
-        String token = st.nextElement().toString() ;
-        if (LOG.isDebugEnabled()) LOG.debug("token is " + token);
-        transactionList.add(token);
-      }
-
-      if (LOG.isTraceEnabled()) LOG.trace("getRecord - exit with list size (" + transactionList.size() + ")");
-      return transactionList;
-
-    }
+   public long getRecord(final int clusterId, final String controlPt) throws IOException {
 
-   public long getRecord(final String controlPt) throws IOException {
-      if (LOG.isTraceEnabled()) LOG.trace("getRecord " + controlPt);
+      if (LOG.isTraceEnabled()) LOG.trace("getRecord clusterId: " + clusterId + " controlPt: " + controlPt);
       long lvValue = -1;
-      Get g = new Get(Bytes.toBytes(controlPt));
-      String recordString;
-      Result r = table.get(g);
-      byte [] currValue = r.getValue(CONTROL_POINT_FAMILY, ASN_HIGH_WATER_MARK);
-      if (currValue != null)
-      {
-          recordString = new String (Bytes.toString(currValue));
-          if (LOG.isDebugEnabled()) LOG.debug("recordString is " + recordString);
-          lvValue = Long.parseLong(recordString, 10);
-      }
-      if (LOG.isTraceEnabled()) LOG.trace("getRecord - exit " + lvValue);
+      Get g = new Get(Bytes.toBytes(clusterId));
+      g.setMaxVersions(versions);  // will return last n versions of row
+      g.addColumn(CONTROL_POINT_FAMILY, CP_NUM_AND_ASN_HWM);
+      String ctrlPtToken;
+      String asnToken;
+         Result r = table.get(g);
+         if (r.isEmpty())
+            return lvValue;
+         List<Cell> list = r.getColumnCells(CONTROL_POINT_FAMILY, CP_NUM_AND_ASN_HWM);  // returns all versions of this column
+         for (Cell element : list) {
+            StringTokenizer stok = new StringTokenizer(Bytes.toString(CellUtil.cloneValue(element)), ",");
+            if (stok.hasMoreElements()) {
+               ctrlPtToken = stok.nextElement().toString();
+               if (LOG.isTraceEnabled()) LOG.trace("Parsing record for controlPt (" + ctrlPtToken + ")");
+               asnToken = stok.nextElement().toString();
+               if (Long.parseLong(ctrlPtToken, 10) == Long.parseLong(controlPt, 10)){
+                  // This is the one we are looking for
+                  lvValue = Long.parseLong(asnToken, 10);
+                  if (LOG.isTraceEnabled()) LOG.trace("ASN value for controlPt: " + controlPt + " is: " + lvValue);
+                  return lvValue;
+               }
+            }
+            else {
+               if (LOG.isTraceEnabled()) LOG.trace("No tokens to parse for controlPt (" + controlPt + ")");
+            }
+         }
+         if (LOG.isTraceEnabled()) LOG.trace("all results scannned for clusterId: " + clusterId + ", but controlPt: " + controlPt + " not found");
       return lvValue;
+   }
 
-    }
-
-   public long getStartingAuditSeqNum() throws IOException {
-      if (LOG.isTraceEnabled()) LOG.trace("getStartingAuditSeqNum");
-      String controlPtString = new String(String.valueOf(currControlPt));
-      long lvAsn;
-      if (LOG.isDebugEnabled()) LOG.debug("getStartingAuditSeqNum new get for control point " + currControlPt);
-      Get g = new Get(Bytes.toBytes(controlPtString));
-      if (LOG.isDebugEnabled()) LOG.debug("getStartingAuditSeqNum setting result");
-      Result r = table.get(g);
-      if (LOG.isDebugEnabled()) LOG.debug("getStartingAuditSeqNum currValue CONTROL_POINT_FAMILY is "
-                 + CONTROL_POINT_FAMILY + " ASN_HIGH_WATER_MARK " + ASN_HIGH_WATER_MARK);
-      byte [] currValue = r.getValue(CONTROL_POINT_FAMILY, ASN_HIGH_WATER_MARK);
-      if (LOG.isDebugEnabled()) LOG.debug("Starting asn setting recordString ");
-      if (currValue == null)
-         lvAsn = 1;
-      else
-      {
-         String recordString = new String(currValue);
-         lvAsn = Long.valueOf(recordString);
-      }
+   public long getStartingAuditSeqNum(final int clusterId) throws IOException {
+      if (LOG.isTraceEnabled()) LOG.trace("getStartingAuditSeqNum for clusterId: " + clusterId);
+      long lvAsn = 1;
+
+      Get g = new Get(Bytes.toBytes(clusterId));
+      String asnToken;
+      if (LOG.isDebugEnabled()) LOG.debug("getStartingAuditSeqNum attempting table.get");
+         Result r = table.get(g);
+         if (r.isEmpty())
+            return lvAsn;
+         if (LOG.isDebugEnabled()) LOG.debug("getStartingAuditSeqNum Result: " + r);
+         String value = new String(Bytes.toString(r.getValue(CONTROL_POINT_FAMILY, CP_NUM_AND_ASN_HWM)));
+         // In theory the above value is the latestversion of the column
+         if (LOG.isDebugEnabled()) LOG.debug("getStartingAuditSeqNum for clusterId: " + clusterId + ", valueString is " + value);
+         StringTokenizer stok = new StringTokenizer(value, ",");
+         if (stok.hasMoreElements()) {
+            if (LOG.isTraceEnabled()) LOG.trace("Parsing record in getStartingAuditSeqNum");
+            stok.nextElement();  // skip the control point token
+            asnToken = stok.nextElement().toString();
+            lvAsn = Long.parseLong(asnToken, 10);
+            if (LOG.isTraceEnabled()) LOG.trace("Value for getStartingAuditSeqNum and clusterId: "
+                + clusterId + " is: " + lvAsn);
+         }
       if (LOG.isTraceEnabled()) LOG.trace("getStartingAuditSeqNum - exit returning " + lvAsn);
       return lvAsn;
     }
 
-   public long getNextAuditSeqNum(int nid) throws IOException {
-      if (LOG.isTraceEnabled()) LOG.trace("getNextAuditSeqNum for node: " + nid);
+   public long getNextAuditSeqNum(int clusterId, int nid) throws IOException {
+      if (LOG.isTraceEnabled()) LOG.trace("getNextAuditSeqNum for cluster " + clusterId + " node: " + nid);
 
       // We need to open the appropriate control point table and read the value from it
       Table remoteTable;
       String lv_tName = new String("TRAFODION._DTM_.TLOG" + String.valueOf(nid) + "_CONTROL_POINT");
-      Connection remoteConnection = ConnectionFactory.createConnection(this.config);
-      remoteTable = remoteConnection.getTable(TableName.valueOf(lv_tName));
+      remoteTable = connection.getTable(TableName.valueOf(lv_tName));
+
+      long lvAsn = 1;
 
-      long highValue = -1;
       try {
-         Scan s = new Scan();
-         s.setCaching(10);
-         s.setCacheBlocks(false);
-         if (LOG.isTraceEnabled()) LOG.trace("getNextAuditSeqNum resultScanner");
-         ResultScanner ss = remoteTable.getScanner(s);
-         try {
-            long currValue;
-            String rowKey;
-            if (LOG.isTraceEnabled()) LOG.trace("getNextAuditSeqNum entering for loop" );
-            for (Result r : ss) {
-               rowKey = new String(r.getRow());
-               if (LOG.isTraceEnabled()) LOG.trace("getNextAuditSeqNum rowKey is " + rowKey );
-               currValue =  Long.parseLong(Bytes.toString(r.value()));
-               if (LOG.isTraceEnabled()) LOG.trace("getNextAuditSeqNum value is " + currValue);
-               if (currValue > highValue) {
-                  if (LOG.isTraceEnabled()) LOG.trace("getNextAuditSeqNum Setting highValue to " + currValue);
-                  highValue = currValue;
-               }
+         Get g = new Get(Bytes.toBytes(clusterId));
+         if (LOG.isDebugEnabled()) LOG.debug("getNextAuditSeqNum attempting remoteTable.get");
+         Result r = remoteTable.get(g);
+         if (!r.isEmpty()){
+            if (LOG.isDebugEnabled()) LOG.debug("getNextAuditSeqNum Result: " + r);
+            String value = new String(Bytes.toString(r.getValue(CONTROL_POINT_FAMILY, CP_NUM_AND_ASN_HWM)));
+            // In theory the above value is the latest version of the column
+            if (LOG.isDebugEnabled()) LOG.debug("getNextAuditSeqNum for clusterId: " + clusterId + ", valueString is " + value);
+            StringTokenizer stok = new StringTokenizer(value, ",");
+            if (stok.hasMoreElements()) {
+               if (LOG.isTraceEnabled()) LOG.trace("Parsing record in getNextAuditSeqNum");
+               stok.nextElement();  // skip the control point token
+               String asnToken = stok.nextElement().toString();
+               lvAsn = Long.parseLong(asnToken, 10);
+               if (LOG.isTraceEnabled()) LOG.trace("Value for getNextAuditSeqNum and clusterId: "
+                            + clusterId + " is: " + (lvAsn + 1));
             }
-         } finally {
-            ss.close();
          }
       } finally {
-         try {
-            remoteTable.close();
-            remoteConnection.close();
-         }
-         catch (IOException e) {
-            LOG.error("getNextAuditSeqNum IOException closing table or connection for " + lv_tName);
-            e.printStackTrace();
-         }
+         remoteTable.close();
       }
-      if (LOG.isTraceEnabled()) LOG.trace("getNextAuditSeqNum returning " + (highValue + 1));
-      return (highValue + 1);
-    }
+      if (LOG.isTraceEnabled()) LOG.trace("getNextAuditSeqNum returning " + (lvAsn + 1));
+      return (lvAsn + 1);
+   }
+
 
+   public long doControlPoint(final int clusterId, final long sequenceNumber, final boolean incrementCP) throws IOException {
+      if (LOG.isTraceEnabled()) LOG.trace("doControlPoint start");
 
-   public long doControlPoint(final long sequenceNumber) throws IOException {
-      currControlPt++;
-      if (LOG.isTraceEnabled()) LOG.trace("doControlPoint interval (" + currControlPt + "), sequenceNumber (" + sequenceNumber+ ") try putRecord");
-      putRecord(currControlPt, sequenceNumber);
+         if (incrementCP) {
+           currControlPt++;
+         }
+         if (LOG.isTraceEnabled()) LOG.trace("doControlPoint interval (" + currControlPt + "), clusterId: " + clusterId + ", sequenceNumber (" + sequenceNumber+ ") try putRecord");
+         putRecord(clusterId, currControlPt, sequenceNumber);
+      if (LOG.isTraceEnabled()) LOG.trace("doControlPoint - exit");
       return currControlPt;
    }
 
+   public long bumpControlPoint(final int clusterId, final int count) throws IOException {
+      if (LOG.isTraceEnabled()) LOG.trace("bumpControlPoint start, count: " + count);
+      long currASN = -1;
+         currControlPt = getCurrControlPt(clusterId);
+         currASN = getStartingAuditSeqNum(clusterId);
+         for ( int i = 0; i < count; i++ ) {
+            currControlPt++;
+            if (LOG.isTraceEnabled()) LOG.trace("bumpControlPoint putting new record " + (i + 1) + " for control point ("
+                 + currControlPt + "), clusterId: " + clusterId + ", ASN (" + currASN + ")");
+            putRecord(clusterId, currControlPt, currASN);
+         }
+      if (LOG.isTraceEnabled()) LOG.trace("bumpControlPoint - exit");
+      return currASN;
+   }
+
    public boolean deleteRecord(final long controlPoint) throws IOException {
       if (LOG.isTraceEnabled()) LOG.trace("deleteRecord start for control point " + controlPoint);
       String controlPtString = new String(String.valueOf(controlPoint));
@@ -317,33 +385,80 @@ public class HBaseAuditControlPoint {
       return true;
    }
 
-   public boolean deleteAgedRecords(final long controlPoint) throws IOException {
-      if (LOG.isTraceEnabled()) LOG.trace("deleteAgedRecords start - control point " + controlPoint);
+   public boolean deleteAgedRecords(final int clusterId, final long controlPoint) throws IOException {
+      if (LOG.isTraceEnabled()) LOG.trace("deleteAgedRecords start - clusterId " + clusterId + " control point " + controlPoint);
       String controlPtString = new String(String.valueOf(controlPoint));
 
-      Scan s = new Scan();
-      s.setCaching(10);
-      s.setCacheBlocks(false);
       ArrayList<Delete> deleteList = new ArrayList<Delete>();
-      ResultScanner ss = table.getScanner(s);
-      try {
-         String rowKey;
-         for (Result r : ss) {
-            rowKey = new String(r.getRow());
-            if (Long.parseLong(rowKey) < controlPoint) {
-               if (LOG.isDebugEnabled()) LOG.debug("Adding  (" + rowKey + ") to delete list");
-               Delete del = new Delete(rowKey.getBytes());
-               deleteList.add(del);
+      Get g = new Get(Bytes.toBytes(clusterId));
+      g.setMaxVersions(versions);  // will return last n versions of row
+      g.addColumn(CONTROL_POINT_FAMILY, CP_NUM_AND_ASN_HWM);
+      String ctrlPtToken;
+         Result r = table.get(g);
+         if (r.isEmpty())
+            return false;
+         List<Cell> list = r.getColumnCells(CONTROL_POINT_FAMILY, CP_NUM_AND_ASN_HWM);  // returns all versions of this column
+         for (Cell cell : list) {
+            StringTokenizer stok = 
+                    new StringTokenizer(Bytes.toString(CellUtil.cloneValue(cell)), ",");
+            if (stok.hasMoreElements()) {
+               ctrlPtToken = stok.nextElement().toString();
+               if (LOG.isTraceEnabled()) LOG.trace("Parsing record for controlPoint (" + ctrlPtToken + ")");
+               if (Long.parseLong(ctrlPtToken, 10) <= controlPoint){
+                  // This is one we are looking for
+                  Delete del = new Delete(cell.getRowArray(), cell.getRowOffset(), cell.getRowLength(), cell.getTimestamp());
+                  deleteList.add(del);
+                  if (LOG.isTraceEnabled()) LOG.trace("Deleting entry for ctrlPtToken: " + ctrlPtToken);
+               }
+            }
+            else {
+               if (LOG.isTraceEnabled()) LOG.trace("No tokens to parse for controlPoint (" + controlPoint + ")");
             }
          }
          if (LOG.isDebugEnabled()) LOG.debug("attempting to delete list with " + deleteList.size() + " elements");
          table.delete(deleteList);
-      } finally {
-         ss.close();
-      }
-
       if (LOG.isTraceEnabled()) LOG.trace("deleteAgedRecords - exit");
       return true;
    }
+   
+   public String getTableName(){
+      return CONTROL_POINT_TABLE_NAME;
+   }
+   
+   public long getNthRecord(int clusterId, int n) throws IOException{
+      if (LOG.isTraceEnabled()) LOG.trace("getNthRecord start - clusterId " + clusterId + " n: " + n);
+
+      Get g = new Get(Bytes.toBytes(clusterId));
+      g.setMaxVersions(n + 1);  // will return last n+1 versions of row just in case
+      g.addColumn(CONTROL_POINT_FAMILY, CP_NUM_AND_ASN_HWM);
+      String ctrlPtToken;
+      long lvReturn = 1;
+         Result r = table.get(g);
+         if (r.isEmpty())
+            return lvReturn; 
+         List<Cell> list = r.getColumnCells(CONTROL_POINT_FAMILY, CP_NUM_AND_ASN_HWM);  // returns all versions of this column
+         int i = 0;
+         for (Cell cell : list) {
+            i++;
+            StringTokenizer stok = 
+                    new StringTokenizer(Bytes.toString(CellUtil.cloneValue(cell)), ",");
+            if (stok.hasMoreElements()) {
+               ctrlPtToken = stok.nextElement().toString();
+               if (LOG.isTraceEnabled()) LOG.trace("Parsing record for controlPoint (" + ctrlPtToken + ")");
+               if ( i < n ){
+                  if (LOG.isTraceEnabled()) LOG.trace("Skipping record " + i + " of " + n + " for controlPoint" );
+                  continue;
+               }
+               lvReturn = Long.parseLong(ctrlPtToken);;
+               if (LOG.isTraceEnabled()) LOG.trace("getNthRecord exit - returning " + lvReturn);
+               return lvReturn;
+            }
+            else {
+               if (LOG.isTraceEnabled()) LOG.trace("No tokens to parse for " + i);
+            }
+         }
+      if (LOG.isTraceEnabled()) LOG.trace("getNthRecord - exit returning 1");
+      return 1;
+   }
 }
 


[22/22] incubator-trafodion git commit: Merge remote branch 'origin/pr/1075/head' into merge_1075

Posted by sa...@apache.org.
Merge remote branch 'origin/pr/1075/head' into merge_1075


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

Branch: refs/heads/master
Commit: 815d03329f8f0c29e5deb45453597d7af1cd0eda
Parents: c25694d 74b2063
Author: Sandhya Sundaresan <sa...@apache.org>
Authored: Fri May 12 20:25:11 2017 +0000
Committer: Sandhya Sundaresan <sa...@apache.org>
Committed: Fri May 12 20:25:11 2017 +0000

----------------------------------------------------------------------
 core/sqf/src/seatrans/.gitignore                |    1 +
 .../NonPendingTransactionException.java         |   47 +
 .../hbase/client/transactional/RMInterface.java |  296 +-
 .../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              |   30 +-
 .../transactional/TransactionalTable.java       |  162 +-
 .../transactional/TrxRegionEndpoint.java.tmpl   | 3439 +++++----
 .../generated/SsccRegionProtos.java             |   42 +-
 .../generated/TrxRegionProtos.java              | 6849 ++++++++++--------
 .../transactional/SsccTransactionState.java     |    4 +-
 .../transactional/TransactionState.java         |  108 +-
 .../TransactionalRegionScannerHolder.java       |   63 -
 .../TransactionalRegionScannerHolder.java.tmpl  |   79 +
 .../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 +++--
 core/sql/regress/seabase/EXPECTED022            | 1020 +--
 core/sql/regress/seabase/TEST022                |   10 +-
 .../java/org/trafodion/sql/HBaseClient.java     |    3 +-
 .../java/org/trafodion/sql/HTableClient.java    |   47 +-
 27 files changed, 9487 insertions(+), 6478 deletions(-)
----------------------------------------------------------------------