-
Notifications
You must be signed in to change notification settings - Fork 52
/
ChangeLoader.java
482 lines (398 loc) · 13.1 KB
/
ChangeLoader.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
470
471
472
473
474
475
476
477
478
479
480
481
482
/**
* Copyright HashData. All Rights Reserved.
*/
package cn.hashdata.bireme;
import java.io.IOException;
import java.io.PipedInputStream;
import java.io.PipedOutputStream;
import java.sql.Connection;
import java.sql.ResultSet;
import java.sql.SQLException;
import java.util.ArrayList;
import java.util.HashSet;
import java.util.Iterator;
import java.util.List;
import java.util.Set;
import java.util.concurrent.Callable;
import java.util.concurrent.ExecutionException;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.Executors;
import java.util.concurrent.Future;
import java.util.concurrent.LinkedBlockingQueue;
import org.apache.commons.lang3.StringUtils;
import org.apache.logging.log4j.Logger;
import org.postgresql.copy.CopyManager;
import org.postgresql.core.BaseConnection;
import com.codahale.metrics.Timer;
import cn.hashdata.bireme.pipeline.PipeLine;
/**
* {@code ChangeLoader} poll tasks and load the tasks to database. Each {@code ChangeLoader}
* corresponds to a specific table in a {@code PipeLine}. All {@code ChangeLoaders} share
* connections to the database.
*
* @author yuze
*
*/
public class ChangeLoader implements Callable<Long> {
protected static final Long DELETE_TIMEOUT_NS = 10000000000L;
protected static final Long NANOSECONDS_TO_SECONDS = 1000000000L;
public Logger logger;
protected boolean optimisticMode = true;
protected Context cxt;
protected Config conf;
protected Connection conn;
protected LinkedBlockingQueue<Future<LoadTask>> taskIn;
protected Table table;
protected LoadTask currentTask;
protected ExecutorService copyThread;
public String mappedTable;
private Timer copyForDeleteTimer;
private Timer deleteTimer;
private Timer copyForInsertTimer;
private Timer.Context timerCTX;
/**
* Create a new {@code ChangeLoader}.
*
* @param cxt the Bireme Context
* @param pipeLine the {@code PipeLine} belongs to
* @param mappedTable the target table
* @param taskIn a queue to get {@code LoadTask}
*/
public ChangeLoader(Context cxt, PipeLine pipeLine, String mappedTable,
LinkedBlockingQueue<Future<LoadTask>> taskIn) {
this.cxt = cxt;
this.conf = cxt.conf;
this.conn = null;
this.mappedTable = mappedTable;
this.table = cxt.tablesInfo.get(mappedTable);
this.taskIn = taskIn;
this.copyThread = Executors.newFixedThreadPool(1);
// add statistics
Timer[] timers = pipeLine.stat.addTimerForLoader(mappedTable);
copyForDeleteTimer = timers[0];
deleteTimer = timers[1];
copyForInsertTimer = timers[2];
logger = pipeLine.logger;
}
/**
* Get the task and copy it to target database
*
* @throws BiremeException load exception
* @throws InterruptedException interrupted when load the task
* @return if normally end, return 0
*/
@Override
public Long call() throws BiremeException, InterruptedException {
while (!cxt.stop) {
// get task
if (currentTask == null) {
currentTask = pollTask();
}
if (currentTask == null) {
break;
}
// get connection
conn = getConnection();
if (conn == null) {
logger.debug("Unable to get Connection.");
break;
}
// Execute task and release connection. If failed, close the connection and abandon it.
try {
executeTask();
releaseConnection();
} catch (BiremeException e) {
logger.error("Fail to execute task. Message: {}", e.getMessage());
try {
conn.rollback();
conn.close();
} catch (Exception ignore) {
logger.error("Fail to roll back after load exception. Message: {}", e.getMessage());
throw e;
}
throw e;
} finally {
currentTask.destory();
currentTask = null;
conn = null;
}
}
return 0L;
}
/**
* Check whether {@code Rows} have been merged to a task. If done, poll the task and return.
*
* @return a task need be loaded to database
* @throws BiremeException merge task failed
* @throws InterruptedException if the current thread was interrupted while waiting
*/
protected LoadTask pollTask() throws BiremeException, InterruptedException {
LoadTask task = null;
Future<LoadTask> head = taskIn.peek();
if (head != null && head.isDone()) {
taskIn.remove();
try {
task = head.get();
} catch (ExecutionException e) {
throw new BiremeException("Merge task failed.\n", e.getCause());
}
}
return task;
}
/**
* Get connection to the destination database from connection pool.
*
* @return the connection
* @throws BiremeException when unable to create temporary table
*/
protected Connection getConnection() throws BiremeException {
Connection connection = cxt.loaderConnections.poll();
if (connection == null) {
String message = "Unable to get Connection.";
logger.fatal(message);
throw new BiremeException(message);
}
HashSet<String> temporaryTables = cxt.temporaryTables.get(connection);
if (!temporaryTables.contains(mappedTable)) {
createTemporaryTable(connection);
temporaryTables.add(mappedTable);
}
return connection;
}
/**
* Return the connection to connection pool.
*
*/
protected void releaseConnection() {
cxt.loaderConnections.offer(conn);
conn = null;
}
/**
* Load the task to destination database. First load the delete set and then load the insert set.
*
* @throws BiremeException Wrap the exception when load the task
* @throws InterruptedException if interrupted while waiting
*/
protected void executeTask() throws BiremeException, InterruptedException {
if (!currentTask.delete.isEmpty() || (!optimisticMode && !currentTask.insert.isEmpty())) {
int size = currentTask.delete.size();
if (!optimisticMode) {
currentTask.delete.addAll(currentTask.insert.keySet());
}
if (executeDelete(currentTask.delete) <= size && optimisticMode == false) {
optimisticMode = true;
logger.info("Chang to optimistic mode.");
}
// After delete, commit
try {
if (cxt.stop) {
conn.rollback();
return;
}
} catch (SQLException e) {
String message = cxt.stop ? "Rollback failed\n" : "Commit failed\n";
throw new BiremeException(message, e);
}
}
if (!currentTask.insert.isEmpty()) {
HashSet<String> insertSet = new HashSet<String>();
insertSet.addAll(currentTask.insert.values());
executeInsert(insertSet);
try {
if (cxt.stop) {
conn.rollback();
return;
}
} catch (SQLException e) {
String message = cxt.stop ? "Rollback failed" : "Commit failed";
throw new BiremeException(message, e);
}
}
try {
conn.commit();
} catch (SQLException e) {
String message = "commit failed.";
throw new BiremeException(message, e);
}
for (CommitCallback callback : currentTask.callbacks) {
callback.done();
}
}
private Long executeDelete(Set<String> delete) throws BiremeException, InterruptedException {
long deleteCounts;
ArrayList<String> keyNames = table.keyNames;
String temporaryTableName = getTemporaryTableName();
timerCTX = copyForDeleteTimer.time();
copyWorker(temporaryTableName, keyNames, delete);
timerCTX.stop();
if (cxt.stop) {
return 0L;
}
timerCTX = deleteTimer.time();
deleteCounts = deleteWorker(mappedTable, temporaryTableName, keyNames);
long deleteTime = timerCTX.stop();
if (deleteTime > DELETE_TIMEOUT_NS) {
String plan = deletePlan(mappedTable, temporaryTableName, keyNames);
logger.warn("Delete operation takes {} seconds, delete plan:\n {}",
deleteTime / NANOSECONDS_TO_SECONDS, plan);
}
return deleteCounts;
}
private void executeInsert(Set<String> insertSet) throws BiremeException, InterruptedException {
ArrayList<String> columnList = table.columnName;
timerCTX = copyForInsertTimer.time();
try {
copyWorker(mappedTable, columnList, insertSet);
} catch (BiremeException e) {
if (e.getCause().getMessage().contains("duplicate key value") && optimisticMode) {
try {
conn.rollback();
} catch (SQLException ignore) {
}
optimisticMode = false;
logger.info("Chang to passimistic mode.");
executeDelete(currentTask.insert.keySet());
executeInsert(insertSet);
} else {
throw e;
}
}
timerCTX.stop();
}
private Long copyWorker(String tableName, ArrayList<String> columnList, Set<String> tuples)
throws BiremeException, InterruptedException {
Future<Long> copyResult;
long copyCount = -1L;
PipedOutputStream pipeOut = new PipedOutputStream();
PipedInputStream pipeIn = null;
try {
pipeIn = new PipedInputStream(pipeOut);
} catch (IOException e) {
throw new BiremeException("I/O error occurs while create PipedInputStream.", e);
}
String sql = getCopySql(tableName, columnList);
copyResult = copyThread.submit(new TupleCopyer(pipeIn, sql, conn));
tupleWriter(pipeOut, tuples);
try {
while (!copyResult.isDone() && !cxt.stop) {
Thread.sleep(1);
}
if (cxt.stop) {
return 0L;
}
copyCount = copyResult.get();
} catch (ExecutionException e) {
throw new BiremeException("Copy failed.", e.getCause());
}
return copyCount;
}
private String getCopySql(String tableName, List<String> columnList) {
StringBuilder sb =
new StringBuilder()
.append("COPY ")
.append(tableName)
.append(" (")
.append(StringUtils.join(columnList, ","))
.append(") FROM STDIN WITH DELIMITER '|' NULL '' CSV QUOTE '\"' ESCAPE E'\\\\';");
String sql = sb.toString();
return sql;
}
private Long deleteWorker(String table, String tmpTable, ArrayList<String> columnList)
throws BiremeException {
StringBuilder sb = new StringBuilder();
Long count = 0L;
for (int i = 0; i < columnList.size(); i++) {
if (i != 0) {
sb.append(" and ");
}
sb.append(table + "." + columnList.get(i) + "=" + tmpTable + "." + columnList.get(i));
}
String sql = "DELETE FROM " + table + " WHERE EXISTS (SELECT 1 FROM " + tmpTable + " WHERE "
+ sb.toString() + ");";
try {
count = (long) conn.createStatement().executeUpdate(sql);
} catch (SQLException e) {
throw new BiremeException("Delete failed.", e);
}
return count;
}
private String deletePlan(String table, String tmpTable, ArrayList<String> columnList)
throws BiremeException {
StringBuilder sb = new StringBuilder();
for (int i = 0; i < columnList.size(); i++) {
if (i != 0) {
sb.append(" and ");
}
sb.append(table + "." + columnList.get(i) + "=" + tmpTable + "." + columnList.get(i));
}
String sql = "EXPLAIN DELETE FROM " + table + " WHERE EXISTS (SELECT 1 FROM " + tmpTable
+ " WHERE " + sb.toString() + ");";
try {
ResultSet rs = conn.createStatement().executeQuery(sql);
if (!rs.wasNull()) {
sb.setLength(0);
while (rs.next()) {
sb.append(rs.getString(1) + "\n");
}
return sb.toString();
} else {
return "Can not get plan.";
}
} catch (SQLException e) {
throw new BiremeException("Fail to get delete plan.", e);
}
}
private class TupleCopyer implements Callable<Long> {
PipedInputStream pipeIn;
String sql;
Connection conn;
public TupleCopyer(PipedInputStream pipeIn, String sql, Connection conn) {
this.pipeIn = pipeIn;
this.sql = sql;
this.conn = conn;
}
@Override
public Long call() throws SQLException, IOException {
try {
CopyManager mgr = new CopyManager((BaseConnection) conn);
return mgr.copyIn(sql, pipeIn);
} finally {
try {
pipeIn.close();
} catch (IOException ignore) {
}
}
}
}
private void tupleWriter(PipedOutputStream pipeOut, Set<String> tuples) throws BiremeException {
byte[] data = null;
try {
Iterator<String> iterator = tuples.iterator();
while (iterator.hasNext() && !cxt.stop) {
data = iterator.next().getBytes("UTF-8");
pipeOut.write(data);
}
pipeOut.flush();
} catch (IOException e) {
throw new BiremeException("I/O error occurs while write to pipe.", e);
} finally {
try {
pipeOut.close();
} catch (IOException ignore) {
}
}
}
private String getTemporaryTableName() {
return mappedTable.replace('.', '_');
}
private void createTemporaryTable(Connection conn) throws BiremeException {
String sql = "CREATE TEMP TABLE " + getTemporaryTableName()
+ " ON COMMIT DELETE ROWS AS SELECT * FROM " + mappedTable + " LIMIT 0;";
try {
conn.createStatement().executeUpdate(sql);
conn.commit();
} catch (SQLException e) {
throw new BiremeException("Fail to create tmporary table.", e);
}
}
}