Browse Source

[fix#12195] Change date to timestamp to support cross diff time zones (#12239)

* fix DS support cross time zone, use timestamp replace date
3.2.0-release
Yann Ann 2 years ago committed by GitHub
parent
commit
c87f6d89a4
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
  1. 19
      dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/utils/DateUtils.java
  2. 37
      dolphinscheduler-common/src/test/java/org/apache/dolphinscheduler/common/utils/DateUtilsTest.java
  3. 7
      dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/builder/TaskExecutionContextBuilder.java
  4. 7
      dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/processor/queue/TaskEvent.java
  5. 2
      dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/processor/TaskAckProcessorTest.java
  6. 4
      dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/processor/queue/TaskResponseServiceTest.java
  7. 4
      dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/command/TaskExecuteResultCommand.java
  8. 4
      dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/command/TaskExecuteRunningCommand.java
  9. 3
      dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/utils/LogUtils.java
  10. 2
      dolphinscheduler-service/src/test/java/org/apache/dolphinscheduler/service/utils/LogUtilsTest.java
  11. 12
      dolphinscheduler-spi/src/main/java/org/apache/dolphinscheduler/spi/utils/DateUtils.java
  12. 2
      dolphinscheduler-task-plugin/dolphinscheduler-task-api/src/main/java/org/apache/dolphinscheduler/plugin/task/api/AbstractCommandExecutor.java
  13. 9
      dolphinscheduler-task-plugin/dolphinscheduler-task-api/src/main/java/org/apache/dolphinscheduler/plugin/task/api/TaskExecutionContext.java
  14. 2
      dolphinscheduler-task-plugin/dolphinscheduler-task-pigeon/src/test/java/org/apache/dolphinscheduler/plugin/task/pigeon/PigeonTaskTest.java
  15. 3
      dolphinscheduler-task-plugin/dolphinscheduler-task-sql/src/main/java/org/apache/dolphinscheduler/plugin/task/sql/SqlTask.java
  16. 2
      dolphinscheduler-worker/src/main/java/org/apache/dolphinscheduler/server/worker/processor/TaskDispatchProcessor.java
  17. 3
      dolphinscheduler-worker/src/main/java/org/apache/dolphinscheduler/server/worker/runner/WorkerDelayTaskExecuteRunnable.java
  18. 12
      dolphinscheduler-worker/src/main/java/org/apache/dolphinscheduler/server/worker/runner/WorkerTaskExecuteRunnable.java
  19. 2
      dolphinscheduler-worker/src/test/java/org/apache/dolphinscheduler/server/worker/processor/TaskDispatchProcessorTest.java
  20. 4
      dolphinscheduler-worker/src/test/java/org/apache/dolphinscheduler/server/worker/runner/DefaultWorkerDelayTaskExecuteRunnableTest.java

19
dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/utils/DateUtils.java

@ -646,4 +646,23 @@ public final class DateUtils {
}
/**
* transform timeStamp to local date
*
* @param timeStamp time stamp (milliseconds)
* @return local date
*/
public static @Nullable Date timeStampToDate(long timeStamp) {
return timeStamp <= 0L ? null : new Date(timeStamp);
}
/**
* transform date to timeStamp
* @param date date
* @return time stamp (milliseconds)
*/
public static long dateToTimeStamp(Date date) {
return date == null ? 0L : date.getTime();
}
}

37
dolphinscheduler-common/src/test/java/org/apache/dolphinscheduler/common/utils/DateUtilsTest.java

@ -260,4 +260,41 @@ public class DateUtilsTest {
String utcNowStr = DateUtils.dateToString(asiaShNow, utc);
Assert.assertEquals(asiaShNowStr, utcNowStr);
}
@Test
public void testDateToTimeStamp() throws ParseException {
// Beijing Date
String timeString = "2022-09-29 21:00:00";
SimpleDateFormat sdf = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss");
sdf.setTimeZone(TimeZone.getTimeZone("Asia/Shanghai"));
Date date = sdf.parse(timeString);
long timeStamp = DateUtils.dateToTimeStamp(date);
Assert.assertEquals(1664456400000L, timeStamp);
// Tokyo Date
String tokyoTime = "2022-09-29 22:00:00";
sdf.setTimeZone(TimeZone.getTimeZone("Asia/Tokyo"));
date = sdf.parse(tokyoTime);
timeStamp = DateUtils.dateToTimeStamp(date);
Assert.assertEquals(1664456400000L, timeStamp);
date = null;
Assert.assertEquals(0L, DateUtils.dateToTimeStamp(date));
}
@Test
public void testTimeStampToDate() {
long timeStamp = 1664456400000L;
SimpleDateFormat sdf=new SimpleDateFormat("yyyy-MM-dd HH:mm:ss");
sdf.setTimeZone(TimeZone.getTimeZone("Asia/Shanghai"));
String sd = sdf.format(new Date(timeStamp));
Assert.assertEquals("2022-09-29 21:00:00", sd);
sdf.setTimeZone(TimeZone.getTimeZone("Asia/Tokyo"));
sd = sdf.format(new Date(timeStamp));
Assert.assertEquals("2022-09-29 22:00:00", sd);
Date date = DateUtils.timeStampToDate(0L);
Assert.assertNull(date);
}
}

