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