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.Set;
30  
31  import org.apache.commons.logging.Log;
32  import org.apache.commons.logging.LogFactory;
33  import org.apache.hadoop.conf.Configuration;
34  import org.apache.hadoop.hbase.HConstants;
35  import org.apache.hadoop.hbase.TableName;
36  import org.apache.hadoop.hbase.client.HTable;
37  import org.apache.hadoop.hbase.client.Put;
38  import org.apache.hadoop.hbase.client.RetriesExhaustedWithDetailsException;
39  import org.apache.hadoop.hbase.util.test.LoadTestDataGenerator;
40  import org.apache.hadoop.util.StringUtils;
41  
42  /** Creates multiple threads that write key/values into the */
43  public class MultiThreadedWriter extends MultiThreadedWriterBase {
44    private static final Log LOG = LogFactory.getLog(MultiThreadedWriter.class);
45  
46    protected Set<HBaseWriterThread> writers = new HashSet<HBaseWriterThread>();
47  
48    protected boolean isMultiPut = false;
49  
50    public MultiThreadedWriter(LoadTestDataGenerator dataGen, Configuration conf,
51        TableName tableName) {
52      super(dataGen, conf, tableName, "W");
53    }
54  
55    /** Use multi-puts vs. separate puts for every column in a row */
56    public void setMultiPut(boolean isMultiPut) {
57      this.isMultiPut = isMultiPut;
58    }
59  
60    @Override
61    public void start(long startKey, long endKey, int numThreads) throws IOException {
62      super.start(startKey, endKey, numThreads);
63  
64      if (verbose) {
65        LOG.debug("Inserting keys [" + startKey + ", " + endKey + ")");
66      }
67  
68      createWriterThreads(numThreads);
69  
70      startThreads(writers);
71    }
72  
73    protected void createWriterThreads(int numThreads) throws IOException {
74      for (int i = 0; i < numThreads; ++i) {
75        HBaseWriterThread writer = new HBaseWriterThread(i);
76        writers.add(writer);
77      }
78    }
79  
80    public class HBaseWriterThread extends Thread {
81      private final HTable table;
82  
83      public HBaseWriterThread(int writerId) throws IOException {
84        setName(getClass().getSimpleName() + "_" + writerId);
85        table = createTable();
86      }
87  
88      protected HTable createTable() throws IOException {
89        return new HTable(conf, tableName);
90      }
91  
92      public void run() {
93        try {
94          long rowKeyBase;
95          byte[][] columnFamilies = dataGenerator.getColumnFamilies();
96          while ((rowKeyBase = nextKeyToWrite.getAndIncrement()) < endKey) {
97            byte[] rowKey = dataGenerator.getDeterministicUniqueKey(rowKeyBase);
98            Put put = new Put(rowKey);
99            numKeys.addAndGet(1);
100           int columnCount = 0;
101           for (byte[] cf : columnFamilies) {
102             byte[][] columns = dataGenerator.generateColumnsForCf(rowKey, cf);
103             for (byte[] column : columns) {
104               byte[] value = dataGenerator.generateValue(rowKey, cf, column);
105               put.add(cf, column, value);
106               ++columnCount;
107               if (!isMultiPut) {
108                 insert(table, put, rowKeyBase);
109                 numCols.addAndGet(1);
110                 put = new Put(rowKey);
111               }
112             }
113             long rowKeyHash = Arrays.hashCode(rowKey);
114             put.add(cf, MUTATE_INFO, HConstants.EMPTY_BYTE_ARRAY);
115             put.add(cf, INCREMENT, Bytes.toBytes(rowKeyHash));
116             if (!isMultiPut) {
117               insert(table, put, rowKeyBase);
118               numCols.addAndGet(1);
119               put = new Put(rowKey);
120             }
121           }
122           if (isMultiPut) {
123             if (verbose) {
124               LOG.debug("Preparing put for key = [" + rowKey + "], " + columnCount + " columns");
125             }
126             insert(table, put, rowKeyBase);
127             numCols.addAndGet(columnCount);
128           }
129           if (trackWroteKeys) {
130             wroteKeys.add(rowKeyBase);
131           }
132         }
133       } finally {
134         closeHTable();
135         numThreadsWorking.decrementAndGet();
136       }
137     }
138 
139     public void insert(HTable table, Put put, long keyBase) {
140       long start = System.currentTimeMillis();
141       try {
142         put = (Put) dataGenerator.beforeMutate(keyBase, put);
143         table.put(put);
144         totalOpTimeMs.addAndGet(System.currentTimeMillis() - start);
145       } catch (IOException e) {
146         failedKeySet.add(keyBase);
147         String exceptionInfo;
148         if (e instanceof RetriesExhaustedWithDetailsException) {
149           RetriesExhaustedWithDetailsException aggEx = (RetriesExhaustedWithDetailsException)e;
150           exceptionInfo = aggEx.getExhaustiveDescription();
151         } else {
152           StringWriter stackWriter = new StringWriter();
153           PrintWriter pw = new PrintWriter(stackWriter);
154           e.printStackTrace(pw);
155           pw.flush();
156           exceptionInfo = StringUtils.stringifyException(e);
157         }
158         LOG.error("Failed to insert: " + keyBase + " after " + (System.currentTimeMillis() - start)
159             + "ms; region information: " + getRegionDebugInfoSafe(table, put.getRow())
160             + "; errors: " + exceptionInfo);
161       }
162     }
163     protected void closeHTable() {
164       try {
165         if (table != null) {
166           table.close();
167         }
168       } catch (IOException e) {
169         LOG.error("Error closing table", e);
170       }
171     }
172   }
173 
174   @Override
175   public void waitForFinish() {
176     super.waitForFinish();
177     System.out.println("Failed to write keys: " + failedKeySet.size());
178     for (Long key : failedKeySet) {
179        System.out.println("Failed to write key: " + key);
180     }
181   }
182 }