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 java.io.IOException;
20 import java.util.HashSet;
21 import java.util.List;
22 import java.util.Random;
23 import java.util.Set;
24 import java.util.concurrent.atomic.AtomicLong;
25
26 import org.apache.commons.logging.Log;
27 import org.apache.commons.logging.LogFactory;
28 import org.apache.hadoop.conf.Configuration;
29 import org.apache.hadoop.hbase.HRegionLocation;
30 import org.apache.hadoop.hbase.KeyValue;
31 import org.apache.hadoop.hbase.client.Get;
32 import org.apache.hadoop.hbase.client.HTable;
33 import org.apache.hadoop.hbase.client.Result;
34
35
36 public class MultiThreadedReader extends MultiThreadedAction
37 {
38 private static final Log LOG = LogFactory.getLog(MultiThreadedReader.class);
39
40 private Set<HBaseReaderThread> readers = new HashSet<HBaseReaderThread>();
41 private final double verifyPercent;
42 private volatile boolean aborted;
43
44 private MultiThreadedWriter writer = null;
45
46
47
48
49
50
51 private final AtomicLong numUniqueKeysVerified = new AtomicLong();
52
53
54
55
56
57 public static final int DEFAULT_MAX_ERRORS = 10;
58
59
60
61
62
63
64
65 public static final int DEFAULT_KEY_WINDOW = 0;
66
67 protected AtomicLong numKeysVerified = new AtomicLong(0);
68 private AtomicLong numReadErrors = new AtomicLong(0);
69 private AtomicLong numReadFailures = new AtomicLong(0);
70
71 private int maxErrors = DEFAULT_MAX_ERRORS;
72 private int keyWindow = DEFAULT_KEY_WINDOW;
73
74 public MultiThreadedReader(Configuration conf, byte[] tableName,
75 byte[] columnFamily, double verifyPercent) {
76 super(conf, tableName, columnFamily, "R");
77 this.verifyPercent = verifyPercent;
78 }
79
80 public void linkToWriter(MultiThreadedWriter writer) {
81 this.writer = writer;
82 writer.setTrackInsertedKeys(true);
83 }
84
85 public void setMaxErrors(int maxErrors) {
86 this.maxErrors = maxErrors;
87 }
88
89 public void setKeyWindow(int keyWindow) {
90 this.keyWindow = keyWindow;
91 }
92
93 @Override
94 public void start(long startKey, long endKey, int numThreads)
95 throws IOException {
96 super.start(startKey, endKey, numThreads);
97 if (verbose) {
98 LOG.debug("Reading keys [" + startKey + ", " + endKey + ")");
99 }
100
101 for (int i = 0; i < numThreads; ++i) {
102 HBaseReaderThread reader = new HBaseReaderThread(i);
103 readers.add(reader);
104 }
105 startThreads(readers);
106 }
107
108 public class HBaseReaderThread extends Thread {
109 private final int readerId;
110 private final HTable table;
111 private final Random random = new Random();
112
113
114 private long curKey;
115
116
117 private long startTimeMs;
118
119
120 private boolean readingRandomKey;
121
122
123
124
125
126 public HBaseReaderThread(int readerId) throws IOException {
127 this.readerId = readerId;
128 table = new HTable(conf, tableName);
129 setName(getClass().getSimpleName() + "_" + readerId);
130 }
131
132 @Override
133 public void run() {
134 try {
135 runReader();
136 } finally {
137 try {
138 table.close();
139 } catch (IOException e) {
140 LOG.error("Error closing table", e);
141 }
142 numThreadsWorking.decrementAndGet();
143 }
144 }
145
146 private void runReader() {
147 if (verbose) {
148 LOG.info("Started thread #" + readerId + " for reads...");
149 }
150
151 startTimeMs = System.currentTimeMillis();
152 curKey = startKey;
153 while (curKey < endKey && !aborted) {
154 long k = getNextKeyToRead();
155
156
157 if (k < startKey || k >= endKey) {
158 numReadErrors.incrementAndGet();
159 throw new AssertionError("Load tester logic error: proposed key " +
160 "to read " + k + " is out of range (startKey=" + startKey +
161 ", endKey=" + endKey + ")");
162 }
163
164 if (k % numThreads != readerId ||
165 writer != null && writer.failedToWriteKey(k)) {
166
167
168 continue;
169 }
170
171 readKey(k);
172 if (k == curKey - 1 && !readingRandomKey) {
173
174 numUniqueKeysVerified.incrementAndGet();
175 }
176 }
177 }
178
179
180
181
182
183
184 private long maxKeyWeCanRead() {
185 long insertedUpToKey = writer.insertedUpToKey();
186 if (insertedUpToKey >= endKey - 1) {
187
188
189 return endKey - 1;
190 }
191 return Math.min(endKey - 1, writer.insertedUpToKey() - keyWindow);
192 }
193
194 private long getNextKeyToRead() {
195 readingRandomKey = false;
196 if (writer == null || curKey <= maxKeyWeCanRead()) {
197 return curKey++;
198 }
199
200
201 long maxKeyToRead;
202 while ((maxKeyToRead = maxKeyWeCanRead()) < startKey) {
203
204
205
206 Threads.sleepWithoutInterrupt(50);
207 }
208
209 if (curKey <= maxKeyToRead) {
210
211
212 return curKey++;
213 }
214
215
216
217
218
219 readingRandomKey = true;
220 return startKey + Math.abs(random.nextLong())
221 % (maxKeyToRead - startKey + 1);
222 }
223
224 private Get readKey(long keyToRead) {
225 Get get = new Get(
226 LoadTestKVGenerator.md5PrefixedKey(keyToRead).getBytes());
227 get.addFamily(columnFamily);
228
229 try {
230 if (verbose) {
231 LOG.info("[" + readerId + "] " + "Querying key " + keyToRead
232 + ", cf " + Bytes.toStringBinary(columnFamily));
233 }
234 queryKey(get, random.nextInt(100) < verifyPercent);
235 } catch (IOException e) {
236 numReadFailures.addAndGet(1);
237 LOG.debug("[" + readerId + "] FAILED read, key = " + (keyToRead + "")
238 + ", time from start: "
239 + (System.currentTimeMillis() - startTimeMs) + " ms");
240 }
241 return get;
242 }
243
244 public void queryKey(Get get, boolean verify) throws IOException {
245 String rowKey = Bytes.toString(get.getRow());
246
247
248 long start = System.currentTimeMillis();
249 Result result = table.get(get);
250 totalOpTimeMs.addAndGet(System.currentTimeMillis() - start);
251 numKeys.addAndGet(1);
252
253
254 if (result.isEmpty()) {
255 HRegionLocation hloc = table.getRegionLocation(
256 Bytes.toBytes(rowKey));
257 LOG.info("Key = " + rowKey + ", RegionServer: "
258 + hloc.getHostname());
259 numReadErrors.addAndGet(1);
260 LOG.error("No data returned, tried to get actions for key = "
261 + rowKey + (writer == null ? "" : ", keys inserted by writer: " +
262 writer.numKeys.get() + ")"));
263
264 if (numReadErrors.get() > maxErrors) {
265 LOG.error("Aborting readers -- found more than " + maxErrors
266 + " errors\n");
267 aborted = true;
268 }
269 }
270
271 if (result.getFamilyMap(columnFamily) != null) {
272
273 numCols.addAndGet(result.getFamilyMap(columnFamily).size());
274
275 if (verify) {
276
277 List<KeyValue> keyValues = result.list();
278 for (KeyValue kv : keyValues) {
279 String qual = new String(kv.getQualifier());
280
281
282 if (!LoadTestKVGenerator.verify(rowKey, qual, kv.getValue())) {
283 numReadErrors.addAndGet(1);
284 LOG.error("Error checking data for key = " + rowKey
285 + ", actionId = " + qual);
286 }
287 }
288 numKeysVerified.addAndGet(1);
289 }
290 }
291 }
292
293 }
294
295 public long getNumReadFailures() {
296 return numReadFailures.get();
297 }
298
299 public long getNumReadErrors() {
300 return numReadErrors.get();
301 }
302
303 public long getNumKeysVerified() {
304 return numKeysVerified.get();
305 }
306
307 public long getNumUniqueKeysVerified() {
308 return numUniqueKeysVerified.get();
309 }
310
311 @Override
312 protected String progressInfo() {
313 StringBuilder sb = new StringBuilder();
314 appendToStatus(sb, "verified", numKeysVerified.get());
315 appendToStatus(sb, "READ FAILURES", numReadFailures.get());
316 appendToStatus(sb, "READ ERRORS", numReadErrors.get());
317 return sb.toString();
318 }
319
320 }