1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17 package org.apache.hadoop.hbase.util;
18
19 import static org.apache.hadoop.hbase.util.test.LoadTestDataGenerator.INCREMENT;
20 import static org.apache.hadoop.hbase.util.test.LoadTestDataGenerator.MUTATE_INFO;
21
22 import java.io.IOException;
23 import java.util.Arrays;
24 import java.util.Collection;
25 import java.util.HashMap;
26 import java.util.Map;
27 import java.util.Random;
28 import java.util.Set;
29 import java.util.concurrent.atomic.AtomicInteger;
30 import java.util.concurrent.atomic.AtomicLong;
31
32 import org.apache.commons.logging.Log;
33 import org.apache.commons.logging.LogFactory;
34 import org.apache.hadoop.conf.Configuration;
35 import org.apache.hadoop.hbase.HBaseTestingUtility;
36 import org.apache.hadoop.hbase.TableName;
37 import org.apache.hadoop.hbase.client.HConnection;
38 import org.apache.hadoop.hbase.client.HConnectionManager;
39 import org.apache.hadoop.hbase.client.Result;
40 import org.apache.hadoop.hbase.io.compress.Compression.Algorithm;
41 import org.apache.hadoop.hbase.io.encoding.DataBlockEncoding;
42 import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.MutationType;
43 import org.apache.hadoop.hbase.util.test.LoadTestDataGenerator;
44 import org.apache.hadoop.hbase.util.test.LoadTestKVGenerator;
45 import org.apache.hadoop.util.StringUtils;
46
47 import com.google.common.base.Preconditions;
48
49
50
51
52
53 public abstract class MultiThreadedAction {
54 private static final Log LOG = LogFactory.getLog(MultiThreadedAction.class);
55
56 protected final TableName tableName;
57 protected final Configuration conf;
58 protected final HConnection connection;
59
60 protected int numThreads = 1;
61
62
63 protected long startKey = 0;
64
65
66 protected long endKey = 1;
67
68 protected AtomicInteger numThreadsWorking = new AtomicInteger();
69 protected AtomicLong numKeys = new AtomicLong();
70 protected AtomicLong numCols = new AtomicLong();
71 protected AtomicLong totalOpTimeMs = new AtomicLong();
72 protected boolean verbose = false;
73
74 protected LoadTestDataGenerator dataGenerator = null;
75
76
77
78
79
80
81
82
83 public static class DefaultDataGenerator extends LoadTestDataGenerator {
84 private byte[][] columnFamilies = null;
85 private int minColumnsPerKey;
86 private int maxColumnsPerKey;
87 private final Random random = new Random();
88
89 public DefaultDataGenerator(int minValueSize, int maxValueSize,
90 int minColumnsPerKey, int maxColumnsPerKey, byte[]... columnFamilies) {
91 super(minValueSize, maxValueSize);
92 this.columnFamilies = columnFamilies;
93 this.minColumnsPerKey = minColumnsPerKey;
94 this.maxColumnsPerKey = maxColumnsPerKey;
95 }
96
97 public DefaultDataGenerator(byte[]... columnFamilies) {
98
99 this(256, 1024, 1, 10, columnFamilies);
100 }
101
102 @Override
103 public byte[] getDeterministicUniqueKey(long keyBase) {
104 return LoadTestKVGenerator.md5PrefixedKey(keyBase).getBytes();
105 }
106
107 @Override
108 public byte[][] getColumnFamilies() {
109 return columnFamilies;
110 }
111
112 @Override
113 public byte[][] generateColumnsForCf(byte[] rowKey, byte[] cf) {
114 int numColumns = minColumnsPerKey + random.nextInt(maxColumnsPerKey - minColumnsPerKey + 1);
115 byte[][] columns = new byte[numColumns][];
116 for (int i = 0; i < numColumns; ++i) {
117 columns[i] = Integer.toString(i).getBytes();
118 }
119 return columns;
120 }
121
122 @Override
123 public byte[] generateValue(byte[] rowKey, byte[] cf, byte[] column) {
124 return kvGenerator.generateRandomSizeValue(rowKey, cf, column);
125 }
126
127 @Override
128 public boolean verify(byte[] rowKey, byte[] cf, byte[] column, byte[] value) {
129 return LoadTestKVGenerator.verify(value, rowKey, cf, column);
130 }
131
132 @Override
133 public boolean verify(byte[] rowKey, byte[] cf, Set<byte[]> columnSet) {
134 return (columnSet.size() >= minColumnsPerKey) && (columnSet.size() <= maxColumnsPerKey);
135 }
136 }
137
138
139 private String actionLetter;
140
141
142 private boolean streamingCounters;
143
144 public static final int REPORTING_INTERVAL_MS = 5000;
145
146 public MultiThreadedAction(LoadTestDataGenerator dataGen, Configuration conf,
147 TableName tableName,
148 String actionLetter) throws IOException {
149 this.conf = conf;
150 this.dataGenerator = dataGen;
151 this.tableName = tableName;
152 this.actionLetter = actionLetter;
153 this.connection = HConnectionManager.createConnection(conf);
154 }
155
156 public void start(long startKey, long endKey, int numThreads) throws IOException {
157 this.startKey = startKey;
158 this.endKey = endKey;
159 this.numThreads = numThreads;
160 (new Thread(new ProgressReporter(actionLetter))).start();
161 }
162
163 private static String formatTime(long elapsedTime) {
164 String format = String.format("%%0%dd", 2);
165 elapsedTime = elapsedTime / 1000;
166 String seconds = String.format(format, elapsedTime % 60);
167 String minutes = String.format(format, (elapsedTime % 3600) / 60);
168 String hours = String.format(format, elapsedTime / 3600);
169 String time = hours + ":" + minutes + ":" + seconds;
170 return time;
171 }
172
173
174 private class ProgressReporter implements Runnable {
175
176 private String reporterId = "";
177
178 public ProgressReporter(String id) {
179 this.reporterId = id;
180 }
181
182 @Override
183 public void run() {
184 long startTime = System.currentTimeMillis();
185 long priorNumKeys = 0;
186 long priorCumulativeOpTime = 0;
187 int priorAverageKeysPerSecond = 0;
188
189
190 Threads.sleep(REPORTING_INTERVAL_MS);
191
192 while (numThreadsWorking.get() != 0) {
193 String threadsLeft =
194 "[" + reporterId + ":" + numThreadsWorking.get() + "] ";
195 if (numKeys.get() == 0) {
196 LOG.info(threadsLeft + "Number of keys = 0");
197 } else {
198 long numKeys = MultiThreadedAction.this.numKeys.get();
199 long time = System.currentTimeMillis() - startTime;
200 long totalOpTime = totalOpTimeMs.get();
201
202 long numKeysDelta = numKeys - priorNumKeys;
203 long totalOpTimeDelta = totalOpTime - priorCumulativeOpTime;
204
205 double averageKeysPerSecond =
206 (time > 0) ? (numKeys * 1000 / time) : 0;
207
208 LOG.info(threadsLeft
209 + "Keys="
210 + numKeys
211 + ", cols="
212 + StringUtils.humanReadableInt(numCols.get())
213 + ", time="
214 + formatTime(time)
215 + ((numKeys > 0 && time > 0) ? (" Overall: [" + "keys/s= "
216 + numKeys * 1000 / time + ", latency=" + totalOpTime
217 / numKeys + " ms]") : "")
218 + ((numKeysDelta > 0) ? (" Current: [" + "keys/s="
219 + numKeysDelta * 1000 / REPORTING_INTERVAL_MS + ", latency="
220 + totalOpTimeDelta / numKeysDelta + " ms]") : "")
221 + progressInfo());
222
223 if (streamingCounters) {
224 printStreamingCounters(numKeysDelta,
225 averageKeysPerSecond - priorAverageKeysPerSecond);
226 }
227
228 priorNumKeys = numKeys;
229 priorCumulativeOpTime = totalOpTime;
230 priorAverageKeysPerSecond = (int) averageKeysPerSecond;
231 }
232
233 Threads.sleep(REPORTING_INTERVAL_MS);
234 }
235 }
236
237 private void printStreamingCounters(long numKeysDelta,
238 double avgKeysPerSecondDelta) {
239
240
241 System.err.println("reporter:counter:numKeys," + reporterId + ","
242 + numKeysDelta);
243 System.err.println("reporter:counter:numCols," + reporterId + ","
244 + numCols.get());
245 System.err.println("reporter:counter:avgKeysPerSecond," + reporterId
246 + "," + (long) (avgKeysPerSecondDelta));
247 }
248 }
249
250 public void close() {
251 if (connection != null) {
252 try {
253 connection.close();
254 } catch (Exception ex) {
255 LOG.warn("Could not close the connection: " + ex);
256 }
257 }
258 }
259
260 public void waitForFinish() {
261 while (numThreadsWorking.get() != 0) {
262 Threads.sleepWithoutInterrupt(1000);
263 }
264 close();
265 }
266
267 public boolean isDone() {
268 return (numThreadsWorking.get() == 0);
269 }
270
271 protected void startThreads(Collection<? extends Thread> threads) {
272 numThreadsWorking.addAndGet(threads.size());
273 for (Thread thread : threads) {
274 thread.start();
275 }
276 }
277
278
279 public long getEndKey() {
280 return endKey;
281 }
282
283
284 protected abstract String progressInfo();
285
286 protected static void appendToStatus(StringBuilder sb, String desc,
287 long v) {
288 if (v == 0) {
289 return;
290 }
291 sb.append(", ");
292 sb.append(desc);
293 sb.append("=");
294 sb.append(v);
295 }
296
297 protected static void appendToStatus(StringBuilder sb, String desc,
298 String v) {
299 sb.append(", ");
300 sb.append(desc);
301 sb.append("=");
302 sb.append(v);
303 }
304
305
306
307
308
309 public boolean verifyResultAgainstDataGenerator(Result result, boolean verifyValues) {
310 return verifyResultAgainstDataGenerator(result, verifyValues, false);
311 }
312
313
314
315
316
317
318
319
320
321
322 public boolean verifyResultAgainstDataGenerator(Result result, boolean verifyValues,
323 boolean verifyCfAndColumnIntegrity) {
324 String rowKeyStr = Bytes.toString(result.getRow());
325
326
327 if (result.isEmpty()) {
328 LOG.error("Error checking data for key [" + rowKeyStr + "], no data returned");
329 return false;
330 }
331
332 if (!verifyValues && !verifyCfAndColumnIntegrity) {
333 return true;
334 }
335
336
337 byte[][] expectedCfs = dataGenerator.getColumnFamilies();
338 if (verifyCfAndColumnIntegrity && (expectedCfs.length != result.getMap().size())) {
339 LOG.error("Error checking data for key [" + rowKeyStr
340 + "], bad family count: " + result.getMap().size());
341 return false;
342 }
343
344
345 for (byte[] cf : result.getMap().keySet()) {
346 String cfStr = Bytes.toString(cf);
347 Map<byte[], byte[]> columnValues = result.getFamilyMap(cf);
348 if (columnValues == null) {
349 LOG.error("Error checking data for key [" + rowKeyStr
350 + "], no data for family [" + cfStr + "]]");
351 return false;
352 }
353
354 Map<String, MutationType> mutateInfo = null;
355 if (verifyCfAndColumnIntegrity || verifyValues) {
356 if (!columnValues.containsKey(MUTATE_INFO)) {
357 LOG.error("Error checking data for key [" + rowKeyStr + "], column family ["
358 + cfStr + "], column [" + Bytes.toString(MUTATE_INFO) + "]; value is not found");
359 return false;
360 }
361
362 long cfHash = Arrays.hashCode(cf);
363
364 byte[] mutateInfoValue = columnValues.remove(MUTATE_INFO);
365 mutateInfo = parseMutateInfo(mutateInfoValue);
366 for (Map.Entry<String, MutationType> mutate: mutateInfo.entrySet()) {
367 if (mutate.getValue() == MutationType.DELETE) {
368 byte[] column = Bytes.toBytes(mutate.getKey());
369 long columnHash = Arrays.hashCode(column);
370 long hashCode = cfHash + columnHash;
371 if (hashCode % 2 == 0) {
372 if (columnValues.containsKey(column)) {
373 LOG.error("Error checking data for key [" + rowKeyStr + "], column family ["
374 + cfStr + "], column [" + mutate.getKey() + "]; should be deleted");
375 return false;
376 }
377 byte[] hashCodeBytes = Bytes.toBytes(hashCode);
378 columnValues.put(column, hashCodeBytes);
379 }
380 }
381 }
382
383
384 if (!columnValues.containsKey(INCREMENT)) {
385 LOG.error("Error checking data for key [" + rowKeyStr + "], column family ["
386 + cfStr + "], column [" + Bytes.toString(INCREMENT) + "]; value is not found");
387 return false;
388 }
389 long currentValue = Bytes.toLong(columnValues.remove(INCREMENT));
390 if (verifyValues) {
391 long amount = mutateInfo.isEmpty() ? 0 : cfHash;
392 long originalValue = Arrays.hashCode(result.getRow());
393 long extra = currentValue - originalValue;
394 if (extra != 0 && (amount == 0 || extra % amount != 0)) {
395 LOG.error("Error checking data for key [" + rowKeyStr + "], column family ["
396 + cfStr + "], column [increment], extra [" + extra + "], amount [" + amount + "]");
397 return false;
398 }
399 if (amount != 0 && extra != amount) {
400 LOG.warn("Warning checking data for key [" + rowKeyStr + "], column family ["
401 + cfStr + "], column [increment], incremented [" + (extra / amount) + "] times");
402 }
403 }
404
405
406 if (verifyCfAndColumnIntegrity
407 && !dataGenerator.verify(result.getRow(), cf, columnValues.keySet())) {
408 String colsStr = "";
409 for (byte[] col : columnValues.keySet()) {
410 if (colsStr.length() > 0) {
411 colsStr += ", ";
412 }
413 colsStr += "[" + Bytes.toString(col) + "]";
414 }
415 LOG.error("Error checking data for key [" + rowKeyStr
416 + "], bad columns for family [" + cfStr + "]: " + colsStr);
417 return false;
418 }
419
420 if (verifyValues) {
421 for (Map.Entry<byte[], byte[]> kv : columnValues.entrySet()) {
422 String column = Bytes.toString(kv.getKey());
423 MutationType mutation = mutateInfo.get(column);
424 boolean verificationNeeded = true;
425 byte[] bytes = kv.getValue();
426 if (mutation != null) {
427 boolean mutationVerified = true;
428 long columnHash = Arrays.hashCode(kv.getKey());
429 long hashCode = cfHash + columnHash;
430 byte[] hashCodeBytes = Bytes.toBytes(hashCode);
431 if (mutation == MutationType.APPEND) {
432 int offset = bytes.length - hashCodeBytes.length;
433 mutationVerified = offset > 0 && Bytes.equals(hashCodeBytes,
434 0, hashCodeBytes.length, bytes, offset, hashCodeBytes.length);
435 if (mutationVerified) {
436 int n = 1;
437 while (true) {
438 int newOffset = offset - hashCodeBytes.length;
439 if (newOffset < 0 || !Bytes.equals(hashCodeBytes, 0,
440 hashCodeBytes.length, bytes, newOffset, hashCodeBytes.length)) {
441 break;
442 }
443 offset = newOffset;
444 n++;
445 }
446 if (n > 1) {
447 LOG.warn("Warning checking data for key [" + rowKeyStr + "], column family ["
448 + cfStr + "], column [" + column + "], appended [" + n + "] times");
449 }
450 byte[] dest = new byte[offset];
451 System.arraycopy(bytes, 0, dest, 0, offset);
452 bytes = dest;
453 }
454 } else if (hashCode % 2 == 0) {
455 mutationVerified = Bytes.equals(bytes, hashCodeBytes);
456 verificationNeeded = false;
457 }
458 if (!mutationVerified) {
459 LOG.error("Error checking data for key [" + rowKeyStr
460 + "], mutation checking failed for column family [" + cfStr + "], column ["
461 + column + "]; mutation [" + mutation + "], hashCode ["
462 + hashCode + "], verificationNeeded ["
463 + verificationNeeded + "]");
464 return false;
465 }
466 }
467 if (verificationNeeded &&
468 !dataGenerator.verify(result.getRow(), cf, kv.getKey(), bytes)) {
469 LOG.error("Error checking data for key [" + rowKeyStr + "], column family ["
470 + cfStr + "], column [" + column + "], mutation [" + mutation
471 + "]; value of length " + bytes.length);
472 return false;
473 }
474 }
475 }
476 }
477 }
478 return true;
479 }
480
481
482 private Map<String, MutationType> parseMutateInfo(byte[] mutateInfo) {
483 Map<String, MutationType> mi = new HashMap<String, MutationType>();
484 if (mutateInfo != null) {
485 String mutateInfoStr = Bytes.toString(mutateInfo);
486 String[] mutations = mutateInfoStr.split("#");
487 for (String mutation: mutations) {
488 if (mutation.isEmpty()) continue;
489 Preconditions.checkArgument(mutation.contains(":"),
490 "Invalid mutation info " + mutation);
491 int p = mutation.indexOf(":");
492 String column = mutation.substring(0, p);
493 MutationType type = MutationType.valueOf(
494 Integer.parseInt(mutation.substring(p+1)));
495 mi.put(column, type);
496 }
497 }
498 return mi;
499 }
500 }