mirror of
https://github.com/alibaba/DataX.git
synced 2025-05-03 22:34:04 +08:00
[improve][plugin][DRDS]
1.Make NetworkTimeout configurable. 2.Let logger Print stack trace.
This commit is contained in:
parent
e26f82ad76
commit
d80ca30fea
@ -8,6 +8,7 @@ import com.alibaba.datax.common.element.LongColumn;
|
||||
import com.alibaba.datax.common.element.Record;
|
||||
import com.alibaba.datax.common.element.StringColumn;
|
||||
import com.alibaba.datax.common.exception.DataXException;
|
||||
import com.alibaba.datax.common.exception.ExceptionTracker;
|
||||
import com.alibaba.datax.common.plugin.RecordSender;
|
||||
import com.alibaba.datax.common.plugin.TaskPluginCollector;
|
||||
import com.alibaba.datax.common.statistics.PerfRecord;
|
||||
@ -128,6 +129,9 @@ public class CommonRdbmsReader {
|
||||
// 作为日志显示信息时,需要附带的通用信息。比如信息所对应的数据库连接等信息,针对哪个表做的操作
|
||||
private String basicMsg;
|
||||
|
||||
// 目前
|
||||
private Integer netWorkTimeOut;
|
||||
|
||||
public Task(DataBaseType dataBaseType) {
|
||||
this(dataBaseType, -1, -1);
|
||||
}
|
||||
@ -145,6 +149,7 @@ public class CommonRdbmsReader {
|
||||
this.username = readerSliceConfig.getString(Key.USERNAME);
|
||||
this.password = readerSliceConfig.getString(Key.PASSWORD);
|
||||
this.jdbcUrl = readerSliceConfig.getString(Key.JDBC_URL);
|
||||
this.netWorkTimeOut=readerSliceConfig.getInt(Key.NETWORK_TIMEOUT,Constant.NETWORK_TIMEOUT_DEFAULT);
|
||||
|
||||
//ob10的处理
|
||||
if (this.jdbcUrl.startsWith(com.alibaba.datax.plugin.rdbms.writer.Constant.OB10_SPLIT_STRING) && this.dataBaseType == DataBaseType.MySql) {
|
||||
@ -190,11 +195,11 @@ public class CommonRdbmsReader {
|
||||
if (DataBaseType.DRDS.equals(dataBaseType)){
|
||||
LOG.info("start DRDS setNetWorkTimeOut...");
|
||||
try {
|
||||
conn.setNetworkTimeout(Executors.newFixedThreadPool(1), 172800000);
|
||||
conn.setNetworkTimeout(Executors.newFixedThreadPool(1), netWorkTimeOut);
|
||||
LOG.info("end DRDS setNetWorkTimeOut...");
|
||||
} catch (SQLException throwables) {
|
||||
LOG.error("setNetWorkTimeOut error ["+throwables.getMessage()+"]");
|
||||
throwables.printStackTrace();
|
||||
LOG.warn("setNetWorkTimeOut error ["+throwables.getMessage()+"]");
|
||||
LOG.warn(ExceptionTracker.trace(throwables));
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -27,4 +27,7 @@ public final class Constant {
|
||||
|
||||
public static Integer SPLIT_FACTOR = 5;
|
||||
|
||||
//增加优化参数
|
||||
public final static Integer NETWORK_TIMEOUT_DEFAULT=28800000;
|
||||
|
||||
}
|
||||
|
@ -55,4 +55,7 @@ public final class Key {
|
||||
public final static String REUSE_CONN = "reuseConn";
|
||||
|
||||
public final static String PARTITION_NAME = "partitionName";
|
||||
|
||||
//增加优化参数
|
||||
public final static String NETWORK_TIMEOUT="networkTimeout";
|
||||
}
|
||||
|
Loading…
Reference in New Issue
Block a user