Skip to content

Commit 4f189b6

Browse files
committed
[feat-34672][kudu] 优化kudu三种模式写入异常处理
1 parent d2feeab commit 4f189b6

File tree

6 files changed

+52
-68
lines changed

6 files changed

+52
-68
lines changed

kudu/kudu-side/kudu-side-core/src/main/java/com/dtstack/flink/sql/side/kudu/table/KuduSideParser.java

Lines changed: 0 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -23,8 +23,6 @@ public class KuduSideParser extends AbstractSideTableParser {
2323

2424
public static final String OPERATION_TIMEOUT_MS = "defaultOperationTimeoutMs";
2525

26-
public static final String SOCKET_READ_TIMEOUT_MS = "defaultSocketReadTimeoutMs";
27-
2826
/**
2927
* 查询返回的最大字节数
3028
*/
@@ -64,7 +62,6 @@ public AbstractTableInfo getTableInfo(String tableName, String fieldsInfo, Map<S
6462
kuduSideTableInfo.setTableName(MathUtil.getString(props.get(TABLE_NAME.toLowerCase())));
6563
kuduSideTableInfo.setWorkerCount(MathUtil.getIntegerVal(props.get(WORKER_COUNT.toLowerCase())));
6664
kuduSideTableInfo.setDefaultOperationTimeoutMs(MathUtil.getIntegerVal(props.get(OPERATION_TIMEOUT_MS.toLowerCase())));
67-
kuduSideTableInfo.setDefaultSocketReadTimeoutMs(MathUtil.getIntegerVal(props.get(SOCKET_READ_TIMEOUT_MS.toLowerCase())));
6865
kuduSideTableInfo.setBatchSizeBytes(MathUtil.getIntegerVal(props.get(BATCH_SIZE_BYTES.toLowerCase())));
6966
kuduSideTableInfo.setLimitNum(MathUtil.getLongVal(props.get(LIMIT_NUM.toLowerCase())));
7067
kuduSideTableInfo.setFaultTolerant(MathUtil.getBoolean(props.get(IS_FAULT_TO_LERANT.toLowerCase())));

kudu/kudu-side/kudu-side-core/src/main/java/com/dtstack/flink/sql/side/kudu/table/KuduSideTableInfo.java

Lines changed: 0 additions & 9 deletions
Original file line numberDiff line numberDiff line change
@@ -18,7 +18,6 @@ public class KuduSideTableInfo extends AbstractSideTableInfo implements Kerberos
1818

1919
private Integer defaultOperationTimeoutMs;
2020

21-
private Integer defaultSocketReadTimeoutMs;
2221
/**
2322
* 查询返回的最大字节数
2423
*/
@@ -91,14 +90,6 @@ public void setDefaultOperationTimeoutMs(Integer defaultOperationTimeoutMs) {
9190
this.defaultOperationTimeoutMs = defaultOperationTimeoutMs;
9291
}
9392

94-
public Integer getDefaultSocketReadTimeoutMs() {
95-
return defaultSocketReadTimeoutMs;
96-
}
97-
98-
public void setDefaultSocketReadTimeoutMs(Integer defaultSocketReadTimeoutMs) {
99-
this.defaultSocketReadTimeoutMs = defaultSocketReadTimeoutMs;
100-
}
101-
10293
public Integer getBatchSizeBytes() {
10394
return batchSizeBytes;
10495
}

kudu/kudu-sink/src/main/java/com/dtstack/flink/sql/sink/kudu/KuduOutputFormat.java

Lines changed: 50 additions & 36 deletions
Original file line numberDiff line numberDiff line change
@@ -32,7 +32,9 @@
3232
import org.apache.kudu.client.KuduSession;
3333
import org.apache.kudu.client.KuduTable;
3434
import org.apache.kudu.client.Operation;
35+
import org.apache.kudu.client.OperationResponse;
3536
import org.apache.kudu.client.PartialRow;
37+
import org.apache.kudu.client.RowError;
3638
import org.apache.kudu.client.SessionConfiguration;
3739
import org.slf4j.Logger;
3840
import org.slf4j.LoggerFactory;
@@ -74,8 +76,6 @@ public class KuduOutputFormat extends AbstractDtRichOutputFormat<Tuple2<Boolean,
7476

7577
private Integer defaultOperationTimeoutMs;
7678

77-
private Integer defaultSocketReadTimeoutMs;
78-
7979
/**
8080
* kerberos
8181
*/
@@ -133,12 +133,7 @@ private void initSchedulerTask() {
133133
);
134134

135135
this.scheduledFuture = this.scheduler.scheduleWithFixedDelay(
136-
() -> {
137-
synchronized (KuduOutputFormat.this) {
138-
flush();
139-
}
140-
}, batchWaitInterval, batchWaitInterval, TimeUnit.MILLISECONDS
141-
);
136+
this::flush, batchWaitInterval, batchWaitInterval, TimeUnit.MILLISECONDS);
142137
}
143138
} catch (Exception e) {
144139
LOG.error("init schedule task failed !");
@@ -151,9 +146,6 @@ private void establishConnection() throws IOException {
151146
if (null != workerCount) {
152147
kuduClientBuilder.workerCount(workerCount);
153148
}
154-
if (null != defaultSocketReadTimeoutMs) {
155-
kuduClientBuilder.defaultSocketReadTimeoutMs(defaultSocketReadTimeoutMs);
156-
}
157149

158150
if (null != defaultOperationTimeoutMs) {
159151
kuduClientBuilder.defaultOperationTimeoutMs(defaultOperationTimeoutMs);
@@ -184,14 +176,13 @@ private void establishConnection() throws IOException {
184176
}
185177

186178
/**
187-
* According to the different flush mode, construct different session. Detail see {@link SessionConfiguration.FlushMode}
179+
* According to the different flush mode, build different session. Detail see {@link SessionConfiguration.FlushMode}
188180
*
189181
* @param flushMode flush mode
190182
* @param kuduClient kudu client
191183
* @return KuduSession with flush mode
192-
* @throws KuduException kudu exception when session flush
193184
*/
194-
private KuduSession buildSessionWithFlushMode(String flushMode, KuduClient kuduClient) throws KuduException {
185+
private KuduSession buildSessionWithFlushMode(String flushMode, KuduClient kuduClient) {
195186
KuduSession kuduSession = kuduClient.newSession();
196187
if (flushMode.equalsIgnoreCase(KuduTableInfo.KuduFlushMode.MANUAL_FLUSH.name())) {
197188
kuduSession.setFlushMode(SessionConfiguration.FlushMode.MANUAL_FLUSH);
@@ -220,14 +211,6 @@ public void writeRecord(Tuple2<Boolean, Row> record) throws IOException {
220211
return;
221212
}
222213
Row row = record.getField(1);
223-
if (row.getArity() != fieldNames.length) {
224-
if (outDirtyRecords.getCount() % DIRTY_PRINT_FREQUENCY == 0) {
225-
LOG.error("record insert failed ..{}", row.toString());
226-
LOG.error("cause by row.getArity() != fieldNames.length");
227-
}
228-
outDirtyRecords.inc();
229-
return;
230-
}
231214

232215
try {
233216
if (outRecords.getCount() % ROW_PRINT_FREQUENCY == 0) {
@@ -236,33 +219,69 @@ public void writeRecord(Tuple2<Boolean, Row> record) throws IOException {
236219
if (rowCount.getAndIncrement() >= batchSize) {
237220
flush();
238221
}
222+
// At AUTO_FLUSH_SYNC mode, kudu automatically flush once session apply operation, then get the response from kudu server.
223+
if (flushMode.equalsIgnoreCase(SessionConfiguration.FlushMode.AUTO_FLUSH_SYNC.name())) {
224+
dealResponse(session.apply(toOperation(writeMode, row)));
225+
}
226+
239227
session.apply(toOperation(writeMode, row));
240228
outRecords.inc();
241229
} catch (KuduException e) {
242-
if (outDirtyRecords.getCount() % DIRTY_PRINT_FREQUENCY == 0) {
243-
LOG.error("record insert failed, total dirty record:{} current row:{}", outDirtyRecords.getCount(), row.toString());
244-
LOG.error("", e);
245-
}
246-
outDirtyRecords.inc();
230+
throw new RuntimeException(e);
247231
}
248232
}
249233

234+
/**
235+
* Flush data with session, then deal the responses of operations and reset rowCount.
236+
* Detail of flush see {@link KuduSession#flush()}
237+
*/
250238
private synchronized void flush() {
251239
try {
252240
if (session.isClosed()) {
253-
throw new IllegalStateException("session is closed! flush data error!");
241+
throw new IllegalStateException("Session is closed! Flush data error!");
254242
}
255243

256-
session.flush();
244+
// At AUTO_FLUSH_SYNC mode, kudu automatically flush once session apply operation
245+
if (flushMode.equalsIgnoreCase(SessionConfiguration.FlushMode.AUTO_FLUSH_SYNC.name())) {
246+
return;
247+
}
248+
session.flush().forEach(this::dealResponse);
257249
// clear
258250
rowCount.set(0);
259251
} catch (KuduException kuduException) {
260-
LOG.error(
261-
"flush data error!", kuduException);
252+
LOG.error("flush data error!", kuduException);
262253
throw new RuntimeException(kuduException);
263254
}
264255
}
265256

257+
/**
258+
* Deal response when operation apply.
259+
* At MANUAL_FLUSH mode, response returns after {@link KuduSession#flush()}.
260+
* But at AUTO_FLUSH_SYNC mode, response returns after {@link KuduSession#apply(Operation)}
261+
*
262+
* @param response {@link OperationResponse} response after operation done.
263+
*/
264+
private void dealResponse(OperationResponse response) {
265+
if (response.hasRowError()) {
266+
RowError error = response.getRowError();
267+
String errorMsg = error.getErrorStatus().toString();
268+
if (outDirtyRecords.getCount() % DIRTY_PRINT_FREQUENCY == 0) {
269+
LOG.error(errorMsg);
270+
LOG.error(String.format("Dirty data count: [%s]. Row data: [%s]",
271+
outDirtyRecords.getCount() + 1, error.getOperation().getRow().toString()));
272+
}
273+
outDirtyRecords.inc();
274+
275+
if (error.getErrorStatus().isNotFound()
276+
|| error.getErrorStatus().isIOError()
277+
|| error.getErrorStatus().isRuntimeError()
278+
|| error.getErrorStatus().isServiceUnavailable()
279+
|| error.getErrorStatus().isIllegalState()) {
280+
throw new RuntimeException(errorMsg);
281+
}
282+
}
283+
}
284+
266285
@Override
267286
public void close() {
268287
if (Objects.nonNull(session) && !session.isClosed()) {
@@ -429,11 +448,6 @@ public KuduOutputFormatBuilder setDefaultOperationTimeoutMs(Integer defaultOpera
429448
return this;
430449
}
431450

432-
public KuduOutputFormatBuilder setDefaultSocketReadTimeoutMs(Integer defaultSocketReadTimeoutMs) {
433-
kuduOutputFormat.defaultSocketReadTimeoutMs = defaultSocketReadTimeoutMs;
434-
return this;
435-
}
436-
437451
public KuduOutputFormatBuilder setPrincipal(String principal) {
438452
kuduOutputFormat.principal = principal;
439453
return this;

kudu/kudu-sink/src/main/java/com/dtstack/flink/sql/sink/kudu/KuduSink.java

Lines changed: 0 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -29,7 +29,6 @@ public class KuduSink implements RetractStreamTableSink<Row>, Serializable, IStr
2929
private KuduOutputFormat.WriteMode writeMode;
3030
private Integer workerCount;
3131
private Integer defaultOperationTimeoutMs;
32-
private Integer defaultSocketReadTimeoutMs;
3332
private int parallelism = 1;
3433
private String principal;
3534
private String keytab;
@@ -44,7 +43,6 @@ public KuduSink genStreamSink(AbstractTargetTableInfo targetTableInfo) {
4443
this.kuduMasters = kuduTableInfo.getKuduMasters();
4544
this.tableName = kuduTableInfo.getTableName();
4645
this.defaultOperationTimeoutMs = kuduTableInfo.getDefaultOperationTimeoutMs();
47-
this.defaultSocketReadTimeoutMs = kuduTableInfo.getDefaultSocketReadTimeoutMs();
4846
this.workerCount = kuduTableInfo.getWorkerCount();
4947
this.writeMode = kuduTableInfo.getWriteMode();
5048
this.principal = kuduTableInfo.getPrincipal();
@@ -73,7 +71,6 @@ public DataStreamSink<Tuple2<Boolean, Row>> consumeDataStream(DataStream<Tuple2<
7371
.setWriteMode(writeMode)
7472
.setWorkerCount(this.workerCount)
7573
.setDefaultOperationTimeoutMs(this.defaultOperationTimeoutMs)
76-
.setDefaultSocketReadTimeoutMs(this.defaultSocketReadTimeoutMs)
7774
.setFieldNames(this.fieldNames)
7875
.setFieldTypes(this.fieldTypes)
7976
.setPrincipal(this.principal)

kudu/kudu-sink/src/main/java/com/dtstack/flink/sql/sink/kudu/table/KuduSinkParser.java

Lines changed: 2 additions & 7 deletions
Original file line numberDiff line numberDiff line change
@@ -26,8 +26,6 @@ public class KuduSinkParser extends AbstractTableParser {
2626

2727
public static final String OPERATION_TIMEOUT_MS = "defaultOperationTimeoutMs";
2828

29-
public static final String SOCKET_READ_TIMEOUT_MS = "defaultSocketReadTimeoutMs";
30-
3129
public static final String BATCH_SIZE_KEY = "batchSize";
3230

3331
public static final Integer DEFAULT_BATCH_SIZE = 1000;
@@ -50,18 +48,17 @@ public AbstractTableInfo getTableInfo(String tableName, String fieldsInfo, Map<S
5048
kuduTableInfo.setWriteMode(transWriteMode(MathUtil.getString(props.get(WRITE_MODE.toLowerCase()))));
5149
kuduTableInfo.setWorkerCount(MathUtil.getIntegerVal(props.get(WORKER_COUNT.toLowerCase())));
5250
kuduTableInfo.setDefaultOperationTimeoutMs(MathUtil.getIntegerVal(props.get(OPERATION_TIMEOUT_MS.toLowerCase())));
53-
kuduTableInfo.setDefaultSocketReadTimeoutMs(MathUtil.getIntegerVal(props.get(SOCKET_READ_TIMEOUT_MS.toLowerCase())));
5451
kuduTableInfo.setBatchSize(MathUtil.getIntegerVal(props.getOrDefault(BATCH_SIZE_KEY.toLowerCase(), DEFAULT_BATCH_SIZE)));
5552
kuduTableInfo.setBatchWaitInterval(MathUtil.getIntegerVal(props.getOrDefault(BATCH_WAIT_INTERVAL_KEY.toLowerCase(), DEFAULT_BATCH_WAIT_INTERVAL)));
5653

57-
if (Objects.isNull(props.get(SESSION_FLUSH_MODE_KEY))) {
54+
if (Objects.isNull(props.get(SESSION_FLUSH_MODE_KEY.toLowerCase()))) {
5855
if (kuduTableInfo.getBatchSize() > 1) {
5956
kuduTableInfo.setFlushMode(KuduTableInfo.KuduFlushMode.MANUAL_FLUSH.name());
6057
} else {
6158
kuduTableInfo.setFlushMode(KuduTableInfo.KuduFlushMode.AUTO_FLUSH_SYNC.name());
6259
}
6360
} else {
64-
kuduTableInfo.setFlushMode(MathUtil.getString(props.get(SESSION_FLUSH_MODE_KEY)));
61+
kuduTableInfo.setFlushMode(MathUtil.getString(props.get(SESSION_FLUSH_MODE_KEY.toLowerCase())));
6562
}
6663

6764
kuduTableInfo.setPrincipal(
@@ -84,8 +81,6 @@ private KuduOutputFormat.WriteMode transWriteMode(String writeMode) {
8481
return KuduOutputFormat.WriteMode.INSERT;
8582
case "update":
8683
return KuduOutputFormat.WriteMode.UPDATE;
87-
case "upsert":
88-
return KuduOutputFormat.WriteMode.UPSERT;
8984
default:
9085
return KuduOutputFormat.WriteMode.UPSERT;
9186
}

kudu/kudu-sink/src/main/java/com/dtstack/flink/sql/sink/kudu/table/KuduTableInfo.java

Lines changed: 0 additions & 10 deletions
Original file line numberDiff line numberDiff line change
@@ -19,8 +19,6 @@ public class KuduTableInfo extends AbstractTargetTableInfo implements KerberosTa
1919

2020
private Integer defaultOperationTimeoutMs;
2121

22-
private Integer defaultSocketReadTimeoutMs;
23-
2422
/**
2523
* kerberos
2624
*/
@@ -82,14 +80,6 @@ public void setDefaultOperationTimeoutMs(Integer defaultOperationTimeoutMs) {
8280
this.defaultOperationTimeoutMs = defaultOperationTimeoutMs;
8381
}
8482

85-
public Integer getDefaultSocketReadTimeoutMs() {
86-
return defaultSocketReadTimeoutMs;
87-
}
88-
89-
public void setDefaultSocketReadTimeoutMs(Integer defaultSocketReadTimeoutMs) {
90-
this.defaultSocketReadTimeoutMs = defaultSocketReadTimeoutMs;
91-
}
92-
9383
@Override
9484
public boolean check() {
9585
Preconditions.checkNotNull(kuduMasters, "kudu field of kuduMasters is required");

0 commit comments

Comments
 (0)