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.Collection;
21  import java.util.Map;
22  import java.util.Random;
23  import java.util.Set;
24  import java.util.concurrent.atomic.AtomicInteger;
25  import java.util.concurrent.atomic.AtomicLong;
26  
27  import org.apache.commons.logging.Log;
28  import org.apache.commons.logging.LogFactory;
29  import org.apache.hadoop.conf.Configuration;
30  import org.apache.hadoop.hbase.client.Result;
31  import org.apache.hadoop.util.StringUtils;
32  
33  /**
34   * Common base class for reader and writer parts of multi-thread HBase load
35   * test ({@link LoadTestTool}).
36   */
37  public abstract class MultiThreadedAction {
38    private static final Log LOG = LogFactory.getLog(MultiThreadedAction.class);
39  
40    protected final byte[] tableName;
41    protected final Configuration conf;
42  
43    protected int numThreads = 1;
44  
45    /** The start key of the key range, inclusive */
46    protected long startKey = 0;
47  
48    /** The end key of the key range, exclusive */
49    protected long endKey = 1;
50  
51    protected AtomicInteger numThreadsWorking = new AtomicInteger();
52    protected AtomicLong numKeys = new AtomicLong();
53    protected AtomicLong numCols = new AtomicLong();
54    protected AtomicLong totalOpTimeMs = new AtomicLong();
55    protected boolean verbose = false;
56  
57    protected LoadTestDataGenerator dataGenerator = null;
58  
59    /**
60     * Default implementation of LoadTestDataGenerator that uses LoadTestKVGenerator, fixed
61     * set of column families, and random number of columns in range. The table for it can
62     * be created manually or, for example, via
63     * {@link HBaseTestingUtility#createPreSplitLoadTestTable(
64     * org.apache.hadoop.hbase.Configuration, byte[], byte[], Algorithm, DataBlockEncoding)}
65     */
66    public static class DefaultDataGenerator extends LoadTestDataGenerator {
67      private byte[][] columnFamilies = null;
68      private int minColumnsPerKey;
69      private int maxColumnsPerKey;
70      private final Random random = new Random();
71  
72      public DefaultDataGenerator(int minValueSize, int maxValueSize,
73          int minColumnsPerKey, int maxColumnsPerKey, byte[]... columnFamilies) {
74        super(minValueSize, maxValueSize);
75        this.columnFamilies = columnFamilies;
76        this.minColumnsPerKey = minColumnsPerKey;
77        this.maxColumnsPerKey = maxColumnsPerKey;
78      }
79  
80      public DefaultDataGenerator(byte[]... columnFamilies) {
81        // Default values for tests that didn't care to provide theirs.
82        this(256, 1024, 1, 10, columnFamilies);
83      }
84  
85      @Override
86      public byte[] getDeterministicUniqueKey(long keyBase) {
87        return LoadTestKVGenerator.md5PrefixedKey(keyBase).getBytes();
88      }
89  
90      @Override
91      public byte[][] getColumnFamilies() {
92        return columnFamilies;
93      }
94  
95      @Override
96      public byte[][] generateColumnsForCf(byte[] rowKey, byte[] cf) {
97        int numColumns = minColumnsPerKey + random.nextInt(maxColumnsPerKey - minColumnsPerKey + 1);
98        byte[][] columns = new byte[numColumns][];
99        for (int i = 0; i < numColumns; ++i) {
100         columns[i] = Integer.toString(i).getBytes();
101       }
102       return columns;
103     }
104 
105     @Override
106     public byte[] generateValue(byte[] rowKey, byte[] cf, byte[] column) {
107       return kvGenerator.generateRandomSizeValue(rowKey, cf, column);
108     }
109 
110     @Override
111     public boolean verify(byte[] rowKey, byte[] cf, byte[] column, byte[] value) {
112       return LoadTestKVGenerator.verify(value, rowKey, cf, column);
113     }
114 
115     @Override
116     public boolean verify(byte[] rowKey, byte[] cf, Set<byte[]> columnSet) {
117       return (columnSet.size() >= minColumnsPerKey) && (columnSet.size() <= maxColumnsPerKey);
118     }
119   }
120 
121   /** "R" or "W" */
122   private String actionLetter;
123 
124   /** Whether we need to print out Hadoop Streaming-style counters */
125   private boolean streamingCounters;
126 
127   public static final int REPORTING_INTERVAL_MS = 5000;
128 
129   public MultiThreadedAction(LoadTestDataGenerator dataGen, Configuration conf, byte[] tableName,
130       String actionLetter) {
131     this.conf = conf;
132     this.tableName = tableName;
133     this.dataGenerator = dataGen;
134     this.actionLetter = actionLetter;
135   }
136 
137   public void start(long startKey, long endKey, int numThreads)
138       throws IOException {
139     this.startKey = startKey;
140     this.endKey = endKey;
141     this.numThreads = numThreads;
142     (new Thread(new ProgressReporter(actionLetter))).start();
143   }
144 
145   private static String formatTime(long elapsedTime) {
146     String format = String.format("%%0%dd", 2);
147     elapsedTime = elapsedTime / 1000;
148     String seconds = String.format(format, elapsedTime % 60);
149     String minutes = String.format(format, (elapsedTime % 3600) / 60);
150     String hours = String.format(format, elapsedTime / 3600);
151     String time =  hours + ":" + minutes + ":" + seconds;
152     return time;
153   }
154 
155   /** Asynchronously reports progress */
156   private class ProgressReporter implements Runnable {
157 
158     private String reporterId = "";
159 
160     public ProgressReporter(String id) {
161       this.reporterId = id;
162     }
163 
164     @Override
165     public void run() {
166       long startTime = System.currentTimeMillis();
167       long priorNumKeys = 0;
168       long priorCumulativeOpTime = 0;
169       int priorAverageKeysPerSecond = 0;
170 
171       // Give other threads time to start.
172       Threads.sleep(REPORTING_INTERVAL_MS);
173 
174       while (numThreadsWorking.get() != 0) {
175         String threadsLeft =
176             "[" + reporterId + ":" + numThreadsWorking.get() + "] ";
177         if (numKeys.get() == 0) {
178           LOG.info(threadsLeft + "Number of keys = 0");
179         } else {
180           long numKeys = MultiThreadedAction.this.numKeys.get();
181           long time = System.currentTimeMillis() - startTime;
182           long totalOpTime = totalOpTimeMs.get();
183 
184           long numKeysDelta = numKeys - priorNumKeys;
185           long totalOpTimeDelta = totalOpTime - priorCumulativeOpTime;
186 
187           double averageKeysPerSecond =
188               (time > 0) ? (numKeys * 1000 / time) : 0;
189 
190           LOG.info(threadsLeft
191               + "Keys="
192               + numKeys
193               + ", cols="
194               + StringUtils.humanReadableInt(numCols.get())
195               + ", time="
196               + formatTime(time)
197               + ((numKeys > 0 && time > 0) ? (" Overall: [" + "keys/s= "
198                   + numKeys * 1000 / time + ", latency=" + totalOpTime
199                   / numKeys + " ms]") : "")
200               + ((numKeysDelta > 0) ? (" Current: [" + "keys/s="
201                   + numKeysDelta * 1000 / REPORTING_INTERVAL_MS + ", latency="
202                   + totalOpTimeDelta / numKeysDelta + " ms]") : "")
203               + progressInfo());
204 
205           if (streamingCounters) {
206             printStreamingCounters(numKeysDelta,
207                 averageKeysPerSecond - priorAverageKeysPerSecond);
208           }
209 
210           priorNumKeys = numKeys;
211           priorCumulativeOpTime = totalOpTime;
212           priorAverageKeysPerSecond = (int) averageKeysPerSecond;
213         }
214 
215         Threads.sleep(REPORTING_INTERVAL_MS);
216       }
217     }
218 
219     private void printStreamingCounters(long numKeysDelta,
220         double avgKeysPerSecondDelta) {
221       // Write stats in a format that can be interpreted as counters by
222       // streaming map-reduce jobs.
223       System.err.println("reporter:counter:numKeys," + reporterId + ","
224           + numKeysDelta);
225       System.err.println("reporter:counter:numCols," + reporterId + ","
226           + numCols.get());
227       System.err.println("reporter:counter:avgKeysPerSecond," + reporterId
228           + "," + (long) (avgKeysPerSecondDelta));
229     }
230   }
231 
232   public void waitForFinish() {
233     while (numThreadsWorking.get() != 0) {
234       Threads.sleepWithoutInterrupt(1000);
235     }
236   }
237 
238   public boolean isDone() {
239     return (numThreadsWorking.get() == 0);
240   }
241 
242   protected void startThreads(Collection<? extends Thread> threads) {
243     numThreadsWorking.addAndGet(threads.size());
244     for (Thread thread : threads) {
245       thread.start();
246     }
247   }
248 
249   /** @return the end key of the key range, exclusive */
250   public long getEndKey() {
251     return endKey;
252   }
253 
254   /** Returns a task-specific progress string */
255   protected abstract String progressInfo();
256 
257   protected static void appendToStatus(StringBuilder sb, String desc,
258       long v) {
259     if (v == 0) {
260       return;
261     }
262     sb.append(", ");
263     sb.append(desc);
264     sb.append("=");
265     sb.append(v);
266   }
267 
268   protected static void appendToStatus(StringBuilder sb, String desc,
269       String v) {
270     sb.append(", ");
271     sb.append(desc);
272     sb.append("=");
273     sb.append(v);
274   }
275 
276   /**
277    * See {@link #verifyResultAgainstDataGenerator(Result, boolean, boolean)}.
278    * Does not verify cf/column integrity.
279    */
280   public boolean verifyResultAgainstDataGenerator(Result result, boolean verifyValues) {
281     return verifyResultAgainstDataGenerator(result, verifyValues, false);
282   }
283 
284   /**
285    * Verifies the result from get or scan using the dataGenerator (that was presumably
286    * also used to generate said result).
287    * @param verifyValues verify that values in the result make sense for row/cf/column combination
288    * @param verifyCfAndColumnIntegrity verify that cf/column set in the result is complete. Note
289    *                                   that to use this multiPut should be used, or verification
290    *                                   has to happen after writes, otherwise there can be races.
291    * @return
292    */
293   public boolean verifyResultAgainstDataGenerator(Result result, boolean verifyValues,
294       boolean verifyCfAndColumnIntegrity) {
295     String rowKeyStr = Bytes.toString(result.getRow());
296 
297     // See if we have any data at all.
298     if (result.isEmpty()) {
299       LOG.error("No data returned for key = [" + rowKeyStr + "]");
300       return false;
301     }
302 
303     if (!verifyValues && !verifyCfAndColumnIntegrity) {
304       return true; // as long as we have something, we are good.
305     }
306 
307     // See if we have all the CFs.
308     byte[][] expectedCfs = dataGenerator.getColumnFamilies();
309     if (verifyCfAndColumnIntegrity && (expectedCfs.length != result.getMap().size())) {
310       LOG.error("Bad family count for [" + rowKeyStr + "]: " + result.getMap().size());
311       return false;
312     }
313 
314     // Verify each column family from get in the result.
315     for (byte[] cf : result.getMap().keySet()) {
316       String cfStr = Bytes.toString(cf);
317       Map<byte[], byte[]> columnValues = result.getFamilyMap(cf);
318       if (columnValues == null) {
319         LOG.error("No data for family [" + cfStr + "] for [" + rowKeyStr + "]");
320         return false;
321       }
322       // See if we have correct columns.
323       if (verifyCfAndColumnIntegrity
324           && !dataGenerator.verify(result.getRow(), cf, columnValues.keySet())) {
325         String colsStr = "";
326         for (byte[] col : columnValues.keySet()) {
327           if (colsStr.length() > 0) {
328             colsStr += ", ";
329           }
330           colsStr += "[" + Bytes.toString(col) + "]";
331         }
332         LOG.error("Bad columns for family [" + cfStr + "] for [" + rowKeyStr + "]: " + colsStr);
333         return false;
334       }
335       // See if values check out.
336       if (verifyValues) {
337         for (Map.Entry<byte[], byte[]> kv : columnValues.entrySet()) {
338           if (!dataGenerator.verify(result.getRow(), cf, kv.getKey(), kv.getValue())) {
339             LOG.error("Error checking data for key [" + rowKeyStr + "], column family ["
340               + cfStr + "], column [" + Bytes.toString(kv.getKey()) + "]; value of length " +
341               + kv.getValue().length);
342             return false;
343           }
344         }
345       }
346     }
347     return true;
348   }
349 
350 }