7
dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/builder/TaskExecutionContextBuilder.java

@ -20,6 +20,7 @@ package org.apache.dolphinscheduler.server.master.builder;
import static org.apache.dolphinscheduler.common.Constants.SEC_2_MINUTES_TIME_UNIT;
import org.apache.dolphinscheduler.common.enums.TimeoutFlag;
import org.apache.dolphinscheduler.common.utils.DateUtils;
import org.apache.dolphinscheduler.dao.entity.ProcessDefinition;
import org.apache.dolphinscheduler.dao.entity.ProcessInstance;
import org.apache.dolphinscheduler.dao.entity.TaskDefinition;
@ -61,8 +62,8 @@ public class TaskExecutionContextBuilder {
public TaskExecutionContextBuilder buildTaskInstanceRelatedInfo(TaskInstance taskInstance) {
taskExecutionContext.setTaskInstanceId(taskInstance.getId());
taskExecutionContext.setTaskName(taskInstance.getName());
taskExecutionContext.setFirstSubmitTime(taskInstance.getFirstSubmitTime());
taskExecutionContext.setStartTime(taskInstance.getStartTime());
taskExecutionContext.setFirstSubmitTime(DateUtils.dateToTimeStamp(taskInstance.getFirstSubmitTime()));
taskExecutionContext.setStartTime(DateUtils.dateToTimeStamp(taskInstance.getStartTime()));
taskExecutionContext.setTaskType(taskInstance.getTaskType());
taskExecutionContext.setLogPath(taskInstance.getLogPath());
taskExecutionContext.setWorkerGroup(taskInstance.getWorkerGroup());
@ -102,7 +103,7 @@ public class TaskExecutionContextBuilder {
*/
public TaskExecutionContextBuilder buildProcessInstanceRelatedInfo(ProcessInstance processInstance) {
taskExecutionContext.setProcessInstanceId(processInstance.getId());
taskExecutionContext.setScheduleTime(processInstance.getScheduleTime());
taskExecutionContext.setScheduleTime(DateUtils.dateToTimeStamp(processInstance.getScheduleTime()));
taskExecutionContext.setGlobalParams(processInstance.getGlobalParams());
taskExecutionContext.setExecutorId(processInstance.getExecutorId());
taskExecutionContext.setCmdTypeIfComplement(processInstance.getCmdTypeIfComplement().getCode());

7
dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/processor/queue/TaskEvent.java

@ -18,6 +18,7 @@
package org.apache.dolphinscheduler.server.master.processor.queue;
import org.apache.dolphinscheduler.common.enums.TaskEventType;
import org.apache.dolphinscheduler.common.utils.DateUtils;
import org.apache.dolphinscheduler.plugin.task.api.enums.TaskExecutionStatus;
import org.apache.dolphinscheduler.remote.command.TaskExecuteResultCommand;
import org.apache.dolphinscheduler.remote.command.TaskExecuteRunningCommand;
@ -110,7 +111,7 @@ public class TaskEvent {
event.setProcessInstanceId(command.getProcessInstanceId());
event.setTaskInstanceId(command.getTaskInstanceId());
event.setState(command.getStatus());
event.setStartTime(command.getStartTime());
event.setStartTime(DateUtils.timeStampToDate(command.getStartTime()));
event.setExecutePath(command.getExecutePath());
event.setLogPath(command.getLogPath());
event.setAppIds(command.getAppIds());
@ -125,10 +126,10 @@ public class TaskEvent {
event.setProcessInstanceId(command.getProcessInstanceId());
event.setTaskInstanceId(command.getTaskInstanceId());
event.setState(TaskExecutionStatus.of(command.getStatus()));
event.setStartTime(command.getStartTime());
event.setStartTime(DateUtils.timeStampToDate(command.getStartTime()));
event.setExecutePath(command.getExecutePath());
event.setLogPath(command.getLogPath());
event.setEndTime(command.getEndTime());
event.setEndTime(DateUtils.timeStampToDate(command.getEndTime()));
event.setProcessId(command.getProcessId());
event.setAppIds(command.getAppIds());
event.setVarPool(command.getVarPool());

2
dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/processor/TaskAckProcessorTest.java

@ -69,7 +69,7 @@ public class TaskAckProcessorTest {
taskExecuteRunningMessage.setExecutePath("/dolphinscheduler/worker");
taskExecuteRunningMessage.setHost("localhost");
taskExecuteRunningMessage.setLogPath("/temp/worker.log");
taskExecuteRunningMessage.setStartTime(new Date());
taskExecuteRunningMessage.setStartTime(System.currentTimeMillis());
taskExecuteRunningMessage.setTaskInstanceId(1);
taskExecuteRunningMessage.setProcessInstanceId(1);
}

4
dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/processor/queue/TaskResponseServiceTest.java

@ -82,7 +82,7 @@ public class TaskResponseServiceTest {
taskExecuteRunningMessage.setExecutePath("path");
taskExecuteRunningMessage.setLogPath("logPath");
taskExecuteRunningMessage.setHost("127.*.*.*");
taskExecuteRunningMessage.setStartTime(new Date());
taskExecuteRunningMessage.setStartTime(System.currentTimeMillis());
ackEvent = TaskEvent.newRunningEvent(taskExecuteRunningMessage,
channel,
@ -94,7 +94,7 @@ public class TaskResponseServiceTest {
taskExecuteResultMessage.setProcessInstanceId(1);
taskExecuteResultMessage.setTaskInstanceId(22);
taskExecuteResultMessage.setStatus(TaskExecutionStatus.SUCCESS.getCode());
taskExecuteResultMessage.setEndTime(new Date());
taskExecuteResultMessage.setEndTime(System.currentTimeMillis());
taskExecuteResultMessage.setVarPool("varPol");
taskExecuteResultMessage.setAppIds("ids");
taskExecuteResultMessage.setProcessId(1);

4
dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/command/TaskExecuteResultCommand.java

@ -57,7 +57,7 @@ public class TaskExecuteResultCommand extends BaseCommand {
/**
* startTime
*/
private Date startTime;
private long startTime;
/**
* host
@ -78,7 +78,7 @@ public class TaskExecuteResultCommand extends BaseCommand {
/**
* end time
*/
private Date endTime;
private long endTime;
/**

4
dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/command/TaskExecuteRunningCommand.java

@ -19,8 +19,6 @@ package org.apache.dolphinscheduler.remote.command;
import org.apache.dolphinscheduler.common.utils.JSONUtils;
import java.util.Date;
import lombok.Data;
import lombok.EqualsAndHashCode;
import lombok.NoArgsConstructor;
@ -49,7 +47,7 @@ public class TaskExecuteRunningCommand extends BaseCommand {
/**
* startTime
*/
private Date startTime;
private long startTime;
/**
* host

3
dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/utils/LogUtils.java

@ -74,7 +74,8 @@ public class LogUtils {
* get task log path by TaskExecutionContext
*/
public static String getTaskLogPath(TaskExecutionContext taskExecutionContext) {
return getTaskLogPath(taskExecutionContext.getFirstSubmitTime(), taskExecutionContext.getProcessDefineCode(),
return getTaskLogPath(DateUtils.timeStampToDate(taskExecutionContext.getFirstSubmitTime()),
taskExecutionContext.getProcessDefineCode(),
taskExecutionContext.getProcessDefineVersion(),
taskExecutionContext.getProcessInstanceId(),
taskExecutionContext.getTaskInstanceId());

2
dolphinscheduler-service/src/test/java/org/apache/dolphinscheduler/service/utils/LogUtilsTest.java

@ -47,7 +47,7 @@ public class LogUtilsTest {
taskExecutionContext.setTaskInstanceId(1000);
taskExecutionContext.setProcessDefineCode(1L);
taskExecutionContext.setProcessDefineVersion(1);
taskExecutionContext.setFirstSubmitTime(firstSubmitTime);
taskExecutionContext.setFirstSubmitTime(firstSubmitTime.getTime());
Logger rootLogger = (Logger) LoggerFactory.getILoggerFactory().getLogger("ROOT");
Assert.assertNotNull(rootLogger);

12
dolphinscheduler-spi/src/main/java/org/apache/dolphinscheduler/spi/utils/DateUtils.java

@ -26,6 +26,7 @@ import java.util.Date;
import java.util.Objects;
import java.util.TimeZone;
import javax.annotation.Nullable;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@ -437,4 +438,15 @@ public class DateUtils {
public static String getTimestampString() {
return String.valueOf(System.currentTimeMillis());
}
/**
* transform timeStamp to local date
*
* @param timeStamp time stamp (milliseconds)
* @return local date
*/
public static @Nullable
Date timeStampToDate(long timeStamp) {
return timeStamp <= 0L ? null : new Date(timeStamp);
}
}

2
dolphinscheduler-task-plugin/dolphinscheduler-task-api/src/main/java/org/apache/dolphinscheduler/plugin/task/api/AbstractCommandExecutor.java

@ -403,7 +403,7 @@ public abstract class AbstractCommandExecutor {
* @return remain time
*/
private long getRemainTime() {
long usedTime = (System.currentTimeMillis() - taskRequest.getStartTime().getTime()) / 1000;
long usedTime = (System.currentTimeMillis() - taskRequest.getStartTime()) / 1000;
long remainTime = taskRequest.getTaskTimeout() - usedTime;
if (remainTime < 0) {

9
dolphinscheduler-task-plugin/dolphinscheduler-task-api/src/main/java/org/apache/dolphinscheduler/plugin/task/api/TaskExecutionContext.java

@ -23,7 +23,6 @@ import org.apache.dolphinscheduler.plugin.task.api.model.Property;
import org.apache.dolphinscheduler.plugin.task.api.parameters.resource.ResourceParametersHelper;
import java.io.Serializable;
import java.util.Date;
import java.util.Map;
import lombok.AllArgsConstructor;
@ -55,12 +54,12 @@ public class TaskExecutionContext implements Serializable {
/**
* task first submit time.
*/
private Date firstSubmitTime;
private long firstSubmitTime;
/**
* task start time
*/
private Date startTime;
private long startTime;
/**
* task type
@ -115,7 +114,7 @@ public class TaskExecutionContext implements Serializable {
/**
* process instance schedule time
*/
private Date scheduleTime;
private long scheduleTime;
/**
* process instance global parameters
@ -223,7 +222,7 @@ public class TaskExecutionContext implements Serializable {
/**
* endTime
*/
private Date endTime;
private long endTime;
/**
* sql TaskExecutionContext

2
dolphinscheduler-task-plugin/dolphinscheduler-task-pigeon/src/test/java/org/apache/dolphinscheduler/plugin/task/pigeon/PigeonTaskTest.java

@ -61,7 +61,7 @@ public class PigeonTaskTest {
Mockito.when(taskExecutionContext.getExecutePath()).thenReturn("/tmp");
Mockito.when(taskExecutionContext.getTaskAppId()).thenReturn(UUID.randomUUID().toString());
Mockito.when(taskExecutionContext.getTenantCode()).thenReturn("root");
Mockito.when(taskExecutionContext.getStartTime()).thenReturn(new Date());
Mockito.when(taskExecutionContext.getStartTime()).thenReturn(System.currentTimeMillis());
Mockito.when(taskExecutionContext.getTaskTimeout()).thenReturn(10000);
Mockito.when(taskExecutionContext.getLogPath()).thenReturn("/tmp/dx");
// Mockito.when(taskExecutionContext.getVarPool())

3
dolphinscheduler-task-plugin/dolphinscheduler-task-sql/src/main/java/org/apache/dolphinscheduler/plugin/task/sql/SqlTask.java

@ -33,6 +33,7 @@ import org.apache.dolphinscheduler.plugin.task.api.parser.ParamUtils;
import org.apache.dolphinscheduler.plugin.task.api.parser.ParameterUtils;
import org.apache.dolphinscheduler.spi.datasource.BaseConnectionParam;
import org.apache.dolphinscheduler.spi.enums.DbType;
import org.apache.dolphinscheduler.spi.utils.DateUtils;
import org.apache.dolphinscheduler.spi.utils.JSONUtils;
import org.apache.dolphinscheduler.spi.utils.StringUtils;
@ -483,7 +484,7 @@ public class SqlTask extends AbstractTask {
//new
//replace variable TIME with $[YYYYmmddd...] in sql when history run job and batch complement job
sql = ParameterUtils.replaceScheduleTime(sql, taskExecutionContext.getScheduleTime());
sql = ParameterUtils.replaceScheduleTime(sql, DateUtils.timeStampToDate(taskExecutionContext.getScheduleTime()));
// special characters need to be escaped, ${} needs to be escaped
setSqlParamsMap(sql, rgex, sqlParamsMap, paramsMap,taskExecutionContext.getTaskInstanceId());
//Replace the original value in sql !{...} ,Does not participate in precompilation

2
dolphinscheduler-worker/src/main/java/org/apache/dolphinscheduler/server/worker/processor/TaskDispatchProcessor.java

@ -116,7 +116,7 @@ public class TaskDispatchProcessor implements NettyRequestProcessor {
taskExecutionContext.setLogPath(LogUtils.getTaskLogPath(taskExecutionContext));
// delay task process
long remainTime = DateUtils.getRemainTime(taskExecutionContext.getFirstSubmitTime(),
long remainTime = DateUtils.getRemainTime(DateUtils.timeStampToDate(taskExecutionContext.getFirstSubmitTime()),
taskExecutionContext.getDelayTime() * 60L);
if (remainTime > 0) {
logger.info("Current taskInstance is choose delay execution, delay time: {}s", remainTime);

3
dolphinscheduler-worker/src/main/java/org/apache/dolphinscheduler/server/worker/runner/WorkerDelayTaskExecuteRunnable.java

@ -47,7 +47,8 @@ public abstract class WorkerDelayTaskExecuteRunnable extends WorkerTaskExecuteRu
TaskExecutionContext taskExecutionContext = getTaskExecutionContext();
return unit.convert(
DateUtils.getRemainTime(
taskExecutionContext.getFirstSubmitTime(), taskExecutionContext.getDelayTime() * 60L), TimeUnit.SECONDS);
DateUtils.timeStampToDate(taskExecutionContext.getFirstSubmitTime()),
taskExecutionContext.getDelayTime() * 60L), TimeUnit.SECONDS);
}
@Override

12
dolphinscheduler-worker/src/main/java/org/apache/dolphinscheduler/server/worker/runner/WorkerTaskExecuteRunnable.java

@ -23,6 +23,7 @@ import org.apache.dolphinscheduler.common.Constants;
import org.apache.dolphinscheduler.common.enums.WarningType;
import org.apache.dolphinscheduler.common.storage.StorageOperate;
import org.apache.dolphinscheduler.common.utils.CommonUtils;
import org.apache.dolphinscheduler.common.utils.DateUtils;
import org.apache.dolphinscheduler.common.utils.JSONUtils;
import org.apache.dolphinscheduler.common.utils.LoggerUtils;
import org.apache.dolphinscheduler.plugin.task.api.AbstractTask;
@ -44,6 +45,7 @@ import org.apache.dolphinscheduler.service.alert.AlertClientService;
import org.apache.dolphinscheduler.service.task.TaskPluginManager;
import org.apache.dolphinscheduler.service.utils.ProcessUtils;
import javax.annotation.Nullable;
import java.io.File;
import java.io.IOException;
import java.nio.file.NoSuchFileException;
@ -90,7 +92,7 @@ public abstract class WorkerTaskExecuteRunnable implements Runnable {
this.alertClientService = alertClientService;
this.taskPluginManager = taskPluginManager;
this.storageOperate = storageOperate;
String taskLogName = LoggerUtils.buildTaskId(taskExecutionContext.getFirstSubmitTime(),
String taskLogName = LoggerUtils.buildTaskId(DateUtils.timeStampToDate(taskExecutionContext.getFirstSubmitTime()),
taskExecutionContext.getProcessDefineCode(),
taskExecutionContext.getProcessDefineVersion(),
taskExecutionContext.getProcessInstanceId(),
@ -118,7 +120,7 @@ public abstract class WorkerTaskExecuteRunnable implements Runnable {
cancelTask();
TaskExecutionContextCacheManager.removeByTaskInstanceId(taskExecutionContext.getTaskInstanceId());
taskExecutionContext.setCurrentExecutionStatus(TaskExecutionStatus.FAILURE);
taskExecutionContext.setEndTime(new Date());
taskExecutionContext.setEndTime(System.currentTimeMillis());
workerMessageSender.sendMessageWithRetry(taskExecutionContext, masterAddress, CommandType.TASK_EXECUTE_RESULT);
logger.info(
"Get a exception when execute the task, will send the task execute result to master, the current task execute result is {}",
@ -157,7 +159,7 @@ public abstract class WorkerTaskExecuteRunnable implements Runnable {
if (Constants.DRY_RUN_FLAG_YES == taskExecutionContext.getDryRun()) {
taskExecutionContext.setCurrentExecutionStatus(TaskExecutionStatus.SUCCESS);
taskExecutionContext.setEndTime(new Date());
taskExecutionContext.setEndTime(System.currentTimeMillis());
TaskExecutionContextCacheManager.removeByTaskInstanceId(taskExecutionContext.getTaskInstanceId());
workerMessageSender.sendMessageWithRetry(taskExecutionContext, masterAddress,
CommandType.TASK_EXECUTE_RESULT);
@ -185,7 +187,7 @@ public abstract class WorkerTaskExecuteRunnable implements Runnable {
protected void initializeTask() {
logger.info("Begin to initialize task");
Date taskStartTime = new Date();
long taskStartTime = System.currentTimeMillis();
taskExecutionContext.setStartTime(taskStartTime);
logger.info("Set task startTime: {}", taskStartTime);
@ -251,7 +253,7 @@ public abstract class WorkerTaskExecuteRunnable implements Runnable {
protected void sendTaskResult() {
taskExecutionContext.setCurrentExecutionStatus(task.getExitStatus());
taskExecutionContext.setEndTime(new Date());
taskExecutionContext.setEndTime(System.currentTimeMillis());
taskExecutionContext.setProcessId(task.getProcessId());
taskExecutionContext.setAppIds(task.getAppIds());
taskExecutionContext.setVarPool(JSONUtils.toJsonString(task.getParameters().getVarPool()));

2
dolphinscheduler-worker/src/test/java/org/apache/dolphinscheduler/server/worker/processor/TaskDispatchProcessorTest.java

@ -97,7 +97,7 @@ public class TaskDispatchProcessorTest {
taskExecutionContext.setProcessDefineCode(1L);
taskExecutionContext.setProcessDefineVersion(1);
taskExecutionContext.setTaskType("SQL");
taskExecutionContext.setFirstSubmitTime(new Date());
taskExecutionContext.setFirstSubmitTime(System.currentTimeMillis());
taskExecutionContext.setDelayTime(0);
taskExecutionContext.setLogPath("/tmp/test.log");
taskExecutionContext.setHost("localhost");

4
dolphinscheduler-worker/src/test/java/org/apache/dolphinscheduler/server/worker/runner/DefaultWorkerDelayTaskExecuteRunnableTest.java

@ -53,7 +53,7 @@ public class DefaultWorkerDelayTaskExecuteRunnableTest {
.dryRun(Constants.DRY_RUN_FLAG_YES)
.taskInstanceId(0)
.processDefineId(0)
.firstSubmitTime(new Date())
.firstSubmitTime(System.currentTimeMillis())
.taskLogName("TestLogName")
.build();
WorkerTaskExecuteRunnable workerTaskExecuteRunnable = new DefaultWorkerDelayTaskExecuteRunnable(
@ -77,7 +77,7 @@ public class DefaultWorkerDelayTaskExecuteRunnableTest {
.testFlag(Constants.TEST_FLAG_YES)
.taskInstanceId(0)
.processDefineId(0)
.firstSubmitTime(new Date())
.firstSubmitTime(System.currentTimeMillis())
.taskLogName("TestLogName")
.taskType("SQL")
.taskParams("{\"localParams\":[],\"resourceList\":[],\"type\":\"POSTGRESQL\",\"datasource\":null,\"sql\":\"select * from t_ds_user\",\"sqlType\":\"0\",\"preStatements\":[],\"postStatements\":[],\"segmentSeparator\":\"\",\"displayRows\":10,\"conditionResult\":\"null\",\"dependence\":\"null\",\"switchResult\":\"null\",\"waitStartTimeout\":null}")

Loading…
Cancel
Save