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