1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
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.HTable;
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.protobuf.generated.ClientProtos.MutationProto.MutationType;
50 import org.apache.hadoop.hbase.util.test.LoadTestDataGenerator;
51 import org.apache.hadoop.util.StringUtils;
52
53 import com.google.common.base.Preconditions;
54
55
56 public class MultiThreadedUpdater extends MultiThreadedWriterBase {
57 private static final Log LOG = LogFactory.getLog(MultiThreadedUpdater.class);
58
59 private Set<HBaseUpdaterThread> updaters = new HashSet<HBaseUpdaterThread>();
60
61 private MultiThreadedWriterBase writer = null;
62 private boolean isBatchUpdate = false;
63 private final double updatePercent;
64
65 public MultiThreadedUpdater(LoadTestDataGenerator dataGen, Configuration conf,
66 TableName tableName, double updatePercent) {
67 super(dataGen, conf, tableName, "U");
68 this.updatePercent = updatePercent;
69 }
70
71
72 public void setBatchUpdate(boolean isBatchUpdate) {
73 this.isBatchUpdate = isBatchUpdate;
74 }
75
76 public void linkToWriter(MultiThreadedWriterBase writer) {
77 this.writer = writer;
78 writer.setTrackWroteKeys(true);
79 }
80
81 @Override
82 public void start(long startKey, long endKey, int numThreads)
83 throws IOException {
84 super.start(startKey, endKey, numThreads);
85
86 if (verbose) {
87 LOG.debug("Updating keys [" + startKey + ", " + endKey + ")");
88 }
89
90 for (int i = 0; i < numThreads; ++i) {
91 HBaseUpdaterThread updater = new HBaseUpdaterThread(i);
92 updaters.add(updater);
93 }
94
95 startThreads(updaters);
96 }
97
98 private long getNextKeyToUpdate() {
99 if (writer == null) {
100 return nextKeyToWrite.getAndIncrement();
101 }
102 synchronized (this) {
103 if (nextKeyToWrite.get() >= endKey) {
104
105 return endKey;
106 }
107 while (nextKeyToWrite.get() > writer.wroteUpToKey()) {
108 Threads.sleepWithoutInterrupt(100);
109 }
110 long k = nextKeyToWrite.getAndIncrement();
111 if (writer.failedToWriteKey(k)) {
112 failedKeySet.add(k);
113 return getNextKeyToUpdate();
114 }
115 return k;
116 }
117 }
118
119 private class HBaseUpdaterThread extends Thread {
120 private final HTable table;
121
122 public HBaseUpdaterThread(int updaterId) throws IOException {
123 setName(getClass().getSimpleName() + "_" + updaterId);
124 table = new HTable(conf, tableName);
125 }
126
127 public void run() {
128 try {
129 long rowKeyBase;
130 StringBuilder buf = new StringBuilder();
131 byte[][] columnFamilies = dataGenerator.getColumnFamilies();
132 while ((rowKeyBase = getNextKeyToUpdate()) < endKey) {
133 if (RandomUtils.nextInt(100) < updatePercent) {
134 byte[] rowKey = dataGenerator.getDeterministicUniqueKey(rowKeyBase);
135 Increment inc = new Increment(rowKey);
136 Append app = new Append(rowKey);
137 numKeys.addAndGet(1);
138 int columnCount = 0;
139 for (byte[] cf : columnFamilies) {
140 long cfHash = Arrays.hashCode(cf);
141 inc.addColumn(cf, INCREMENT, cfHash);
142 buf.setLength(0);
143 buf.append("#").append(Bytes.toString(INCREMENT));
144 buf.append(":").append(MutationType.INCREMENT.getNumber());
145 app.add(cf, MUTATE_INFO, Bytes.toBytes(buf.toString()));
146 ++columnCount;
147 if (!isBatchUpdate) {
148 mutate(table, inc, rowKeyBase);
149 numCols.addAndGet(1);
150 inc = new Increment(rowKey);
151 mutate(table, app, rowKeyBase);
152 numCols.addAndGet(1);
153 app = new Append(rowKey);
154 }
155 Result result = null;
156 try {
157 Get get = new Get(rowKey);
158 get.addFamily(cf);
159 result = table.get(get);
160 } catch (IOException ie) {
161 LOG.warn("Failed to get the row for key = ["
162 + rowKey + "], column family = [" + Bytes.toString(cf) + "]", ie);
163 }
164 Map<byte[], byte[]> columnValues =
165 result != null ? result.getFamilyMap(cf) : null;
166 if (columnValues == null) {
167 failedKeySet.add(rowKeyBase);
168 LOG.error("Failed to update the row with key = ["
169 + rowKey + "], since we could not get the original row");
170 }
171 for (byte[] column : columnValues.keySet()) {
172 if (Bytes.equals(column, INCREMENT)
173 || Bytes.equals(column, MUTATE_INFO)) {
174 continue;
175 }
176 MutationType mt = MutationType.valueOf(
177 RandomUtils.nextInt(MutationType.values().length));
178 long columnHash = Arrays.hashCode(column);
179 long hashCode = cfHash + columnHash;
180 byte[] hashCodeBytes = Bytes.toBytes(hashCode);
181 byte[] checkedValue = HConstants.EMPTY_BYTE_ARRAY;
182 if (hashCode % 2 == 0) {
183 Cell kv = result.getColumnLatestCell(cf, column);
184 checkedValue = kv != null ? CellUtil.cloneValue(kv) : null;
185 Preconditions.checkNotNull(checkedValue,
186 "Column value to be checked should not be null");
187 }
188 buf.setLength(0);
189 buf.append("#").append(Bytes.toString(column)).append(":");
190 ++columnCount;
191 switch (mt) {
192 case PUT:
193 Put put = new Put(rowKey);
194 put.add(cf, column, hashCodeBytes);
195 mutate(table, put, rowKeyBase, rowKey, cf, column, checkedValue);
196 buf.append(MutationType.PUT.getNumber());
197 break;
198 case DELETE:
199 Delete delete = new Delete(rowKey);
200
201
202 delete.deleteColumns(cf, column);
203 mutate(table, delete, rowKeyBase, rowKey, cf, column, checkedValue);
204 buf.append(MutationType.DELETE.getNumber());
205 break;
206 default:
207 buf.append(MutationType.APPEND.getNumber());
208 app.add(cf, column, hashCodeBytes);
209 }
210 app.add(cf, MUTATE_INFO, Bytes.toBytes(buf.toString()));
211 if (!isBatchUpdate) {
212 mutate(table, app, rowKeyBase);
213 numCols.addAndGet(1);
214 app = new Append(rowKey);
215 }
216 }
217 }
218 if (isBatchUpdate) {
219 if (verbose) {
220 LOG.debug("Preparing increment and append for key = ["
221 + rowKey + "], " + columnCount + " columns");
222 }
223 mutate(table, inc, rowKeyBase);
224 mutate(table, app, rowKeyBase);
225 numCols.addAndGet(columnCount);
226 }
227 }
228 if (trackWroteKeys) {
229 wroteKeys.add(rowKeyBase);
230 }
231 }
232 } finally {
233 try {
234 table.close();
235 } catch (IOException e) {
236 LOG.error("Error closing table", e);
237 }
238 numThreadsWorking.decrementAndGet();
239 }
240 }
241 }
242
243 @Override
244 public void waitForFinish() {
245 super.waitForFinish();
246 System.out.println("Failed to update keys: " + failedKeySet.size());
247 for (Long key : failedKeySet) {
248 System.out.println("Failed to update key: " + key);
249 }
250 }
251
252 public void mutate(HTable table, Mutation m, long keyBase) {
253 mutate(table, m, keyBase, null, null, null, null);
254 }
255
256 public void mutate(HTable table, Mutation m,
257 long keyBase, byte[] row, byte[] cf, byte[] q, byte[] v) {
258 long start = System.currentTimeMillis();
259 try {
260 if (m instanceof Increment) {
261 table.increment((Increment)m);
262 } else if (m instanceof Append) {
263 table.append((Append)m);
264 } else if (m instanceof Put) {
265 table.checkAndPut(row, cf, q, v, (Put)m);
266 } else if (m instanceof Delete) {
267 table.checkAndDelete(row, cf, q, v, (Delete)m);
268 } else {
269 throw new IllegalArgumentException(
270 "unsupported mutation " + m.getClass().getSimpleName());
271 }
272 totalOpTimeMs.addAndGet(System.currentTimeMillis() - start);
273 } catch (IOException e) {
274 failedKeySet.add(keyBase);
275 String exceptionInfo;
276 if (e instanceof RetriesExhaustedWithDetailsException) {
277 RetriesExhaustedWithDetailsException aggEx = (RetriesExhaustedWithDetailsException)e;
278 exceptionInfo = aggEx.getExhaustiveDescription();
279 } else {
280 StringWriter stackWriter = new StringWriter();
281 PrintWriter pw = new PrintWriter(stackWriter);
282 e.printStackTrace(pw);
283 pw.flush();
284 exceptionInfo = StringUtils.stringifyException(e);
285 }
286 LOG.error("Failed to mutate: " + keyBase + " after " + (System.currentTimeMillis() - start) +
287 "ms; region information: " + getRegionDebugInfoSafe(table, m.getRow()) + "; errors: "
288 + exceptionInfo);
289 }
290 }
291 }