You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by jx...@apache.org on 2013/10/14 22:05:05 UTC
svn commit: r1532048 -
/hbase/trunk/hbase-it/src/test/java/org/apache/hadoop/hbase/test/IntegrationTestBigLinkedList.java
Author: jxiang
Date: Mon Oct 14 20:05:05 2013
New Revision: 1532048
URL: http://svn.apache.org/r1532048
Log:
HBASE-9758 Log missing rows and their regions if ITBLL fails
Modified:
hbase/trunk/hbase-it/src/test/java/org/apache/hadoop/hbase/test/IntegrationTestBigLinkedList.java
Modified: hbase/trunk/hbase-it/src/test/java/org/apache/hadoop/hbase/test/IntegrationTestBigLinkedList.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-it/src/test/java/org/apache/hadoop/hbase/test/IntegrationTestBigLinkedList.java?rev=1532048&r1=1532047&r2=1532048&view=diff
==============================================================================
--- hbase/trunk/hbase-it/src/test/java/org/apache/hadoop/hbase/test/IntegrationTestBigLinkedList.java (original)
+++ hbase/trunk/hbase-it/src/test/java/org/apache/hadoop/hbase/test/IntegrationTestBigLinkedList.java Mon Oct 14 20:05:05 2013
@@ -23,10 +23,12 @@ import java.io.DataOutput;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Arrays;
+import java.util.Iterator;
import java.util.List;
import java.util.Random;
import java.util.Set;
import java.util.UUID;
+import java.util.concurrent.atomic.AtomicInteger;
import org.apache.commons.cli.CommandLine;
import org.apache.commons.cli.GnuParser;
@@ -40,6 +42,7 @@ import org.apache.hadoop.conf.Configured
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.HBaseConfiguration;
import org.apache.hadoop.hbase.HColumnDescriptor;
+import org.apache.hadoop.hbase.HRegionLocation;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.IntegrationTestBase;
import org.apache.hadoop.hbase.IntegrationTestingUtility;
@@ -48,6 +51,8 @@ import org.apache.hadoop.hbase.TableName
import org.apache.hadoop.hbase.chaos.monkies.CalmChaosMonkey;
import org.apache.hadoop.hbase.client.Get;
import org.apache.hadoop.hbase.client.HBaseAdmin;
+import org.apache.hadoop.hbase.client.HConnection;
+import org.apache.hadoop.hbase.client.HConnectionManager;
import org.apache.hadoop.hbase.client.HTable;
import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.client.Result;
@@ -66,6 +71,7 @@ import org.apache.hadoop.io.NullWritable
import org.apache.hadoop.io.Text;
import org.apache.hadoop.io.Writable;
import org.apache.hadoop.mapreduce.Counter;
+import org.apache.hadoop.mapreduce.CounterGroup;
import org.apache.hadoop.mapreduce.Counters;
import org.apache.hadoop.mapreduce.InputFormat;
import org.apache.hadoop.mapreduce.InputSplit;
@@ -187,6 +193,8 @@ public class IntegrationTestBigLinkedLis
protected int NUM_SLAVES_BASE = 3; // number of slaves for the cluster
+ private static final int MISSING_ROWS_TO_LOG = 50;
+
private static final int WIDTH_DEFAULT = 1000000;
private static final int WRAP_DEFAULT = 25;
private static final int ROWKEY_LENGTH = 16;
@@ -552,6 +560,8 @@ public class IntegrationTestBigLinkedLis
public static class VerifyReducer extends Reducer<BytesWritable,BytesWritable,Text,Text> {
private ArrayList<byte[]> refs = new ArrayList<byte[]>();
+ private AtomicInteger rows = new AtomicInteger(0);
+
@Override
public void reduce(BytesWritable key, Iterable<BytesWritable> values, Context context)
throws IOException, InterruptedException {
@@ -591,10 +601,16 @@ public class IntegrationTestBigLinkedLis
// lost, emit some info about this node for debugging purposes.
context.write(new Text(keyString), new Text(refsSb.toString()));
context.getCounter(Counts.UNDEFINED).increment(1);
+ if (rows.addAndGet(1) < MISSING_ROWS_TO_LOG) {
+ context.getCounter("undef", keyString).increment(1);
+ }
} else if (defCount > 0 && refs.size() == 0) {
// node is defined but not referenced
context.write(new Text(keyString), new Text("none"));
context.getCounter(Counts.UNREFERENCED).increment(1);
+ if (rows.addAndGet(1) < MISSING_ROWS_TO_LOG) {
+ context.getCounter("unref", keyString).increment(1);
+ }
} else {
if (refs.size() > 1) {
if (refsSb != null) {
@@ -658,6 +674,7 @@ public class IntegrationTestBigLinkedLis
return success ? 0 : 1;
}
+ @SuppressWarnings("deprecation")
public boolean verify(long expectedReferenced) throws Exception {
if (job == null) {
throw new IllegalStateException("You should call run() first");
@@ -690,6 +707,27 @@ public class IntegrationTestBigLinkedLis
success = false;
}
+ if (!success) {
+ Configuration conf = job.getConfiguration();
+ HConnection conn = HConnectionManager.getConnection(conf);
+ TableName tableName = getTableName(conf);
+ CounterGroup g = counters.getGroup("undef");
+ Iterator<Counter> it = g.iterator();
+ while (it.hasNext()) {
+ String keyString = it.next().getName();
+ byte[] key = Bytes.toBytes(keyString);
+ HRegionLocation loc = conn.relocateRegion(tableName, key);
+ LOG.error("undefined row " + keyString + " region " + loc);
+ }
+ g = counters.getGroup("unref");
+ it = g.iterator();
+ while (it.hasNext()) {
+ String keyString = it.next().getName();
+ byte[] key = Bytes.toBytes(keyString);
+ HRegionLocation loc = conn.relocateRegion(tableName, key);
+ LOG.error("unreferred row " + keyString + " region " + loc);
+ }
+ }
return success;
}
}