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