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