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.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  /** 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 MultiThreadedWriter 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(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     /** The "current" key being read. Increases from startKey to endKey. */
114     private long curKey;
115 
116     /** Time when the thread started */
117     private long startTimeMs;
118 
119     /** If we are ahead of the writer and reading a random key. */
120     private boolean readingRandomKey;
121 
122     /**
123      * @param readerId only the keys with this remainder from division by
124      *          {@link #numThreads} will be read by this thread
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         // A sanity check for the key range.
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           // Skip keys that this thread should not read, as well as the keys
167           // that we know the writer failed to write.
168           continue;
169         }
170 
171         readKey(k);
172         if (k == curKey - 1 && !readingRandomKey) {
173           // We have verified another unique key.
174           numUniqueKeysVerified.incrementAndGet();
175         }
176       }
177     }
178 
179     /**
180      * Should only be used for the concurrent writer/reader workload. The
181      * maximum key we are allowed to read, subject to the "key window"
182      * constraint.
183      */
184     private long maxKeyWeCanRead() {
185       long insertedUpToKey = writer.insertedUpToKey();
186       if (insertedUpToKey >= endKey - 1) {
187         // The writer has finished writing our range, so we can read any
188         // key in the range.
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       // We caught up with the writer. See if we can read any keys at all.
201       long maxKeyToRead;
202       while ((maxKeyToRead = maxKeyWeCanRead()) < startKey) {
203         // The writer has not written sufficient keys for us to be able to read
204         // anything at all. Sleep a bit. This should only happen in the
205         // beginning of a load test run.
206         Threads.sleepWithoutInterrupt(50);
207       }
208 
209       if (curKey <= maxKeyToRead) {
210         // The writer wrote some keys, and we are now allowed to read our
211         // current key.
212         return curKey++;
213       }
214 
215       // startKey <= maxKeyToRead <= curKey - 1. Read one of the previous keys.
216       // Don't increment the current key -- we still have to try reading it
217       // later. Set a flag to make sure that we don't count this key towards
218       // the set of unique keys we have verified.
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       // read the data
248       long start = System.currentTimeMillis();
249       Result result = table.get(get);
250       totalOpTimeMs.addAndGet(System.currentTimeMillis() - start);
251       numKeys.addAndGet(1);
252 
253       // if we got no data report error
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         // increment number of columns read
273         numCols.addAndGet(result.getFamilyMap(columnFamily).size());
274 
275         if (verify) {
276           // verify the result
277           List<KeyValue> keyValues = result.list();
278           for (KeyValue kv : keyValues) {
279             String qual = new String(kv.getQualifier());
280 
281             // if something does not look right report it
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 }