-
Notifications
You must be signed in to change notification settings - Fork 52
/
PipeLine.java
469 lines (391 loc) · 13.5 KB
/
PipeLine.java
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
281
282
283
284
285
286
287
288
289
290
291
292
293
294
295
296
297
298
299
300
301
302
303
304
305
306
307
308
309
310
311
312
313
314
315
316
317
318
319
320
321
322
323
324
325
326
327
328
329
330
331
332
333
334
335
336
337
338
339
340
341
342
343
344
345
346
347
348
349
350
351
352
353
354
355
356
357
358
359
360
361
362
363
364
365
366
367
368
369
370
371
372
373
374
375
376
377
378
379
380
381
382
383
384
385
386
387
388
389
390
391
392
393
394
395
396
397
398
399
400
401
402
403
404
405
406
407
408
409
410
411
412
413
414
415
416
417
418
419
420
421
422
423
424
425
426
427
428
429
430
431
432
433
434
435
436
437
438
439
440
441
442
443
444
445
446
447
448
449
450
451
452
453
454
455
456
457
458
459
460
461
462
463
464
465
466
467
468
469
package cn.hashdata.bireme.pipeline;
import java.sql.Types;
import java.util.ArrayList;
import java.util.HashMap;
import java.util.LinkedList;
import java.util.concurrent.Callable;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.Future;
import java.util.concurrent.LinkedBlockingQueue;
import org.apache.logging.log4j.Logger;
import cn.hashdata.bireme.BiremeException;
import cn.hashdata.bireme.ChangeSet;
import cn.hashdata.bireme.Context;
import cn.hashdata.bireme.Dispatcher;
import cn.hashdata.bireme.PipeLineStat;
import cn.hashdata.bireme.Record;
import cn.hashdata.bireme.Row;
import cn.hashdata.bireme.RowCache;
import cn.hashdata.bireme.RowSet;
import cn.hashdata.bireme.Table;
/**
* {@code PipeLine} is a bridge between data source and target table. The data flow order is
* guaranteed. A {@code PipeLine} does four things as follows:
* <ul>
* <li>Poll data and allocate {@link Transformer} to convert the data.</li>
* <li>Dispatch the transformed data and insert it into {@link RowCache}.</li>
* <li>Drive the {@code RowBatchMerger} to work</li>
* <li>Drivet the {@code ChangeLoader} to work</li>
* </ul>
*
* @author yuze
*
*/
public abstract class PipeLine implements Callable<PipeLine> {
public enum PipeLineState { NORMAL, ERROR, STOP }
public Logger logger;
public String myName;
public volatile PipeLineState state;
public BiremeException e;
public PipeLineStat stat;
public Context cxt;
public SourceConfig conf;
public LinkedBlockingQueue<Future<RowSet>> transResult;
private LinkedList<Transformer> localTransformer;
private Dispatcher dispatcher;
public ConcurrentHashMap<String, RowCache> cache;
public PipeLine(Context cxt, SourceConfig conf, String myName) {
this.myName = myName;
this.state = PipeLineState.NORMAL;
this.e = null;
this.cxt = cxt;
this.conf = conf;
int queueSize = cxt.conf.transform_queue_size;
transResult = new LinkedBlockingQueue<Future<RowSet>>(queueSize);
localTransformer = new LinkedList<Transformer>();
cache = new ConcurrentHashMap<String, RowCache>();
dispatcher = new Dispatcher(cxt, this);
for (int i = 0; i < queueSize; i++) {
localTransformer.add(createTransformer());
}
// initialize statistics
this.stat = new PipeLineStat(this);
}
@Override
public PipeLine call() {
// Poll data and start transformer
while (transResult.remainingCapacity() != 0) {
ChangeSet changeSet = null;
try {
changeSet = pollChangeSet();
} catch (BiremeException e) {
state = PipeLineState.ERROR;
this.e = e;
logger.error("Poll change set failed. Message: {}", e.getMessage());
logger.error("Stack Trace: ", e);
return this;
}
if (changeSet == null) {
break;
}
Transformer trans = localTransformer.remove();
trans.setChangeSet(changeSet);
startTransform(trans);
localTransformer.add(trans);
}
// Start dispatcher, only one dispatcher for each pipeline
try {
dispatcher.dispatch();
} catch (BiremeException e) {
state = PipeLineState.ERROR;
this.e = e;
logger.error("Dispatch failed. Message: {}", e.getMessage());
logger.error("Stack Trace: ", e);
return this;
} catch (InterruptedException e) {
state = PipeLineState.ERROR;
this.e = new BiremeException("Dispatcher failed, be interrupted", e);
logger.info("Interrupted when getting transform result. Message: {}.", e.getMessage());
logger.info("Stack Trace: ", e);
return this;
}
// Start merger
for (RowCache rowCache : cache.values()) {
if (rowCache.shouldMerge()) {
rowCache.startMerge();
}
try {
rowCache.startLoad();
} catch (BiremeException e) {
state = PipeLineState.ERROR;
this.e = e;
logger.info("Loader for {} failed. Message: {}.", rowCache.tableName, e.getMessage());
logger.info("Stack Trace: ", e);
return this;
} catch (InterruptedException e) {
state = PipeLineState.ERROR;
this.e = new BiremeException("Get Future<Long> failed, be interrupted", e);
logger.info("Interrupted when getting loader result for {}. Message: {}.",
rowCache.tableName, e.getMessage());
logger.info("Stack Trace: ", e);
return this;
}
}
// Commit result
checkAndCommit();
return this;
}
/**
* Poll a set of change data from source and pack it to {@link ChangeSet}.
*
* @return a packed change set
* @throws BiremeException Exceptions when poll data from source
*/
public abstract ChangeSet pollChangeSet() throws BiremeException;
/**
* Check whether the loading operation is complete. If true, commit it.
*
*/
public abstract void checkAndCommit();
/**
* Create a new {@link Transformer} to work parallel.
*
* @return a new {@code Transformer}
*/
public abstract Transformer createTransformer();
private void startTransform(Transformer trans) {
ExecutorService transformerPool = cxt.transformerPool;
Future<RowSet> result = transformerPool.submit(trans);
transResult.add(result);
}
/**
* Get the unique name for the {@code PipeLine}.
*
* @return the name for the {@code PipeLine}
*/
public String getPipeLineName() {
return conf.name;
}
/**
* {@code Transformer} convert a group of change data to unified form {@link Row}.
*
* @author yuze
*
*/
public abstract class Transformer implements Callable<RowSet> {
private static final char FIELD_DELIMITER = '|';
private static final char NEWLINE = '\n';
private static final char QUOTE = '"';
private static final char ESCAPE = '\\';
public ChangeSet changeSet;
public StringBuilder tupleStringBuilder;
public StringBuilder fieldStringBuilder;
public Transformer() {
tupleStringBuilder = new StringBuilder();
fieldStringBuilder = new StringBuilder();
}
/**
* Borrow an empty {@code RowSet} and write the data acquired from {@code ChangeSet} to the
* {@code RowSet}. Finally, return the filled {@code RowSet}.
*
* @throws BiremeException when unable to transform the recoed
*/
@Override
public RowSet call() throws BiremeException {
RowSet rowSet = new RowSet();
fillRowSet(rowSet);
changeSet.destory();
changeSet = null;
return rowSet;
}
/**
* Format the change data into csv tuple, which is then loaded to database by COPY.
*
* @param record contain change data polled by {@code Provider}.
* @param table metadata of the target table
* @param columns the indexes of columns to assemble a csv tuple
* @param oldValue only for update operation when primary key was updated, we need to get the
* old key and delete the old tuple
* @return the csv tuple in string
* @throws BiremeException when can not get the field value
*/
protected String formatColumns(Record record, Table table, ArrayList<String> columns,
boolean oldValue) throws BiremeException {
tupleStringBuilder.setLength(0);
for (int i = 0; i < columns.size(); ++i) {
String columnName = columns.get(i);
int sqlType = table.columnType.get(columnName);
String data = null;
data = record.getField(columnName, oldValue);
if (data != null) {
switch (sqlType) {
case Types.CHAR:
case Types.NCHAR:
case Types.VARCHAR:
case Types.LONGVARCHAR:
case Types.NVARCHAR:
case Types.LONGNVARCHAR: {
tupleStringBuilder.append(QUOTE);
tupleStringBuilder.append(escapeString(data));
tupleStringBuilder.append(QUOTE);
break;
}
case Types.BINARY:
case Types.BLOB:
case Types.CLOB:
case Types.LONGVARBINARY:
case Types.NCLOB:
case Types.VARBINARY: {
byte[] decoded = null;
decoded = decodeToBinary(data);
tupleStringBuilder.append(escapeBinary(decoded));
break;
}
case Types.BIT: {
int precision = table.columnPrecision.get(columnName);
tupleStringBuilder.append(decodeToBit(data, precision));
break;
}
case Types.DATE:
case Types.TIME:
case Types.TIMESTAMP: {
int scale = table.columnScale.get(columnName);
String time = decodeToTime(data, sqlType, scale);
tupleStringBuilder.append(time);
break;
}
case Types.DECIMAL:
case Types.NUMERIC: {
int scale = table.columnScale.get(columnName);
String numeric = decodeToNumeric(data, sqlType, scale);
tupleStringBuilder.append(numeric);
break;
}
default: {
tupleStringBuilder.append(data);
break;
}
}
}
if (i + 1 < columns.size()) {
tupleStringBuilder.append(FIELD_DELIMITER);
}
}
tupleStringBuilder.append(NEWLINE);
return tupleStringBuilder.toString();
}
/**
* For binary type, {@code Transformer} need to decode the extracted string and transform it to
* origin binary.
*
* @param data the encoded string
* @return the array of byte, decode result
*/
protected abstract byte[] decodeToBinary(String data);
/**
* For bit type, {@code Transformer} need to decode the extracted string and transform it to
* origin bit.
*
* @param data the encoded string
* @param precision the length of the bit field, acquired from the table's metadata
* @return the string of 1 or 0
*/
protected abstract String decodeToBit(String data, int precision);
/**
* For Date/Time type, {@code Transformer} need to decode the extracted string and transform it
* to origin Date/Time string.
*
* @param data the encoded string from provider
* @param sqlType particular type of this field, such as Time, Date
* @param precision specifies the number of fractional digits retained in the seconds field
* @return the Date/Time format
*/
protected String decodeToTime(String data, int sqlType, int precision) {
return data;
};
/**
* For Numeric type, {@code Transformer} need to decode the extracted string and transform it to
* origin Numeric in String.
*
* @param data the value from provider
* @param sqlType particular type of this field
* @param precision the count of decimal digits in the fractional part
* @return the numeric number in String
*/
protected String decodeToNumeric(String data, int sqlType, int precision) {
return data;
};
/**
* Add escape character to a data string.
*
* @param data the origin string
* @return the modified string
*/
protected String escapeString(String data) {
fieldStringBuilder.setLength(0);
for (int i = 0; i < data.length(); ++i) {
char c = data.charAt(i);
switch (c) {
case 0x00:
logger.warn("illegal character 0x00, deleted.");
continue;
case QUOTE:
case ESCAPE:
fieldStringBuilder.append(ESCAPE);
}
fieldStringBuilder.append(c);
}
return fieldStringBuilder.toString();
}
/**
* Encode the binary data into string for COPY into target database.
*
* @param data the origin binary data
* @return the encoded string
*/
protected String escapeBinary(byte[] data) {
fieldStringBuilder.setLength(0);
for (int i = 0; i < data.length; ++i) {
if (data[i] == '\\') {
fieldStringBuilder.append('\\');
fieldStringBuilder.append('\\');
} else if (data[i] < 0x20 || data[i] > 0x7e) {
byte b = data[i];
char[] val = new char[3];
val[2] = (char) ((b & 07) + '0');
b >>= 3;
val[1] = (char) ((b & 07) + '0');
b >>= 3;
val[0] = (char) ((b & 03) + '0');
fieldStringBuilder.append('\\');
fieldStringBuilder.append(val);
} else {
fieldStringBuilder.append((char) (data[i]));
}
}
return fieldStringBuilder.toString();
}
/**
* Appoint a {@code ChangeSet} to the {@code Transformer}
*
* @param changeSet a package of change data
*/
public void setChangeSet(ChangeSet changeSet) {
this.changeSet = changeSet;
}
/**
* Write the change data into a {@code RowSet}.
*
* @param rowSet a empty {@code RowSet} to store change data
* @throws BiremeException Exceptions when fill the {@code RowSet}
*/
public abstract void fillRowSet(RowSet rowSet) throws BiremeException;
/**
* After convert a single change data to a {@code Row}, insert into the {@code RowSet}.
*
* @param row the converted change data
* @param rowSet the {@code RowSet} to organize the {@code Row}
*/
public void addToRowSet(Row row, RowSet rowSet) {
HashMap<String, ArrayList<Row>> bucket = rowSet.rowBucket;
String mappedTable = row.mappedTable;
ArrayList<Row> array = bucket.get(mappedTable);
if (array == null) {
array = new ArrayList<Row>();
bucket.put(mappedTable, array);
}
array.add(row);
}
}
}