HBASE-21731 Do not need to use ClusterConnection in IntegrationTestBigLinkedListWithVisibility
Signed-off-by: Peter Somogyi <psomogyi@apache.org>
This commit is contained in:
parent
c49cdc3e61
commit
0363f7890a
|
@ -22,7 +22,6 @@ import java.security.PrivilegedExceptionAction;
|
|||
import java.util.Arrays;
|
||||
import java.util.Iterator;
|
||||
import java.util.UUID;
|
||||
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.conf.Configured;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
|
@ -38,7 +37,6 @@ import org.apache.hadoop.hbase.chaos.factories.MonkeyFactory;
|
|||
import org.apache.hadoop.hbase.client.Admin;
|
||||
import org.apache.hadoop.hbase.client.BufferedMutator;
|
||||
import org.apache.hadoop.hbase.client.BufferedMutatorParams;
|
||||
import org.apache.hadoop.hbase.client.ClusterConnection;
|
||||
import org.apache.hadoop.hbase.client.Connection;
|
||||
import org.apache.hadoop.hbase.client.ConnectionFactory;
|
||||
import org.apache.hadoop.hbase.client.Delete;
|
||||
|
@ -71,6 +69,7 @@ import org.junit.Test;
|
|||
import org.junit.experimental.categories.Category;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
import org.apache.hbase.thirdparty.org.apache.commons.cli.CommandLine;
|
||||
|
||||
/**
|
||||
|
@ -450,24 +449,24 @@ public class IntegrationTestBigLinkedListWithVisibility extends IntegrationTestB
|
|||
|
||||
@Override
|
||||
protected void handleFailure(Counters counters) throws IOException {
|
||||
Configuration conf = job.getConfiguration();
|
||||
ClusterConnection conn = (ClusterConnection) ConnectionFactory.createConnection(conf);
|
||||
TableName tableName = TableName.valueOf(COMMON_TABLE_NAME);
|
||||
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 + ", " + 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 + ", " + loc);
|
||||
try (Connection conn = ConnectionFactory.createConnection(job.getConfiguration())) {
|
||||
TableName tableName = TableName.valueOf(COMMON_TABLE_NAME);
|
||||
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.getRegionLocator(tableName).getRegionLocation(key, true);
|
||||
LOG.error("undefined row " + keyString + ", " + loc);
|
||||
}
|
||||
g = counters.getGroup("unref");
|
||||
it = g.iterator();
|
||||
while (it.hasNext()) {
|
||||
String keyString = it.next().getName();
|
||||
byte[] key = Bytes.toBytes(keyString);
|
||||
HRegionLocation loc = conn.getRegionLocator(tableName).getRegionLocation(key, true);
|
||||
LOG.error("unreferred row " + keyString + ", " + loc);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
Loading…
Reference in New Issue