View Javadoc

1   /*
2    * Licensed to the Apache Software Foundation (ASF) under one or more
3    * contributor license agreements. See the NOTICE file distributed with this
4    * work for additional information regarding copyright ownership. The ASF
5    * licenses this file to you under the Apache License, Version 2.0 (the
6    * "License"); you may not use this file except in compliance with the License.
7    * You may obtain a copy of the License at
8    *
9    * http://www.apache.org/licenses/LICENSE-2.0
10   *
11   * Unless required by applicable law or agreed to in writing, software
12   * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
13   * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
14   * License for the specific language governing permissions and limitations
15   * under the License.
16   */
17  package org.apache.hadoop.hbase.util;
18  
19  import java.io.IOException;
20  import java.util.HashSet;
21  import java.util.Set;
22  import java.util.concurrent.atomic.AtomicLong;
23  
24  import org.apache.commons.lang.math.RandomUtils;
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.TableName;
30  import org.apache.hadoop.hbase.client.Get;
31  import org.apache.hadoop.hbase.client.HTable;
32  import org.apache.hadoop.hbase.client.Result;
33  import org.apache.hadoop.hbase.util.test.LoadTestDataGenerator;
34  
35  /** Creates multiple threads that read and verify previously written data */
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 MultiThreadedWriterBase writer = null;
45  
46    /**
47     * The number of keys verified in a sequence. This will never be larger than
48     * the total number of keys in the range. The reader might also verify
49     * random keys when it catches up with the writer.
50     */
51    private final AtomicLong numUniqueKeysVerified = new AtomicLong();
52  
53    /**
54     * Default maximum number of read errors to tolerate before shutting down all
55     * readers.
56     */
57    public static final int DEFAULT_MAX_ERRORS = 10;
58  
59    /**
60     * Default "window" size between the last key written by the writer and the
61     * key that we attempt to read. The lower this number, the stricter our
62     * testing is. If this is zero, we always attempt to read the highest key
63     * in the contiguous sequence of keys written by the writers.
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(LoadTestDataGenerator dataGen, Configuration conf,
75        TableName tableName, double verifyPercent) {
76      super(dataGen, conf, tableName, "R");
77      this.verifyPercent = verifyPercent;
78    }
79  
80    public void linkToWriter(MultiThreadedWriterBase writer) {
81      this.writer = writer;
82      writer.setTrackWroteKeys(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 
112     /** The "current" key being read. Increases from startKey to endKey. */
113     private long curKey;
114 
115     /** Time when the thread started */
116     private long startTimeMs;
117 
118     /** If we are ahead of the writer and reading a random key. */
119     private boolean readingRandomKey;
120 
121     /**
122      * @param readerId only the keys with this remainder from division by
123      *          {@link #numThreads} will be read by this thread
124      */
125     public HBaseReaderThread(int readerId) throws IOException {
126       this.readerId = readerId;
127       table = new HTable(conf, tableName);
128       setName(getClass().getSimpleName() + "_" + readerId);
129     }
130 
131     @Override
132     public void run() {
133       try {
134         runReader();
135       } finally {
136         try {
137           table.close();
138         } catch (IOException e) {
139           LOG.error("Error closing table", e);
140         }
141         numThreadsWorking.decrementAndGet();
142       }
143     }
144 
145     private void runReader() {
146       if (verbose) {
147         LOG.info("Started thread #" + readerId + " for reads...");
148       }
149 
150       startTimeMs = System.currentTimeMillis();
151       curKey = startKey;
152       while (curKey < endKey && !aborted) {
153         long k = getNextKeyToRead();
154 
155         // A sanity check for the key range.
156         if (k < startKey || k >= endKey) {
157           numReadErrors.incrementAndGet();
158           throw new AssertionError("Load tester logic error: proposed key " +
159               "to read " + k + " is out of range (startKey=" + startKey +
160               ", endKey=" + endKey + ")");
161         }
162 
163         if (k % numThreads != readerId ||
164             writer != null && writer.failedToWriteKey(k)) {
165           // Skip keys that this thread should not read, as well as the keys
166           // that we know the writer failed to write.
167           continue;
168         }
169 
170         readKey(k);
171         if (k == curKey - 1 && !readingRandomKey) {
172           // We have verified another unique key.
173           numUniqueKeysVerified.incrementAndGet();
174         }
175       }
176     }
177 
178     /**
179      * Should only be used for the concurrent writer/reader workload. The
180      * maximum key we are allowed to read, subject to the "key window"
181      * constraint.
182      */
183     private long maxKeyWeCanRead() {
184       long insertedUpToKey = writer.wroteUpToKey();
185       if (insertedUpToKey >= endKey - 1) {
186         // The writer has finished writing our range, so we can read any
187         // key in the range.
188         return endKey - 1;
189       }
190       return Math.min(endKey - 1, writer.wroteUpToKey() - keyWindow);
191     }
192 
193     private long getNextKeyToRead() {
194       readingRandomKey = false;
195       if (writer == null || curKey <= maxKeyWeCanRead()) {
196         return curKey++;
197       }
198 
199       // We caught up with the writer. See if we can read any keys at all.
200       long maxKeyToRead;
201       while ((maxKeyToRead = maxKeyWeCanRead()) < startKey) {
202         // The writer has not written sufficient keys for us to be able to read
203         // anything at all. Sleep a bit. This should only happen in the
204         // beginning of a load test run.
205         Threads.sleepWithoutInterrupt(50);
206       }
207 
208       if (curKey <= maxKeyToRead) {
209         // The writer wrote some keys, and we are now allowed to read our
210         // current key.
211         return curKey++;
212       }
213 
214       // startKey <= maxKeyToRead <= curKey - 1. Read one of the previous keys.
215       // Don't increment the current key -- we still have to try reading it
216       // later. Set a flag to make sure that we don't count this key towards
217       // the set of unique keys we have verified.
218       readingRandomKey = true;
219       return startKey + Math.abs(RandomUtils.nextLong())
220           % (maxKeyToRead - startKey + 1);
221     }
222 
223     private Get readKey(long keyToRead) {
224       Get get = new Get(dataGenerator.getDeterministicUniqueKey(keyToRead));
225       String cfsString = "";
226       byte[][] columnFamilies = dataGenerator.getColumnFamilies();
227       for (byte[] cf : columnFamilies) {
228         get.addFamily(cf);
229         if (verbose) {
230           if (cfsString.length() > 0) {
231             cfsString += ", ";
232           }
233           cfsString += "[" + Bytes.toStringBinary(cf) + "]";
234         }
235       }
236 
237       try {
238         if (verbose) {
239           LOG.info("[" + readerId + "] " + "Querying key " + keyToRead + ", cfs " + cfsString);
240         }
241         queryKey(get, RandomUtils.nextInt(100) < verifyPercent);
242       } catch (IOException e) {
243         numReadFailures.addAndGet(1);
244         LOG.debug("[" + readerId + "] FAILED read, key = " + (keyToRead + "")
245             + ", time from start: "
246             + (System.currentTimeMillis() - startTimeMs) + " ms");
247       }
248       return get;
249     }
250 
251     public void queryKey(Get get, boolean verify) throws IOException {
252       String rowKey = Bytes.toString(get.getRow());
253 
254       // read the data
255       long start = System.currentTimeMillis();
256       Result result = table.get(get);
257       totalOpTimeMs.addAndGet(System.currentTimeMillis() - start);
258       numKeys.addAndGet(1);
259       if (!result.isEmpty()) {
260         if (verify) {
261           numKeysVerified.incrementAndGet();
262         }
263       } else {
264          HRegionLocation hloc = table.getRegionLocation(
265              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         // Count the columns for reporting purposes.
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 wrote " + 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 }