diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/Action.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/Action.java index 350e18a454..6db6da0a3d 100644 --- a/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/Action.java +++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/Action.java @@ -21,9 +21,11 @@ package org.apache.hadoop.hbase.chaos.actions; import java.io.IOException; import java.util.ArrayList; import java.util.Collection; +import java.util.HashSet; import java.util.LinkedList; import java.util.List; import java.util.Map; +import java.util.Set; import java.util.function.BiConsumer; import java.util.function.Consumer; @@ -139,16 +141,13 @@ public class Action { return new ServerName [] {}; } ServerName master = clusterStatus.getMasterName(); - if (master == null || !regionServers.contains(master)) { - return regionServers.toArray(new ServerName[count]); - } - if (count == 1) { - return new ServerName [] {}; - } + Set masters = new HashSet(); + masters.add(master); + masters.addAll(clusterStatus.getBackupMasterNames()); ArrayList tmp = new ArrayList<>(count); tmp.addAll(regionServers); - tmp.remove(master); - return tmp.toArray(new ServerName[count-1]); + tmp.removeAll(masters); + return tmp.toArray(new ServerName[tmp.size()]); } protected void killMaster(ServerName server) throws IOException { diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/test/IntegrationTestBigLinkedList.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/test/IntegrationTestBigLinkedList.java index 35bc7a1a42..a670e40159 100644 --- a/hbase-it/src/test/java/org/apache/hadoop/hbase/test/IntegrationTestBigLinkedList.java +++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/test/IntegrationTestBigLinkedList.java @@ -18,6 +18,7 @@ package org.apache.hadoop.hbase.test; +import org.apache.hbase.thirdparty.com.google.common.base.Preconditions; import java.io.DataInput; import java.io.DataOutput; import java.io.FileNotFoundException; @@ -58,6 +59,7 @@ 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.Connection; +import org.apache.hadoop.hbase.client.ConnectionConfiguration; import org.apache.hadoop.hbase.client.ConnectionFactory; import org.apache.hadoop.hbase.client.Get; import org.apache.hadoop.hbase.client.Mutation; @@ -267,6 +269,15 @@ public class IntegrationTestBigLinkedList extends IntegrationTestBase { public static final String MULTIPLE_UNEVEN_COLUMNFAMILIES_KEY = "generator.multiple.columnfamilies"; + /** + * Set this configuration if you want to test huge data. + *

+ * $ ./bin/hbase org.apache.hadoop.hbase.test.IntegrationTestBigLinkedList + * -Dgenerator.big.family.value.size=1024 generator 1 10 output + */ + public static final String BIG_FAMILY_VALUE_SIZE_KEY = "generator.big.family.value.size"; + + public static enum Counts { SUCCESS, TERMINATING, UNDEFINED, IOEXCEPTION } @@ -300,7 +311,7 @@ public class IntegrationTestBigLinkedList extends IntegrationTestBase { static class GeneratorRecordReader extends RecordReader { private long count; private long numNodes; - private Random rand; + private long seed; @Override public void close() throws IOException { @@ -308,9 +319,7 @@ public class IntegrationTestBigLinkedList extends IntegrationTestBase { @Override public BytesWritable getCurrentKey() throws IOException, InterruptedException { - byte[] bytes = new byte[ROWKEY_LENGTH]; - rand.nextBytes(bytes); - return new BytesWritable(bytes); + return new BytesWritable(nextRow(ROWKEY_LENGTH)); } @Override @@ -327,8 +336,22 @@ public class IntegrationTestBigLinkedList extends IntegrationTestBase { public void initialize(InputSplit arg0, TaskAttemptContext context) throws IOException, InterruptedException { numNodes = context.getConfiguration().getLong(GENERATOR_NUM_ROWS_PER_MAP_KEY, 25000000); - // Use SecureRandom to avoid issue described in HBASE-13382. - rand = new SecureRandom(); + seed = (new Random()).nextLong(); + } + + private byte[] nextRow(final int len) { + byte[] row = new byte[len]; + + for (int i = 0; i < len;) { + // 64-bit LCG, see HBASE-21256 + seed = seed * 6364136223846793005L + 1442695040888963407L; + for (int j = 0; j < 64 && i < len; j+=8, i++) { + // We regard seed as unsigned long, therefore used '>>>' instead of '>>'. + row[i] = (byte) ((seed >>> j) & 0xffL); + } + } + + return row; } @Override @@ -437,6 +460,34 @@ public class IntegrationTestBigLinkedList extends IntegrationTestBase { this.numWalkers = context.getConfiguration().getInt(CONCURRENT_WALKER_KEY, CONCURRENT_WALKER_DEFAULT); this.walkersStop = false; this.conf = context.getConfiguration(); + + if (multipleUnevenColumnFamilies) { + int n = context.getConfiguration().getInt(BIG_FAMILY_VALUE_SIZE_KEY, 256); + int limit = context.getConfiguration().getInt( + ConnectionConfiguration.MAX_KEYVALUE_SIZE_KEY, + ConnectionConfiguration.MAX_KEYVALUE_SIZE_DEFAULT); + + Preconditions.checkArgument( + n <= limit, + BIG_FAMILY_VALUE_SIZE_KEY+ " > " + ConnectionConfiguration.MAX_KEYVALUE_SIZE_KEY); + + bigValue = new byte[n]; + ThreadLocalRandom.current().nextBytes(bigValue); + LOG.info("Create a bigValue with " + n + " bytes."); + } + + Preconditions.checkArgument( + numNodes > 0, + "numNodes=%s <= 0", + numNodes); + Preconditions.checkArgument( + numNodes % width == 0, + "numNodes=%s mod width=%s != 0", + numNodes, width); + Preconditions.checkArgument( + numNodes % wrap == 0, + "numNodes=%s mod wrap=%s != 0" + ); } protected void instantiateHTable() throws IOException { @@ -457,7 +508,7 @@ public class IntegrationTestBigLinkedList extends IntegrationTestBase { current[i] = new byte[key.getLength()]; System.arraycopy(key.getBytes(), 0, current[i], 0, key.getLength()); if (++i == current.length) { - LOG.info("Persisting current.length=" + current.length + ", count=" + count + ", id=" + + LOG.debug("Persisting current.length=" + current.length + ", count=" + count + ", id=" + Bytes.toStringBinary(id) + ", current=" + Bytes.toStringBinary(current[0]) + ", i=" + i); persist(output, count, prev, current, id); @@ -526,11 +577,6 @@ public class IntegrationTestBigLinkedList extends IntegrationTestBase { if (this.multipleUnevenColumnFamilies) { // Use any column name. put.addColumn(TINY_FAMILY_NAME, TINY_FAMILY_NAME, this.tinyValue); - // If we've not allocated bigValue, do it now. Reuse same value each time. - if (this.bigValue == null) { - this.bigValue = new byte[current[i].length * 10]; - ThreadLocalRandom.current().nextBytes(this.bigValue); - } // Use any column name. put.addColumn(BIG_FAMILY_NAME, BIG_FAMILY_NAME, this.bigValue); } @@ -1155,13 +1201,14 @@ public class IntegrationTestBigLinkedList extends IntegrationTestBase { // TODO check for more than one def, should not happen StringBuilder refsSb = null; - String keyString = Bytes.toStringBinary(key.getBytes(), 0, key.getLength()); if (defCount == 0 || refs.size() != 1) { + String keyString = Bytes.toStringBinary(key.getBytes(), 0, key.getLength()); refsSb = dumpExtraInfoOnRefs(key, context, refs); LOG.error("LinkedListError: key=" + keyString + ", reference(s)=" + (refsSb != null? refsSb.toString(): "")); } if (lostFamilies) { + String keyString = Bytes.toStringBinary(key.getBytes(), 0, key.getLength()); LOG.error("LinkedListError: key=" + keyString + ", lost big or tiny families"); context.getCounter(Counts.LOST_FAMILIES).increment(1); context.write(key, LOSTFAM); @@ -1188,6 +1235,7 @@ public class IntegrationTestBigLinkedList extends IntegrationTestBase { // was added which can help a little debugging. This info is only available in mapper // output -- the 'Linked List error Key...' log message above. What we emit here is // useless for debugging. + String keyString = Bytes.toStringBinary(key.getBytes(), 0, key.getLength()); context.getCounter("undef", keyString).increment(1); } } else if (defCount > 0 && refs.isEmpty()) { @@ -1195,6 +1243,7 @@ public class IntegrationTestBigLinkedList extends IntegrationTestBase { context.write(key, UNREF); context.getCounter(Counts.UNREFERENCED).increment(1); if (rows.addAndGet(1) < MISSING_ROWS_TO_LOG) { + String keyString = Bytes.toStringBinary(key.getBytes(), 0, key.getLength()); context.getCounter("unref", keyString).increment(1); } } else {