HBASE-12548. Improve debuggability of IntegrationTestTimeBoundedRequestsWithRegionReplicas

This commit is contained in:
Devaraj Das 2014-12-06 18:18:03 -08:00
parent b7d1790411
commit ca0c1774c9
8 changed files with 163 additions and 5 deletions

View File

@ -854,4 +854,33 @@ public final class CellUtil {
}
return commonPrefix;
}
/** Returns a string representation of the cell */
public static String toString(Cell cell, boolean verbose) {
if (cell == null) {
return "";
}
StringBuilder builder = new StringBuilder();
String keyStr = getCellKeyAsString(cell);
String tag = null;
String value = null;
if (verbose) {
// TODO: pretty print tags as well
tag = Bytes.toStringBinary(cell.getTagsArray(), cell.getTagsOffset(), cell.getTagsLength());
value = Bytes.toStringBinary(cell.getValueArray(), cell.getValueOffset(),
cell.getValueLength());
}
builder
.append(keyStr);
if (tag != null && !tag.isEmpty()) {
builder.append("/").append(tag);
}
if (value != null) {
builder.append("/").append(value);
}
return builder.toString();
}
}

View File

@ -18,6 +18,8 @@ package org.apache.hadoop.hbase.util.test;
import java.util.Random;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.MD5Hash;
@ -32,6 +34,9 @@ import org.apache.hadoop.hbase.util.MD5Hash;
@InterfaceAudience.Private
public class LoadTestKVGenerator {
private static final Log LOG = LogFactory.getLog(LoadTestKVGenerator.class);
private static int logLimit = 10;
/** A random number generator for determining value size */
private Random randomForValueSize = new Random();
@ -56,7 +61,13 @@ public class LoadTestKVGenerator {
*/
public static boolean verify(byte[] value, byte[]... seedStrings) {
byte[] expectedData = getValueForRowColumn(value.length, seedStrings);
return Bytes.equals(expectedData, value);
boolean equals = Bytes.equals(expectedData, value);
if (!equals && LOG.isDebugEnabled() && logLimit > 0) {
LOG.debug("verify failed, expected value: " + Bytes.toStringBinary(expectedData)
+ " actual value: "+ Bytes.toStringBinary(value));
logLimit--; // this is not thread safe, but at worst we will have more logging
}
return equals;
}
/**

View File

@ -397,4 +397,38 @@ public class TestCellUtil {
assertEquals(kv.toString(), cellToString);
}
@Test
public void testToString1() {
String row = "test.row";
String family = "test.family";
String qualifier = "test.qualifier";
long timestamp = 42;
Type type = Type.Put;
String value = "test.value";
long seqId = 1042;
Cell cell = CellUtil.createCell(Bytes.toBytes(row), Bytes.toBytes(family),
Bytes.toBytes(qualifier), timestamp, type.getCode(), Bytes.toBytes(value), seqId);
String nonVerbose = CellUtil.toString(cell, false);
String verbose = CellUtil.toString(cell, true);
System.out.println("nonVerbose=" + nonVerbose);
System.out.println("verbose=" + verbose);
Assert.assertEquals(
String.format("%s/%s:%s/%d/%s/vlen=%s/seqid=%s",
row, family, qualifier, timestamp, type.toString(),
Bytes.toBytes(value).length, seqId),
nonVerbose);
Assert.assertEquals(
String.format("%s/%s:%s/%d/%s/vlen=%s/seqid=%s/%s",
row, family, qualifier, timestamp, type.toString(), Bytes.toBytes(value).length,
seqId, value),
verbose);
// TODO: test with tags
}
}

View File

@ -29,6 +29,7 @@ import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.testclassification.IntegrationTests;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.LoadTestTool;
import org.apache.hadoop.hbase.util.Threads;
import org.apache.hadoop.util.ToolRunner;
import org.junit.Assert;
import org.junit.Test;
@ -170,7 +171,13 @@ public class IntegrationTestIngest extends IntegrationTestBase {
, startKey, numKeys));
if (0 != ret) {
String errorMsg = "Verification failed with error code " + ret;
LOG.error(errorMsg);
LOG.error(errorMsg + " Rerunning verification after 1 minute for debugging");
Threads.sleep(1000 * 60);
ret = loadTool.run(getArgsForLoadTestTool("-read", String.format("100:%d", readThreads)
, startKey, numKeys));
if (0 != ret) {
LOG.error("Rerun of Verification failed with error code " + ret);
}
Assert.fail(errorMsg);
}
startKey += numKeys;

View File

@ -199,6 +199,9 @@ public class IntegrationTestBulkLoad extends IntegrationTestBase {
HTableDescriptor desc = admin.getTableDescriptor(t);
desc.addCoprocessor(SlowMeCoproScanOperations.class.getName());
HBaseTestingUtility.modifyTableSync(admin, desc);
//sleep for sometime. Hope is that the regions are closed/opened before
//the sleep returns. TODO: do this better
Thread.sleep(30000);
}
@Test

View File

@ -19,7 +19,9 @@
package org.apache.hadoop.hbase.test;
import java.io.IOException;
import java.util.HashSet;
import java.util.List;
import java.util.Set;
import java.util.concurrent.Executors;
import java.util.concurrent.ScheduledExecutorService;
import java.util.concurrent.ScheduledFuture;
@ -31,12 +33,15 @@ import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.HBaseConfiguration;
import org.apache.hadoop.hbase.HRegionLocation;
import org.apache.hadoop.hbase.IntegrationTestIngest;
import org.apache.hadoop.hbase.IntegrationTestingUtility;
import org.apache.hadoop.hbase.RegionLocations;
import org.apache.hadoop.hbase.testclassification.IntegrationTests;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.chaos.factories.MonkeyFactory;
import org.apache.hadoop.hbase.client.Admin;
import org.apache.hadoop.hbase.client.ClusterConnection;
import org.apache.hadoop.hbase.client.Consistency;
import org.apache.hadoop.hbase.client.Get;
import org.apache.hadoop.hbase.client.Result;
@ -163,7 +168,7 @@ public class IntegrationTestTimeBoundedRequestsWithRegionReplicas extends Integr
long refreshTime = conf.getLong(StorefileRefresherChore.REGIONSERVER_STOREFILE_REFRESH_PERIOD, 0);
if (refreshTime > 0 && refreshTime <= 10000) {
LOG.info("Sleeping " + refreshTime + "ms to ensure that the data is replicated");
Threads.sleep(refreshTime);
Threads.sleep(refreshTime*3);
} else {
LOG.info("Reopening the table");
admin.disableTable(getTablename());
@ -337,6 +342,15 @@ public class IntegrationTestTimeBoundedRequestsWithRegionReplicas extends Integr
if (elapsedNano > timeoutNano) {
timedOutReads.incrementAndGet();
numReadFailures.addAndGet(1); // fail the test
for (Result r : results) {
LOG.error("FAILED FOR " + r);
RegionLocations rl = ((ClusterConnection)connection).
locateRegion(tableName, r.getRow(), true, true);
HRegionLocation locations[] = rl.getRegionLocations();
for (HRegionLocation h : locations) {
LOG.error("LOCATION " + h);
}
}
}
}
}

View File

@ -394,7 +394,11 @@ public class RegionStates {
regionsInTransition.remove(encodedName);
ServerName oldServerName = regionAssignments.put(hri, serverName);
if (!serverName.equals(oldServerName)) {
LOG.info("Onlined " + hri.getShortNameToLog() + " on " + serverName);
if (LOG.isDebugEnabled()) {
LOG.debug("Onlined " + hri.getShortNameToLog() + " on " + serverName + " " + hri);
} else {
LOG.debug("Onlined " + hri.getShortNameToLog() + " on " + serverName);
}
addToServerHoldings(serverName, hri);
addToReplicaMapping(hri);
if (oldServerName == null) {

View File

@ -32,8 +32,13 @@ import java.util.concurrent.atomic.AtomicLong;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.CellUtil;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HRegionLocation;
import org.apache.hadoop.hbase.RegionLocations;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.client.ClusterConnection;
import org.apache.hadoop.hbase.client.HConnection;
import org.apache.hadoop.hbase.client.HConnectionManager;
import org.apache.hadoop.hbase.client.Result;
@ -322,10 +327,10 @@ public abstract class MultiThreadedAction {
public boolean verifyResultAgainstDataGenerator(Result result, boolean verifyValues,
boolean verifyCfAndColumnIntegrity) {
String rowKeyStr = Bytes.toString(result.getRow());
// See if we have any data at all.
if (result.isEmpty()) {
LOG.error("Error checking data for key [" + rowKeyStr + "], no data returned");
printLocations(result);
return false;
}
@ -338,6 +343,7 @@ public abstract class MultiThreadedAction {
if (verifyCfAndColumnIntegrity && (expectedCfs.length != result.getMap().size())) {
LOG.error("Error checking data for key [" + rowKeyStr
+ "], bad family count: " + result.getMap().size());
printLocations(result);
return false;
}
@ -348,6 +354,7 @@ public abstract class MultiThreadedAction {
if (columnValues == null) {
LOG.error("Error checking data for key [" + rowKeyStr
+ "], no data for family [" + cfStr + "]]");
printLocations(result);
return false;
}
@ -356,6 +363,7 @@ public abstract class MultiThreadedAction {
if (!columnValues.containsKey(MUTATE_INFO)) {
LOG.error("Error checking data for key [" + rowKeyStr + "], column family ["
+ cfStr + "], column [" + Bytes.toString(MUTATE_INFO) + "]; value is not found");
printLocations(result);
return false;
}
@ -372,6 +380,7 @@ public abstract class MultiThreadedAction {
if (columnValues.containsKey(column)) {
LOG.error("Error checking data for key [" + rowKeyStr + "], column family ["
+ cfStr + "], column [" + mutate.getKey() + "]; should be deleted");
printLocations(result);
return false;
}
byte[] hashCodeBytes = Bytes.toBytes(hashCode);
@ -384,6 +393,7 @@ public abstract class MultiThreadedAction {
if (!columnValues.containsKey(INCREMENT)) {
LOG.error("Error checking data for key [" + rowKeyStr + "], column family ["
+ cfStr + "], column [" + Bytes.toString(INCREMENT) + "]; value is not found");
printLocations(result);
return false;
}
long currentValue = Bytes.toLong(columnValues.remove(INCREMENT));
@ -394,6 +404,7 @@ public abstract class MultiThreadedAction {
if (extra != 0 && (amount == 0 || extra % amount != 0)) {
LOG.error("Error checking data for key [" + rowKeyStr + "], column family ["
+ cfStr + "], column [increment], extra [" + extra + "], amount [" + amount + "]");
printLocations(result);
return false;
}
if (amount != 0 && extra != amount) {
@ -414,6 +425,7 @@ public abstract class MultiThreadedAction {
}
LOG.error("Error checking data for key [" + rowKeyStr
+ "], bad columns for family [" + cfStr + "]: " + colsStr);
printLocations(result);
return false;
}
// See if values check out.
@ -461,6 +473,7 @@ public abstract class MultiThreadedAction {
+ column + "]; mutation [" + mutation + "], hashCode ["
+ hashCode + "], verificationNeeded ["
+ verificationNeeded + "]");
printLocations(result);
return false;
}
} // end of mutation checking
@ -469,6 +482,7 @@ public abstract class MultiThreadedAction {
LOG.error("Error checking data for key [" + rowKeyStr + "], column family ["
+ cfStr + "], column [" + column + "], mutation [" + mutation
+ "]; value of length " + bytes.length);
printLocations(result);
return false;
}
}
@ -478,6 +492,48 @@ public abstract class MultiThreadedAction {
return true;
}
private void printLocations(Result r) {
RegionLocations rl = null;
if (r == null) {
LOG.info("FAILED FOR null Result");
return;
}
LOG.info("FAILED FOR " + resultToString(r) + " Stale " + r.isStale());
if (r.getRow() == null) {
return;
}
try {
rl = ((ClusterConnection)connection).locateRegion(tableName, r.getRow(), true, true);
} catch (IOException e) {
LOG.warn("Couldn't get locations for row " + Bytes.toString(r.getRow()));
}
HRegionLocation locations[] = rl.getRegionLocations();
for (HRegionLocation h : locations) {
LOG.info("LOCATION " + h);
}
}
private String resultToString(Result result) {
StringBuilder sb = new StringBuilder();
sb.append("cells=");
if(result.isEmpty()) {
sb.append("NONE");
return sb.toString();
}
sb.append("{");
boolean moreThanOne = false;
for(Cell cell : result.listCells()) {
if(moreThanOne) {
sb.append(", ");
} else {
moreThanOne = true;
}
sb.append(CellUtil.toString(cell, true));
}
sb.append("}");
return sb.toString();
}
// Parse mutate info into a map of <column name> => <update action>
private Map<String, MutationType> parseMutateInfo(byte[] mutateInfo) {
Map<String, MutationType> mi = new HashMap<String, MutationType>();