Skip to content

Commit 330be96

Browse files
author
dapeng
committed
添加记述和超过次数任务退出
1 parent caa8824 commit 330be96

File tree

2 files changed

+18
-21
lines changed

2 files changed

+18
-21
lines changed

core/src/main/java/com/dtstack/flink/sql/table/AbstractSideTableParser.java

Lines changed: 0 additions & 6 deletions
Original file line numberDiff line numberDiff line change
@@ -108,12 +108,6 @@ protected void parseCacheProp(AbstractSideTableInfo sideTableInfo, Map<String, O
108108
}
109109
sideTableInfo.setAsyncTimeout(asyncTimeout);
110110
}
111-
if(props.containsKey(AbstractSideTableInfo.ASYNC_TIMEOUT_NUM_KEY.toLowerCase())){
112-
Integer asyncTimeoutNum = MathUtil.getIntegerVal(props.get(AbstractSideTableInfo.ASYNC_TIMEOUT_NUM_KEY.toLowerCase()));
113-
if (asyncTimeoutNum > 0){
114-
sideTableInfo.setAsyncTimeoutNumLimit(asyncTimeoutNum);
115-
}
116-
}
117111

118112
if(props.containsKey(AbstractSideTableInfo.ASYNC_FAIL_MAX_NUM_KEY.toLowerCase())){
119113
Long asyncFailNum = MathUtil.getLongVal(props.get(AbstractSideTableInfo.ASYNC_FAIL_MAX_NUM_KEY.toLowerCase()));

rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/async/RdbAsyncReqRow.java

Lines changed: 18 additions & 15 deletions
Original file line numberDiff line numberDiff line change
@@ -20,6 +20,7 @@
2020
package com.dtstack.flink.sql.side.rdb.async;
2121

2222
import com.dtstack.flink.sql.enums.ECacheContentType;
23+
import com.dtstack.flink.sql.metric.MetricConstant;
2324
import com.dtstack.flink.sql.side.BaseAsyncReqRow;
2425
import com.dtstack.flink.sql.side.BaseSideInfo;
2526
import com.dtstack.flink.sql.side.CacheMissVal;
@@ -31,6 +32,7 @@
3132
import io.vertx.ext.sql.SQLConnection;
3233
import org.apache.calcite.sql.JoinType;
3334
import org.apache.commons.lang3.StringUtils;
35+
import org.apache.flink.metrics.Counter;
3436
import org.apache.flink.streaming.api.functions.async.ResultFuture;
3537
import org.apache.flink.table.runtime.types.CRow;
3638
import org.apache.flink.table.typeutils.TimeIndicatorTypeInfo;
@@ -77,10 +79,12 @@ public class RdbAsyncReqRow extends BaseAsyncReqRow {
7779

7880
private final static AtomicBoolean CONN_STATUS = new AtomicBoolean(true);
7981

80-
private final static AtomicLong TIMOUT_NUM = new AtomicLong(0);
82+
private final static AtomicLong FAIL_NUM = new AtomicLong(0);
8183

8284
private Logger logger = LoggerFactory.getLogger(getClass());
8385

86+
private Counter counter = getRuntimeContext().getMetricGroup().counter(MetricConstant.DT_NUM_SIDE_PARSE_ERROR_RECORDS);
87+
8488
public RdbAsyncReqRow(BaseSideInfo sideInfo) {
8589
super(sideInfo);
8690
}
@@ -123,7 +127,12 @@ private void connectWithRetry(Map<String, Object> inputParams, CRow input, Resul
123127
logger.error("getConnection error", conn.cause());
124128
}
125129
if(failCounter.get() >= sideInfo.getSideTableInfo().getAsyncFailMaxNum(3L)){
126-
outByJoinType(resultFuture, conn.cause());
130+
if(FAIL_NUM.incrementAndGet() > sideInfo.getSideTableInfo().getAsyncFailMaxNum(Long.MAX_VALUE)){
131+
counter.inc();
132+
resultFuture.completeExceptionally(conn.cause());
133+
} else {
134+
dealMissKey(input, resultFuture);
135+
}
127136
finishFlag.set(true);
128137
}
129138
conn.result().close();
@@ -135,6 +144,7 @@ private void connectWithRetry(Map<String, Object> inputParams, CRow input, Resul
135144
handleQuery(conn.result(), inputParams, input, resultFuture);
136145
finishFlag.set(true);
137146
} catch (Exception e) {
147+
dealFillDataError(resultFuture, e, null);
138148
logger.error("", e);
139149
} finally {
140150
latch.countDown();
@@ -200,12 +210,13 @@ private void handleQuery(SQLConnection connection,Map<String, Object> inputParam
200210
JsonArray params = new JsonArray(Lists.newArrayList(inputParams.values()));
201211
connection.queryWithParams(sideInfo.getSqlCondition(), params, rs -> {
202212
if (rs.failed()) {
203-
if(TIMOUT_NUM.incrementAndGet() > sideInfo.getSideTableInfo().getAsyncFailMaxNum(Long.MAX_VALUE)){
204-
outByJoinType(resultFuture, rs.cause());
205-
return;
213+
if(FAIL_NUM.incrementAndGet() > sideInfo.getSideTableInfo().getAsyncFailMaxNum(Long.MAX_VALUE)){
214+
LOG.error("Cannot retrieve the data from the database", rs.cause());
215+
counter.inc();
216+
resultFuture.completeExceptionally(rs.cause());
217+
} else {
218+
dealMissKey(input, resultFuture);
206219
}
207-
LOG.error("Cannot retrieve the data from the database", rs.cause());
208-
resultFuture.complete(null);
209220
return;
210221
}
211222

@@ -244,12 +255,4 @@ private void handleQuery(SQLConnection connection,Map<String, Object> inputParam
244255
});
245256
}
246257

247-
private void outByJoinType(ResultFuture<CRow> resultFuture, Throwable e){
248-
if(sideInfo.getJoinType() == JoinType.LEFT){
249-
resultFuture.complete(null);
250-
return;
251-
}
252-
resultFuture.completeExceptionally(e);
253-
}
254-
255258
}

0 commit comments

Comments
 (0)