1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17 package org.apache.hadoop.hbase.util;
18
19 import java.io.IOException;
20 import java.util.HashSet;
21 import java.util.PriorityQueue;
22 import java.util.Queue;
23 import java.util.Random;
24 import java.util.Set;
25 import java.util.concurrent.ArrayBlockingQueue;
26 import java.util.concurrent.BlockingQueue;
27 import java.util.concurrent.ConcurrentSkipListSet;
28 import java.util.concurrent.TimeUnit;
29 import java.util.concurrent.atomic.AtomicLong;
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.client.HTable;
35 import org.apache.hadoop.hbase.client.Put;
36
37
38 public class MultiThreadedWriter extends MultiThreadedAction {
39 private static final Log LOG = LogFactory.getLog(MultiThreadedWriter.class);
40
41 private long minColumnsPerKey = 1;
42 private long maxColumnsPerKey = 10;
43 private Set<HBaseWriterThread> writers = new HashSet<HBaseWriterThread>();
44
45 private boolean isMultiPut = false;
46
47
48
49
50
51
52
53 private BlockingQueue<Long> insertedKeys =
54 new ArrayBlockingQueue<Long>(10000);
55
56
57
58
59
60 private AtomicLong nextKeyToInsert = new AtomicLong();
61
62
63
64
65 private AtomicLong insertedUpToKey = new AtomicLong();
66
67
68 private Set<Long> failedKeySet = new ConcurrentSkipListSet<Long>();
69
70
71
72
73
74
75 private AtomicLong insertedKeyQueueSize = new AtomicLong();
76
77
78 private boolean trackInsertedKeys;
79
80 public MultiThreadedWriter(Configuration conf, byte[] tableName,
81 byte[] columnFamily) {
82 super(conf, tableName, columnFamily, "W");
83 }
84
85
86 public void setMultiPut(boolean isMultiPut) {
87 this.isMultiPut = isMultiPut;
88 }
89
90 public void setColumnsPerKey(long minColumnsPerKey, long maxColumnsPerKey) {
91 this.minColumnsPerKey = minColumnsPerKey;
92 this.maxColumnsPerKey = maxColumnsPerKey;
93 }
94
95 @Override
96 public void start(long startKey, long endKey, int numThreads)
97 throws IOException {
98 super.start(startKey, endKey, numThreads);
99
100 if (verbose) {
101 LOG.debug("Inserting keys [" + startKey + ", " + endKey + ")");
102 }
103
104 nextKeyToInsert.set(startKey);
105 insertedUpToKey.set(startKey - 1);
106
107 for (int i = 0; i < numThreads; ++i) {
108 HBaseWriterThread writer = new HBaseWriterThread(i);
109 writers.add(writer);
110 }
111
112 if (trackInsertedKeys) {
113 new Thread(new InsertedKeysTracker()).start();
114 numThreadsWorking.incrementAndGet();
115 }
116
117 startThreads(writers);
118 }
119
120 public static byte[] longToByteArrayKey(long rowKey) {
121 return LoadTestKVGenerator.md5PrefixedKey(rowKey).getBytes();
122 }
123
124 private class HBaseWriterThread extends Thread {
125 private final HTable table;
126
127 private final Random random = new Random();
128 private final LoadTestKVGenerator dataGenerator = new LoadTestKVGenerator(
129 minDataSize, maxDataSize);
130
131 public HBaseWriterThread(int writerId) throws IOException {
132 setName(getClass().getSimpleName() + "_" + writerId);
133 table = new HTable(conf, tableName);
134 }
135
136 public void run() {
137 try {
138 long rowKey;
139 while ((rowKey = nextKeyToInsert.getAndIncrement()) < endKey) {
140 long numColumns = minColumnsPerKey + Math.abs(random.nextLong())
141 % (maxColumnsPerKey - minColumnsPerKey);
142 numKeys.addAndGet(1);
143 if (isMultiPut) {
144 multiPutInsertKey(rowKey, 0, numColumns);
145 } else {
146 for (long col = 0; col < numColumns; ++col) {
147 insert(rowKey, col);
148 }
149 }
150 if (trackInsertedKeys) {
151 insertedKeys.add(rowKey);
152 }
153 }
154 } finally {
155 try {
156 table.close();
157 } catch (IOException e) {
158 LOG.error("Error closing table", e);
159 }
160 numThreadsWorking.decrementAndGet();
161 }
162 }
163
164 public void insert(long rowKey, long col) {
165 Put put = new Put(longToByteArrayKey(rowKey));
166 String colAsStr = String.valueOf(col);
167 put.add(columnFamily, Bytes.toBytes(colAsStr),
168 dataGenerator.generateRandomSizeValue(rowKey, colAsStr));
169 try {
170 long start = System.currentTimeMillis();
171 table.put(put);
172 numCols.addAndGet(1);
173 totalOpTimeMs.addAndGet(System.currentTimeMillis() - start);
174 } catch (IOException e) {
175 failedKeySet.add(rowKey);
176 LOG.error("Failed to insert: " + rowKey);
177 e.printStackTrace();
178 }
179 }
180
181 public void multiPutInsertKey(long rowKey, long startCol, long endCol) {
182 if (verbose) {
183 LOG.debug("Preparing put for key = " + rowKey + ", cols = ["
184 + startCol + ", " + endCol + ")");
185 }
186
187 if (startCol >= endCol) {
188 return;
189 }
190
191 Put put = new Put(LoadTestKVGenerator.md5PrefixedKey(
192 rowKey).getBytes());
193 byte[] columnQualifier;
194 byte[] value;
195 for (long i = startCol; i < endCol; ++i) {
196 String qualStr = String.valueOf(i);
197 columnQualifier = qualStr.getBytes();
198 value = dataGenerator.generateRandomSizeValue(rowKey, qualStr);
199 put.add(columnFamily, columnQualifier, value);
200 }
201
202 try {
203 long start = System.currentTimeMillis();
204 table.put(put);
205 numCols.addAndGet(endCol - startCol);
206 totalOpTimeMs.addAndGet(
207 System.currentTimeMillis() - start);
208 } catch (IOException e) {
209 failedKeySet.add(rowKey);
210 e.printStackTrace();
211 }
212 }
213 }
214
215
216
217
218
219 private class InsertedKeysTracker implements Runnable {
220
221 @Override
222 public void run() {
223 Thread.currentThread().setName(getClass().getSimpleName());
224 try {
225 long expectedKey = startKey;
226 Queue<Long> sortedKeys = new PriorityQueue<Long>();
227 while (expectedKey < endKey) {
228
229 Long k;
230 try {
231 k = insertedKeys.poll(1, TimeUnit.SECONDS);
232 } catch (InterruptedException e) {
233 LOG.info("Inserted key tracker thread interrupted", e);
234 break;
235 }
236 if (k == null) {
237 continue;
238 }
239 if (k == expectedKey) {
240
241 insertedUpToKey.set(k);
242 ++expectedKey;
243 } else {
244 sortedKeys.add(k);
245 }
246
247
248 while (!sortedKeys.isEmpty()
249 && ((k = sortedKeys.peek()) == expectedKey)) {
250 sortedKeys.poll();
251 insertedUpToKey.set(k);
252 ++expectedKey;
253 }
254
255 insertedKeyQueueSize.set(insertedKeys.size() + sortedKeys.size());
256 }
257 } catch (Exception ex) {
258 LOG.error("Error in inserted key tracker", ex);
259 } finally {
260 numThreadsWorking.decrementAndGet();
261 }
262 }
263
264 }
265
266 @Override
267 public void waitForFinish() {
268 super.waitForFinish();
269 System.out.println("Failed to write keys: " + failedKeySet.size());
270 for (Long key : failedKeySet) {
271 System.out.println("Failed to write key: " + key);
272 }
273 }
274
275 public int getNumWriteFailures() {
276 return failedKeySet.size();
277 }
278
279
280
281
282
283 public long insertedUpToKey() {
284 return insertedUpToKey.get();
285 }
286
287 public boolean failedToWriteKey(long k) {
288 return failedKeySet.contains(k);
289 }
290
291 @Override
292 protected String progressInfo() {
293 StringBuilder sb = new StringBuilder();
294 appendToStatus(sb, "insertedUpTo", insertedUpToKey.get());
295 appendToStatus(sb, "insertedQSize", insertedKeyQueueSize.get());
296 return sb.toString();
297 }
298
299
300
301
302
303
304 void setTrackInsertedKeys(boolean enable) {
305 trackInsertedKeys = enable;
306 }
307
308 }