View Javadoc

1   /**
2    * Licensed to the Apache Software Foundation (ASF) under one
3    * or more contributor license agreements.  See the NOTICE file
4    * distributed with this work for additional information
5    * regarding copyright ownership.  The ASF licenses this file
6    * to you under the Apache License, Version 2.0 (the
7    * "License"); you may not use this file except in compliance
8    * with the License.  You may obtain a copy of the License at
9    *
10   *     http://www.apache.org/licenses/LICENSE-2.0
11   *
12   * Unless required by applicable law or agreed to in writing, software
13   * distributed under the License is distributed on an "AS IS" BASIS,
14   * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15   * See the License for the specific language governing permissions and
16   * limitations under the License.
17   */
18  
19  package org.apache.hadoop.hbase.util;
20  
21  import static org.apache.hadoop.hbase.util.test.LoadTestDataGenerator.INCREMENT;
22  import static org.apache.hadoop.hbase.util.test.LoadTestDataGenerator.MUTATE_INFO;
23  
24  import java.io.IOException;
25  import java.io.PrintWriter;
26  import java.io.StringWriter;
27  import java.util.Arrays;
28  import java.util.HashSet;
29  import java.util.Map;
30  import java.util.Set;
31  
32  import org.apache.commons.lang.math.RandomUtils;
33  import org.apache.commons.logging.Log;
34  import org.apache.commons.logging.LogFactory;
35  import org.apache.hadoop.conf.Configuration;
36  import org.apache.hadoop.hbase.Cell;
37  import org.apache.hadoop.hbase.CellUtil;
38  import org.apache.hadoop.hbase.HConstants;
39  import org.apache.hadoop.hbase.TableName;
40  import org.apache.hadoop.hbase.client.Append;
41  import org.apache.hadoop.hbase.client.Delete;
42  import org.apache.hadoop.hbase.client.Get;
43  import org.apache.hadoop.hbase.client.HTableInterface;
44  import org.apache.hadoop.hbase.client.Increment;
45  import org.apache.hadoop.hbase.client.Mutation;
46  import org.apache.hadoop.hbase.client.Put;
47  import org.apache.hadoop.hbase.client.Result;
48  import org.apache.hadoop.hbase.client.RetriesExhaustedWithDetailsException;
49  import org.apache.hadoop.hbase.client.Table;
50  import org.apache.hadoop.hbase.exceptions.OperationConflictException;
51  import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.MutationType;
52  import org.apache.hadoop.hbase.util.test.LoadTestDataGenerator;
53  import org.apache.hadoop.util.StringUtils;
54  
55  import com.google.common.base.Preconditions;
56  
57  /** Creates multiple threads that write key/values into the */
58  public class MultiThreadedUpdater extends MultiThreadedWriterBase {
59    private static final Log LOG = LogFactory.getLog(MultiThreadedUpdater.class);
60  
61    protected Set<HBaseUpdaterThread> updaters = new HashSet<HBaseUpdaterThread>();
62  
63    private MultiThreadedWriterBase writer = null;
64    private boolean isBatchUpdate = false;
65    private boolean ignoreNonceConflicts = false;
66    private final double updatePercent;
67  
68    public MultiThreadedUpdater(LoadTestDataGenerator dataGen, Configuration conf,
69        TableName tableName, double updatePercent) throws IOException {
70      super(dataGen, conf, tableName, "U");
71      this.updatePercent = updatePercent;
72    }
73  
74    /** Use batch vs. separate updates for every column in a row */
75    public void setBatchUpdate(boolean isBatchUpdate) {
76      this.isBatchUpdate = isBatchUpdate;
77    }
78  
79    public void linkToWriter(MultiThreadedWriterBase writer) {
80      this.writer = writer;
81      writer.setTrackWroteKeys(true);
82    }
83  
84    @Override
85    public void start(long startKey, long endKey, int numThreads) throws IOException {
86      super.start(startKey, endKey, numThreads);
87  
88      if (verbose) {
89        LOG.debug("Updating keys [" + startKey + ", " + endKey + ")");
90      }
91  
92      addUpdaterThreads(numThreads);
93  
94      startThreads(updaters);
95    }
96  
97    protected void addUpdaterThreads(int numThreads) throws IOException {
98      for (int i = 0; i < numThreads; ++i) {
99        HBaseUpdaterThread updater = new HBaseUpdaterThread(i);
100       updaters.add(updater);
101     }
102   }
103 
104   private long getNextKeyToUpdate() {
105     if (writer == null) {
106       return nextKeyToWrite.getAndIncrement();
107     }
108     synchronized (this) {
109       if (nextKeyToWrite.get() >= endKey) {
110         // Finished the whole key range
111         return endKey;
112       }
113       while (nextKeyToWrite.get() > writer.wroteUpToKey()) {
114         Threads.sleepWithoutInterrupt(100);
115       }
116       long k = nextKeyToWrite.getAndIncrement();
117       if (writer.failedToWriteKey(k)) {
118         failedKeySet.add(k);
119         return getNextKeyToUpdate();
120       }
121       return k;
122     }
123   }
124 
125   protected class HBaseUpdaterThread extends Thread {
126     protected final Table table;
127 
128     public HBaseUpdaterThread(int updaterId) throws IOException {
129       setName(getClass().getSimpleName() + "_" + updaterId);
130       table = createTable();
131     }
132 
133     protected HTableInterface createTable() throws IOException {
134       return connection.getTable(tableName);
135     }
136 
137     @Override
138     public void run() {
139       try {
140         long rowKeyBase;
141         StringBuilder buf = new StringBuilder();
142         byte[][] columnFamilies = dataGenerator.getColumnFamilies();
143         while ((rowKeyBase = getNextKeyToUpdate()) < endKey) {
144           if (RandomUtils.nextInt(100) < updatePercent) {
145             byte[] rowKey = dataGenerator.getDeterministicUniqueKey(rowKeyBase);
146             Increment inc = new Increment(rowKey);
147             Append app = new Append(rowKey);
148             numKeys.addAndGet(1);
149             int columnCount = 0;
150             for (byte[] cf : columnFamilies) {
151               long cfHash = Arrays.hashCode(cf);
152               inc.addColumn(cf, INCREMENT, cfHash);
153               buf.setLength(0); // Clear the buffer
154               buf.append("#").append(Bytes.toString(INCREMENT));
155               buf.append(":").append(MutationType.INCREMENT.getNumber());
156               app.add(cf, MUTATE_INFO, Bytes.toBytes(buf.toString()));
157               ++columnCount;
158               if (!isBatchUpdate) {
159                 mutate(table, inc, rowKeyBase);
160                 numCols.addAndGet(1);
161                 inc = new Increment(rowKey);
162                 mutate(table, app, rowKeyBase);
163                 numCols.addAndGet(1);
164                 app = new Append(rowKey);
165               }
166               Get get = new Get(rowKey);
167               get.addFamily(cf);
168               try {
169                 get = dataGenerator.beforeGet(rowKeyBase, get);
170               } catch (Exception e) {
171                 // Ideally wont happen
172                 LOG.warn("Failed to modify the get from the load generator  = ["
173                     + Bytes.toString(get.getRow()) + "], column family = ["
174                     + Bytes.toString(cf) + "]", e);
175               }
176               Result result = getRow(get, rowKeyBase, cf);
177               Map<byte[], byte[]> columnValues =
178                 result != null ? result.getFamilyMap(cf) : null;
179               if (columnValues == null) {
180                 int specialPermCellInsertionFactor = Integer.parseInt(dataGenerator.getArgs()[2]);
181                 if (((int) rowKeyBase % specialPermCellInsertionFactor == 0)) {
182                   LOG.info("Null result expected for the rowkey " + Bytes.toString(rowKey));
183                 } else {
184                   failedKeySet.add(rowKeyBase);
185                   LOG.error("Failed to update the row with key = [" + Bytes.toString(rowKey)
186                       + "], since we could not get the original row");
187                 }
188               }
189               if(columnValues != null) {
190                 for (byte[] column : columnValues.keySet()) {
191                   if (Bytes.equals(column, INCREMENT) || Bytes.equals(column, MUTATE_INFO)) {
192                     continue;
193                   }
194                   MutationType mt = MutationType
195                       .valueOf(RandomUtils.nextInt(MutationType.values().length));
196                   long columnHash = Arrays.hashCode(column);
197                   long hashCode = cfHash + columnHash;
198                   byte[] hashCodeBytes = Bytes.toBytes(hashCode);
199                   byte[] checkedValue = HConstants.EMPTY_BYTE_ARRAY;
200                   if (hashCode % 2 == 0) {
201                     Cell kv = result.getColumnLatestCell(cf, column);
202                     checkedValue = kv != null ? CellUtil.cloneValue(kv) : null;
203                     Preconditions.checkNotNull(checkedValue,
204                         "Column value to be checked should not be null");
205                   }
206                   buf.setLength(0); // Clear the buffer
207                   buf.append("#").append(Bytes.toString(column)).append(":");
208                   ++columnCount;
209                   switch (mt) {
210                   case PUT:
211                     Put put = new Put(rowKey);
212                     put.add(cf, column, hashCodeBytes);
213                     mutate(table, put, rowKeyBase, rowKey, cf, column, checkedValue);
214                     buf.append(MutationType.PUT.getNumber());
215                     break;
216                   case DELETE:
217                     Delete delete = new Delete(rowKey);
218                     // Delete all versions since a put
219                     // could be called multiple times if CM is used
220                     delete.deleteColumns(cf, column);
221                     mutate(table, delete, rowKeyBase, rowKey, cf, column, checkedValue);
222                     buf.append(MutationType.DELETE.getNumber());
223                     break;
224                   default:
225                     buf.append(MutationType.APPEND.getNumber());
226                     app.add(cf, column, hashCodeBytes);
227                   }
228                   app.add(cf, MUTATE_INFO, Bytes.toBytes(buf.toString()));
229                   if (!isBatchUpdate) {
230                     mutate(table, app, rowKeyBase);
231                     numCols.addAndGet(1);
232                     app = new Append(rowKey);
233                   }
234                 }
235               }
236             }
237             if (isBatchUpdate) {
238               if (verbose) {
239                 LOG.debug("Preparing increment and append for key = ["
240                   + Bytes.toString(rowKey) + "], " + columnCount + " columns");
241               }
242               mutate(table, inc, rowKeyBase);
243               mutate(table, app, rowKeyBase);
244               numCols.addAndGet(columnCount);
245             }
246           }
247           if (trackWroteKeys) {
248             wroteKeys.add(rowKeyBase);
249           }
250         }
251       } finally {
252         closeHTable();
253         numThreadsWorking.decrementAndGet();
254       }
255     }
256 
257     protected void closeHTable() {
258       try {
259         if (table != null) {
260           table.close();
261         }
262       } catch (IOException e) {
263         LOG.error("Error closing table", e);
264       }
265     }
266 
267     protected Result getRow(Get get, long rowKeyBase, byte[] cf) {
268       Result result = null;
269       try {
270         result = table.get(get);
271       } catch (IOException ie) {
272         LOG.warn(
273             "Failed to get the row for key = [" + Bytes.toString(get.getRow())
274               + "], column family = [" + Bytes.toString(cf) + "]", ie);
275       }
276       return result;
277     }
278 
279     public void mutate(Table table, Mutation m, long keyBase) {
280       mutate(table, m, keyBase, null, null, null, null);
281     }
282 
283     public void mutate(Table table, Mutation m,
284         long keyBase, byte[] row, byte[] cf, byte[] q, byte[] v) {
285       long start = System.currentTimeMillis();
286       try {
287         m = dataGenerator.beforeMutate(keyBase, m);
288         if (m instanceof Increment) {
289           table.increment((Increment)m);
290         } else if (m instanceof Append) {
291           table.append((Append)m);
292         } else if (m instanceof Put) {
293           table.checkAndPut(row, cf, q, v, (Put)m);
294         } else if (m instanceof Delete) {
295           table.checkAndDelete(row, cf, q, v, (Delete)m);
296         } else {
297           throw new IllegalArgumentException(
298             "unsupported mutation " + m.getClass().getSimpleName());
299         }
300         totalOpTimeMs.addAndGet(System.currentTimeMillis() - start);
301       } catch (IOException e) {
302         if (ignoreNonceConflicts && (e instanceof OperationConflictException)) {
303           LOG.info("Detected nonce conflict, ignoring: " + e.getMessage());
304           totalOpTimeMs.addAndGet(System.currentTimeMillis() - start);
305           return;
306         }
307         failedKeySet.add(keyBase);
308         String exceptionInfo;
309         if (e instanceof RetriesExhaustedWithDetailsException) {
310           RetriesExhaustedWithDetailsException aggEx = (RetriesExhaustedWithDetailsException) e;
311           exceptionInfo = aggEx.getExhaustiveDescription();
312         } else {
313           exceptionInfo = StringUtils.stringifyException(e);
314         }
315         LOG.error("Failed to mutate: " + keyBase + " after " +
316             (System.currentTimeMillis() - start) +
317           "ms; region information: " + getRegionDebugInfoSafe(table, m.getRow()) + "; errors: "
318             + exceptionInfo);
319       }
320     }
321   }
322 
323   @Override
324   public void waitForFinish() {
325     super.waitForFinish();
326     System.out.println("Failed to update keys: " + failedKeySet.size());
327     for (Long key : failedKeySet) {
328        System.out.println("Failed to update key: " + key);
329     }
330   }
331 
332   public void mutate(Table table, Mutation m, long keyBase) {
333     mutate(table, m, keyBase, null, null, null, null);
334   }
335 
336   public void mutate(Table table, Mutation m,
337       long keyBase, byte[] row, byte[] cf, byte[] q, byte[] v) {
338     long start = System.currentTimeMillis();
339     try {
340       m = dataGenerator.beforeMutate(keyBase, m);
341       if (m instanceof Increment) {
342         table.increment((Increment)m);
343       } else if (m instanceof Append) {
344         table.append((Append)m);
345       } else if (m instanceof Put) {
346         table.checkAndPut(row, cf, q, v, (Put)m);
347       } else if (m instanceof Delete) {
348         table.checkAndDelete(row, cf, q, v, (Delete)m);
349       } else {
350         throw new IllegalArgumentException(
351           "unsupported mutation " + m.getClass().getSimpleName());
352       }
353       totalOpTimeMs.addAndGet(System.currentTimeMillis() - start);
354     } catch (IOException e) {
355       failedKeySet.add(keyBase);
356       String exceptionInfo;
357       if (e instanceof RetriesExhaustedWithDetailsException) {
358         RetriesExhaustedWithDetailsException aggEx = (RetriesExhaustedWithDetailsException)e;
359         exceptionInfo = aggEx.getExhaustiveDescription();
360       } else {
361         StringWriter stackWriter = new StringWriter();
362         PrintWriter pw = new PrintWriter(stackWriter);
363         e.printStackTrace(pw);
364         pw.flush();
365         exceptionInfo = StringUtils.stringifyException(e);
366       }
367       LOG.error("Failed to mutate: " + keyBase + " after " + (System.currentTimeMillis() - start) +
368         "ms; region information: " + getRegionDebugInfoSafe(table, m.getRow()) + "; errors: "
369           + exceptionInfo);
370     }
371   }
372 
373   public void setIgnoreNonceConflicts(boolean value) {
374     this.ignoreNonceConflicts = value;
375   }
376 }