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.Set;
24 import java.util.concurrent.ArrayBlockingQueue;
25 import java.util.concurrent.BlockingQueue;
26 import java.util.concurrent.ConcurrentSkipListSet;
27 import java.util.concurrent.TimeUnit;
28 import java.util.concurrent.atomic.AtomicLong;
29
30 import org.apache.commons.logging.Log;
31 import org.apache.commons.logging.LogFactory;
32 import org.apache.hadoop.conf.Configuration;
33 import org.apache.hadoop.hbase.client.HTable;
34 import org.apache.hadoop.hbase.client.Put;
35
36
37 public class MultiThreadedWriter extends MultiThreadedAction {
38 private static final Log LOG = LogFactory.getLog(MultiThreadedWriter.class);
39
40 private Set<HBaseWriterThread> writers = new HashSet<HBaseWriterThread>();
41
42 private boolean isMultiPut = false;
43
44
45
46
47
48
49
50 private BlockingQueue<Long> insertedKeys = new ArrayBlockingQueue<Long>(10000);
51
52
53
54
55
56 private AtomicLong nextKeyToInsert = new AtomicLong();
57
58
59
60
61 private AtomicLong insertedUpToKey = new AtomicLong();
62
63
64 private Set<Long> failedKeySet = new ConcurrentSkipListSet<Long>();
65
66
67
68
69
70
71 private AtomicLong insertedKeyQueueSize = new AtomicLong();
72
73
74 private boolean trackInsertedKeys;
75
76 public MultiThreadedWriter(LoadTestDataGenerator dataGen, Configuration conf,
77 byte[] tableName) {
78 super(dataGen, conf, tableName, "W");
79 }
80
81
82 public void setMultiPut(boolean isMultiPut) {
83 this.isMultiPut = isMultiPut;
84 }
85
86 @Override
87 public void start(long startKey, long endKey, int numThreads)
88 throws IOException {
89 super.start(startKey, endKey, numThreads);
90
91 if (verbose) {
92 LOG.debug("Inserting keys [" + startKey + ", " + endKey + ")");
93 }
94
95 nextKeyToInsert.set(startKey);
96 insertedUpToKey.set(startKey - 1);
97
98 for (int i = 0; i < numThreads; ++i) {
99 HBaseWriterThread writer = new HBaseWriterThread(i);
100 writers.add(writer);
101 }
102
103 if (trackInsertedKeys) {
104 new Thread(new InsertedKeysTracker()).start();
105 numThreadsWorking.incrementAndGet();
106 }
107
108 startThreads(writers);
109 }
110
111 private class HBaseWriterThread extends Thread {
112 private final HTable table;
113
114 public HBaseWriterThread(int writerId) throws IOException {
115 setName(getClass().getSimpleName() + "_" + writerId);
116 table = new HTable(conf, tableName);
117 }
118
119 public void run() {
120 try {
121 long rowKeyBase;
122 byte[][] columnFamilies = dataGenerator.getColumnFamilies();
123 while ((rowKeyBase = nextKeyToInsert.getAndIncrement()) < endKey) {
124 byte[] rowKey = dataGenerator.getDeterministicUniqueKey(rowKeyBase);
125 Put put = new Put(rowKey);
126 numKeys.addAndGet(1);
127 int columnCount = 0;
128 for (byte[] cf : columnFamilies) {
129 byte[][] columns = dataGenerator.generateColumnsForCf(rowKey, cf);
130 for (byte[] column : columns) {
131 byte[] value = dataGenerator.generateValue(rowKey, cf, column);
132 put.add(cf, column, value);
133 ++columnCount;
134 if (!isMultiPut) {
135 insert(table, put, rowKeyBase);
136 numCols.addAndGet(1);
137 put = new Put(rowKey);
138 }
139 }
140 }
141 if (isMultiPut) {
142 if (verbose) {
143 LOG.debug("Preparing put for key = [" + rowKey + "], " + columnCount + " columns");
144 }
145 insert(table, put, rowKeyBase);
146 numCols.addAndGet(columnCount);
147 }
148 if (trackInsertedKeys) {
149 insertedKeys.add(rowKeyBase);
150 }
151 }
152 } finally {
153 try {
154 table.close();
155 } catch (IOException e) {
156 LOG.error("Error closing table", e);
157 }
158 numThreadsWorking.decrementAndGet();
159 }
160 }
161 }
162
163 public void insert(HTable table, Put put, long keyBase) {
164 try {
165 long start = System.currentTimeMillis();
166 table.put(put);
167 totalOpTimeMs.addAndGet(System.currentTimeMillis() - start);
168 } catch (IOException e) {
169 failedKeySet.add(keyBase);
170 LOG.error("Failed to insert: " + keyBase);
171 e.printStackTrace();
172 }
173 }
174
175
176
177
178
179 private class InsertedKeysTracker implements Runnable {
180
181 @Override
182 public void run() {
183 Thread.currentThread().setName(getClass().getSimpleName());
184 try {
185 long expectedKey = startKey;
186 Queue<Long> sortedKeys = new PriorityQueue<Long>();
187 while (expectedKey < endKey) {
188
189 Long k;
190 try {
191 k = insertedKeys.poll(1, TimeUnit.SECONDS);
192 } catch (InterruptedException e) {
193 LOG.info("Inserted key tracker thread interrupted", e);
194 break;
195 }
196 if (k == null) {
197 continue;
198 }
199 if (k == expectedKey) {
200
201 insertedUpToKey.set(k);
202 ++expectedKey;
203 } else {
204 sortedKeys.add(k);
205 }
206
207
208 while (!sortedKeys.isEmpty()
209 && ((k = sortedKeys.peek()) == expectedKey)) {
210 sortedKeys.poll();
211 insertedUpToKey.set(k);
212 ++expectedKey;
213 }
214
215 insertedKeyQueueSize.set(insertedKeys.size() + sortedKeys.size());
216 }
217 } catch (Exception ex) {
218 LOG.error("Error in inserted key tracker", ex);
219 } finally {
220 numThreadsWorking.decrementAndGet();
221 }
222 }
223
224 }
225
226 @Override
227 public void waitForFinish() {
228 super.waitForFinish();
229 System.out.println("Failed to write keys: " + failedKeySet.size());
230 for (Long key : failedKeySet) {
231 System.out.println("Failed to write key: " + key);
232 }
233 }
234
235 public int getNumWriteFailures() {
236 return failedKeySet.size();
237 }
238
239
240
241
242
243 public long insertedUpToKey() {
244 return insertedUpToKey.get();
245 }
246
247 public boolean failedToWriteKey(long k) {
248 return failedKeySet.contains(k);
249 }
250
251 @Override
252 protected String progressInfo() {
253 StringBuilder sb = new StringBuilder();
254 appendToStatus(sb, "insertedUpTo", insertedUpToKey.get());
255 appendToStatus(sb, "insertedQSize", insertedKeyQueueSize.get());
256 return sb.toString();
257 }
258
259
260
261
262
263
264 public void setTrackInsertedKeys(boolean enable) {
265 trackInsertedKeys = enable;
266 }
267
268 }