You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@dolphinscheduler.apache.org by GitBox <gi...@apache.org> on 2022/06/27 16:00:24 UTC

[GitHub] [dolphinscheduler] aiwenmo opened a new pull request, #10640: [Feature-10444][Task] Add Dinky task to better support the development and execution of FlinkSQL

aiwenmo opened a new pull request, #10640:
URL: https://github.com/apache/dolphinscheduler/pull/10640

   
   <!--Thanks very much for contributing to Apache DolphinScheduler. Please review https://dolphinscheduler.apache.org/en-us/community/development/pull-request.html before opening a pull request.-->
   
   
   ## Purpose of the pull request
   Add Dinky task to better support the development and execution of FlinkSQL.
   <!--(For example: This pull request adds checkstyle plugin).-->
   
   ## Brief change log
   
   <!--*(for example:)*
     - *Add maven-checkstyle-plugin to root pom.xml*
   -->
   ## Verify this pull request
   
   <!--*(Please pick either of the following options)*-->
   
   This pull request is code cleanup without any test coverage.
   
   *(or)*
   
   This pull request is already covered by existing tests, such as *(please describe tests)*.
   
   (or)
   
   This change added tests and can be verified as follows:
   
   <!--*(example:)*
     - *Added dolphinscheduler-dao tests for end-to-end.*
     - *Added CronUtilsTest to verify the change.*
     - *Manually verified the change by testing locally.* -->
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@dolphinscheduler.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [dolphinscheduler] caishunfeng commented on a diff in pull request #10640: [Feature-10444][Task] Add Dinky task to better support the development and execution of FlinkSQL

Posted by GitBox <gi...@apache.org>.
caishunfeng commented on code in PR #10640:
URL: https://github.com/apache/dolphinscheduler/pull/10640#discussion_r909158269


##########
dolphinscheduler-task-plugin/dolphinscheduler-task-dinky/src/main/java/org/apache/dolphinscheduler/plugin/task/dinky/DinkyTask.java:
##########
@@ -0,0 +1,272 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.dolphinscheduler.plugin.task.dinky;
+
+import org.apache.dolphinscheduler.plugin.task.api.AbstractTaskExecutor;
+import org.apache.dolphinscheduler.plugin.task.api.TaskConstants;
+import org.apache.dolphinscheduler.plugin.task.api.TaskExecutionContext;
+import org.apache.dolphinscheduler.plugin.task.api.parameters.AbstractParameters;
+import org.apache.dolphinscheduler.spi.utils.JSONUtils;
+import org.apache.dolphinscheduler.spi.utils.StringUtils;
+
+import org.apache.http.HttpResponse;
+import org.apache.http.HttpStatus;
+import org.apache.http.client.HttpClient;
+import org.apache.http.client.methods.HttpGet;
+import org.apache.http.client.methods.HttpPost;
+import org.apache.http.client.utils.URIBuilder;
+import org.apache.http.entity.StringEntity;
+import org.apache.http.impl.client.HttpClientBuilder;
+import org.apache.http.util.EntityUtils;
+
+import java.net.URI;
+import java.nio.charset.StandardCharsets;
+import java.util.HashMap;
+import java.util.Map;
+
+import com.fasterxml.jackson.core.JsonProcessingException;
+import com.fasterxml.jackson.databind.JsonNode;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.databind.node.MissingNode;
+
+public class DinkyTask extends AbstractTaskExecutor {
+
+    /**
+     * taskExecutionContext
+     */
+    private final TaskExecutionContext taskExecutionContext;
+
+    /**
+     * dinky parameters
+     */
+    private DinkyParameters dinkyParameters;
+
+    /**
+     * constructor
+     *
+     * @param taskExecutionContext taskExecutionContext
+     */
+    protected DinkyTask(TaskExecutionContext taskExecutionContext) {
+        super(taskExecutionContext);
+        this.taskExecutionContext = taskExecutionContext;
+    }
+
+    @Override
+    public void init() {
+        final String taskParams = taskExecutionContext.getTaskParams();
+        logger.info("dinky task params:{}", taskParams);
+        this.dinkyParameters = JSONUtils.parseObject(taskParams, DinkyParameters.class);
+        if (this.dinkyParameters == null || !this.dinkyParameters.checkParameters()) {
+            throw new DinkyTaskException("dinky task params is not valid");
+        }
+    }
+
+    @Override
+    public void handle() throws Exception {
+        try {
+            String address = this.dinkyParameters.getAddress();
+            String taskId = this.dinkyParameters.getTaskId();
+            boolean isOnline = this.dinkyParameters.isOnline();
+            JsonNode result;
+            if (isOnline) {
+                // Online dinky task, and only one job is allowed to execute
+                result = onlineTask(address, taskId);
+            } else {
+                // Submit dinky task
+                result = submitTask(address, taskId);
+            }
+            if (checkResult(result)) {
+                boolean status = result.get(DinkyTaskConstants.API_RESULT_DATAS).get("success").asBoolean();
+                String jobInstanceId = result.get(DinkyTaskConstants.API_RESULT_DATAS).get("jobInstanceId").asText();
+                boolean finishFlag = false;
+                while (!finishFlag) {
+                    JsonNode jobInstanceInfoResult = getJobInstanceInfo(address, jobInstanceId);
+                    if (!checkResult(jobInstanceInfoResult)) {
+                        break;
+                    }
+                    String jobInstanceStatus = jobInstanceInfoResult.get(DinkyTaskConstants.API_RESULT_DATAS).get("status").asText();
+                    switch (jobInstanceStatus) {
+                        case DinkyTaskConstants.STATUS_FINISHED:
+                            final int exitStatusCode = mapStatusToExitCode(status);
+                            // Use address-taskId as app id
+                            setAppIds(String.format("%s-%s", address, taskId));
+                            setExitStatusCode(exitStatusCode);
+                            logger.info("dinky task finished with results: {}", result.get(DinkyTaskConstants.API_RESULT_DATAS));
+                            finishFlag = true;
+                            break;
+                        case DinkyTaskConstants.STATUS_FAILED:
+                        case DinkyTaskConstants.STATUS_CANCELED:
+                        case DinkyTaskConstants.STATUS_UNKNOWN:
+                            errorHandle(jobInstanceInfoResult.get(DinkyTaskConstants.API_RESULT_DATAS).get("error").asText());
+                            finishFlag = true;
+                            break;
+                        default:
+                            Thread.sleep(DinkyTaskConstants.SLEEP_MILLIS);

Review Comment:
   Why to sleep here? What state does default represent?



##########
dolphinscheduler-task-plugin/dolphinscheduler-task-dinky/src/main/java/org/apache/dolphinscheduler/plugin/task/dinky/DinkyTask.java:
##########
@@ -0,0 +1,272 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.dolphinscheduler.plugin.task.dinky;
+
+import org.apache.dolphinscheduler.plugin.task.api.AbstractTaskExecutor;
+import org.apache.dolphinscheduler.plugin.task.api.TaskConstants;
+import org.apache.dolphinscheduler.plugin.task.api.TaskExecutionContext;
+import org.apache.dolphinscheduler.plugin.task.api.parameters.AbstractParameters;
+import org.apache.dolphinscheduler.spi.utils.JSONUtils;
+import org.apache.dolphinscheduler.spi.utils.StringUtils;
+
+import org.apache.http.HttpResponse;
+import org.apache.http.HttpStatus;
+import org.apache.http.client.HttpClient;
+import org.apache.http.client.methods.HttpGet;
+import org.apache.http.client.methods.HttpPost;
+import org.apache.http.client.utils.URIBuilder;
+import org.apache.http.entity.StringEntity;
+import org.apache.http.impl.client.HttpClientBuilder;
+import org.apache.http.util.EntityUtils;
+
+import java.net.URI;
+import java.nio.charset.StandardCharsets;
+import java.util.HashMap;
+import java.util.Map;
+
+import com.fasterxml.jackson.core.JsonProcessingException;
+import com.fasterxml.jackson.databind.JsonNode;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.databind.node.MissingNode;
+
+public class DinkyTask extends AbstractTaskExecutor {
+
+    /**
+     * taskExecutionContext
+     */
+    private final TaskExecutionContext taskExecutionContext;
+
+    /**
+     * dinky parameters
+     */
+    private DinkyParameters dinkyParameters;
+
+    /**
+     * constructor
+     *
+     * @param taskExecutionContext taskExecutionContext
+     */
+    protected DinkyTask(TaskExecutionContext taskExecutionContext) {
+        super(taskExecutionContext);
+        this.taskExecutionContext = taskExecutionContext;
+    }
+
+    @Override
+    public void init() {
+        final String taskParams = taskExecutionContext.getTaskParams();
+        logger.info("dinky task params:{}", taskParams);
+        this.dinkyParameters = JSONUtils.parseObject(taskParams, DinkyParameters.class);
+        if (this.dinkyParameters == null || !this.dinkyParameters.checkParameters()) {
+            throw new DinkyTaskException("dinky task params is not valid");
+        }
+    }
+
+    @Override
+    public void handle() throws Exception {
+        try {
+            String address = this.dinkyParameters.getAddress();
+            String taskId = this.dinkyParameters.getTaskId();
+            boolean isOnline = this.dinkyParameters.isOnline();
+            JsonNode result;
+            if (isOnline) {
+                // Online dinky task, and only one job is allowed to execute
+                result = onlineTask(address, taskId);
+            } else {
+                // Submit dinky task
+                result = submitTask(address, taskId);
+            }
+            if (checkResult(result)) {
+                boolean status = result.get(DinkyTaskConstants.API_RESULT_DATAS).get("success").asBoolean();
+                String jobInstanceId = result.get(DinkyTaskConstants.API_RESULT_DATAS).get("jobInstanceId").asText();
+                boolean finishFlag = false;
+                while (!finishFlag) {
+                    JsonNode jobInstanceInfoResult = getJobInstanceInfo(address, jobInstanceId);
+                    if (!checkResult(jobInstanceInfoResult)) {
+                        break;
+                    }
+                    String jobInstanceStatus = jobInstanceInfoResult.get(DinkyTaskConstants.API_RESULT_DATAS).get("status").asText();
+                    switch (jobInstanceStatus) {
+                        case DinkyTaskConstants.STATUS_FINISHED:
+                            final int exitStatusCode = mapStatusToExitCode(status);
+                            // Use address-taskId as app id
+                            setAppIds(String.format("%s-%s", address, taskId));
+                            setExitStatusCode(exitStatusCode);
+                            logger.info("dinky task finished with results: {}", result.get(DinkyTaskConstants.API_RESULT_DATAS));
+                            finishFlag = true;
+                            break;
+                        case DinkyTaskConstants.STATUS_FAILED:
+                        case DinkyTaskConstants.STATUS_CANCELED:
+                        case DinkyTaskConstants.STATUS_UNKNOWN:
+                            errorHandle(jobInstanceInfoResult.get(DinkyTaskConstants.API_RESULT_DATAS).get("error").asText());
+                            finishFlag = true;
+                            break;
+                        default:
+                            Thread.sleep(DinkyTaskConstants.SLEEP_MILLIS);
+                    }
+                }
+            }
+        } catch (Exception e) {
+            errorHandle(e);

Review Comment:
   It seems unnecessary to package error handle. You can handle error directly.



##########
dolphinscheduler-task-plugin/dolphinscheduler-task-dinky/src/main/java/org/apache/dolphinscheduler/plugin/task/dinky/DinkyTask.java:
##########
@@ -0,0 +1,272 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.dolphinscheduler.plugin.task.dinky;
+
+import org.apache.dolphinscheduler.plugin.task.api.AbstractTaskExecutor;
+import org.apache.dolphinscheduler.plugin.task.api.TaskConstants;
+import org.apache.dolphinscheduler.plugin.task.api.TaskExecutionContext;
+import org.apache.dolphinscheduler.plugin.task.api.parameters.AbstractParameters;
+import org.apache.dolphinscheduler.spi.utils.JSONUtils;
+import org.apache.dolphinscheduler.spi.utils.StringUtils;
+
+import org.apache.http.HttpResponse;
+import org.apache.http.HttpStatus;
+import org.apache.http.client.HttpClient;
+import org.apache.http.client.methods.HttpGet;
+import org.apache.http.client.methods.HttpPost;
+import org.apache.http.client.utils.URIBuilder;
+import org.apache.http.entity.StringEntity;
+import org.apache.http.impl.client.HttpClientBuilder;
+import org.apache.http.util.EntityUtils;
+
+import java.net.URI;
+import java.nio.charset.StandardCharsets;
+import java.util.HashMap;
+import java.util.Map;
+
+import com.fasterxml.jackson.core.JsonProcessingException;
+import com.fasterxml.jackson.databind.JsonNode;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.databind.node.MissingNode;
+
+public class DinkyTask extends AbstractTaskExecutor {
+
+    /**
+     * taskExecutionContext
+     */
+    private final TaskExecutionContext taskExecutionContext;
+
+    /**
+     * dinky parameters
+     */
+    private DinkyParameters dinkyParameters;
+
+    /**
+     * constructor
+     *
+     * @param taskExecutionContext taskExecutionContext
+     */
+    protected DinkyTask(TaskExecutionContext taskExecutionContext) {
+        super(taskExecutionContext);
+        this.taskExecutionContext = taskExecutionContext;
+    }
+
+    @Override
+    public void init() {
+        final String taskParams = taskExecutionContext.getTaskParams();
+        logger.info("dinky task params:{}", taskParams);
+        this.dinkyParameters = JSONUtils.parseObject(taskParams, DinkyParameters.class);
+        if (this.dinkyParameters == null || !this.dinkyParameters.checkParameters()) {
+            throw new DinkyTaskException("dinky task params is not valid");
+        }
+    }
+
+    @Override
+    public void handle() throws Exception {
+        try {
+            String address = this.dinkyParameters.getAddress();
+            String taskId = this.dinkyParameters.getTaskId();
+            boolean isOnline = this.dinkyParameters.isOnline();
+            JsonNode result;
+            if (isOnline) {
+                // Online dinky task, and only one job is allowed to execute
+                result = onlineTask(address, taskId);
+            } else {
+                // Submit dinky task
+                result = submitTask(address, taskId);
+            }
+            if (checkResult(result)) {
+                boolean status = result.get(DinkyTaskConstants.API_RESULT_DATAS).get("success").asBoolean();
+                String jobInstanceId = result.get(DinkyTaskConstants.API_RESULT_DATAS).get("jobInstanceId").asText();
+                boolean finishFlag = false;
+                while (!finishFlag) {
+                    JsonNode jobInstanceInfoResult = getJobInstanceInfo(address, jobInstanceId);
+                    if (!checkResult(jobInstanceInfoResult)) {
+                        break;
+                    }
+                    String jobInstanceStatus = jobInstanceInfoResult.get(DinkyTaskConstants.API_RESULT_DATAS).get("status").asText();
+                    switch (jobInstanceStatus) {
+                        case DinkyTaskConstants.STATUS_FINISHED:
+                            final int exitStatusCode = mapStatusToExitCode(status);
+                            // Use address-taskId as app id
+                            setAppIds(String.format("%s-%s", address, taskId));
+                            setExitStatusCode(exitStatusCode);
+                            logger.info("dinky task finished with results: {}", result.get(DinkyTaskConstants.API_RESULT_DATAS));
+                            finishFlag = true;
+                            break;
+                        case DinkyTaskConstants.STATUS_FAILED:
+                        case DinkyTaskConstants.STATUS_CANCELED:
+                        case DinkyTaskConstants.STATUS_UNKNOWN:
+                            errorHandle(jobInstanceInfoResult.get(DinkyTaskConstants.API_RESULT_DATAS).get("error").asText());
+                            finishFlag = true;
+                            break;
+                        default:
+                            Thread.sleep(DinkyTaskConstants.SLEEP_MILLIS);
+                    }
+                }
+            }
+        } catch (Exception e) {
+            errorHandle(e);
+        }
+    }
+
+    /**
+     * map dinky task status to exitStatusCode
+     *
+     * @param status dinky job status
+     * @return exitStatusCode
+     */
+    private int mapStatusToExitCode(boolean status) {
+        if (status) {
+            return TaskConstants.EXIT_CODE_SUCCESS;
+        } else {
+            return TaskConstants.EXIT_CODE_FAILURE;
+        }
+    }
+
+    private boolean checkResult(JsonNode result) {
+        if (result instanceof MissingNode || result == null) {
+            errorHandle(DinkyTaskConstants.API_VERSION_ERROR_TIPS);
+            return false;
+        } else if (result.get("code").asInt() == DinkyTaskConstants.API_ERROR) {
+            errorHandle(result.get("msg"));
+            return false;
+        }
+        return true;
+    }
+
+    private void errorHandle(Object msg) {
+        setExitStatusCode(TaskConstants.EXIT_CODE_FAILURE);
+        logger.error("dinky task submit failed with error: {}", msg);
+    }
+
+    @Override
+    public AbstractParameters getParameters() {
+        return dinkyParameters;
+    }
+
+    @Override
+    public void cancelApplication(boolean status) throws Exception {
+        super.cancelApplication(status);
+        String address = this.dinkyParameters.getAddress();
+        String taskId = this.dinkyParameters.getTaskId();
+        logger.info("trying terminate dinky task, taskId: {}, address: {}, taskId: {}",
+            this.taskExecutionContext.getTaskInstanceId(),
+            address,
+            taskId);
+        cancelTask(address, taskId);
+        logger.info("dinky task terminated, taskId: {}, address: {}, taskId: {}",
+            this.taskExecutionContext.getTaskInstanceId(),
+            address,
+            taskId);
+    }
+
+    private JsonNode submitTask(String address, String taskId) {
+        Map<String, String> params = new HashMap<>();
+        params.put(DinkyTaskConstants.PARAM_TASK_ID, taskId);
+        return parse(doGet(address + DinkyTaskConstants.SUBMIT_TASK, params));
+    }
+
+    private JsonNode onlineTask(String address, String taskId) {
+        Map<String, String> params = new HashMap<>();
+        params.put(DinkyTaskConstants.PARAM_TASK_ID, taskId);
+        return parse(doGet(address + DinkyTaskConstants.ONLINE_TASK, params));
+    }
+
+    private JsonNode cancelTask(String address, String taskId) {
+        Map<String, String> params = new HashMap<>();
+        params.put(DinkyTaskConstants.PARAM_JSON_TASK_ID, taskId);
+        params.put(DinkyTaskConstants.PARAM_SAVEPOINT_TYPE, DinkyTaskConstants.SAVEPOINT_CANCEL);
+        return parse(sendJsonStr(address + DinkyTaskConstants.SAVEPOINT_TASK, JSONUtils.toJsonString(params)));
+    }
+
+    private JsonNode getJobInstanceInfo(String address, String taskId) {
+        Map<String, String> params = new HashMap<>();
+        params.put(DinkyTaskConstants.PARAM_JOB_INSTANCE_ID, taskId);
+        return parse(doGet(address + DinkyTaskConstants.GET_JOB_INFO, params));
+    }
+
+    private JsonNode parse(String res) {
+        ObjectMapper mapper = new ObjectMapper();
+        JsonNode result = null;
+        try {
+            result = mapper.readTree(res);
+        } catch (JsonProcessingException e) {
+            logger.error("dinky task submit failed with error", e);
+        }
+        return result;
+    }
+
+    private String doGet(String url, Map<String, String> params) {
+        String result = "";
+        HttpClient httpClient = HttpClientBuilder.create().build();
+        HttpGet httpGet = null;
+        try {
+            URIBuilder uriBuilder = new URIBuilder(url);
+            if (null != params && !params.isEmpty()) {
+                for (Map.Entry<String, String> entry : params.entrySet()) {
+                    uriBuilder.addParameter(entry.getKey(), entry.getValue());
+                }
+            }
+            URI uri = uriBuilder.build();
+            httpGet = new HttpGet(uri);
+            logger.info("access url: {}", uri);
+            HttpResponse response = httpClient.execute(httpGet);
+            if (response.getStatusLine().getStatusCode() == HttpStatus.SC_OK) {
+                result = EntityUtils.toString(response.getEntity());
+                logger.info("dinky task succeed with results: {}", result);
+            } else {
+                logger.info("dinky task terminated,response: {}", response);

Review Comment:
   Change the log level to `warn` or `error` is better, WDYT?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@dolphinscheduler.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [dolphinscheduler] aiwenmo commented on a diff in pull request #10640: [Feature-10444][Task] Add Dinky task to better support the development and execution of FlinkSQL

Posted by GitBox <gi...@apache.org>.
aiwenmo commented on code in PR #10640:
URL: https://github.com/apache/dolphinscheduler/pull/10640#discussion_r909484859


##########
dolphinscheduler-task-plugin/dolphinscheduler-task-dinky/src/main/java/org/apache/dolphinscheduler/plugin/task/dinky/DinkyTask.java:
##########
@@ -0,0 +1,272 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.dolphinscheduler.plugin.task.dinky;
+
+import org.apache.dolphinscheduler.plugin.task.api.AbstractTaskExecutor;
+import org.apache.dolphinscheduler.plugin.task.api.TaskConstants;
+import org.apache.dolphinscheduler.plugin.task.api.TaskExecutionContext;
+import org.apache.dolphinscheduler.plugin.task.api.parameters.AbstractParameters;
+import org.apache.dolphinscheduler.spi.utils.JSONUtils;
+import org.apache.dolphinscheduler.spi.utils.StringUtils;
+
+import org.apache.http.HttpResponse;
+import org.apache.http.HttpStatus;
+import org.apache.http.client.HttpClient;
+import org.apache.http.client.methods.HttpGet;
+import org.apache.http.client.methods.HttpPost;
+import org.apache.http.client.utils.URIBuilder;
+import org.apache.http.entity.StringEntity;
+import org.apache.http.impl.client.HttpClientBuilder;
+import org.apache.http.util.EntityUtils;
+
+import java.net.URI;
+import java.nio.charset.StandardCharsets;
+import java.util.HashMap;
+import java.util.Map;
+
+import com.fasterxml.jackson.core.JsonProcessingException;
+import com.fasterxml.jackson.databind.JsonNode;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.databind.node.MissingNode;
+
+public class DinkyTask extends AbstractTaskExecutor {
+
+    /**
+     * taskExecutionContext
+     */
+    private final TaskExecutionContext taskExecutionContext;
+
+    /**
+     * dinky parameters
+     */
+    private DinkyParameters dinkyParameters;
+
+    /**
+     * constructor
+     *
+     * @param taskExecutionContext taskExecutionContext
+     */
+    protected DinkyTask(TaskExecutionContext taskExecutionContext) {
+        super(taskExecutionContext);
+        this.taskExecutionContext = taskExecutionContext;
+    }
+
+    @Override
+    public void init() {
+        final String taskParams = taskExecutionContext.getTaskParams();
+        logger.info("dinky task params:{}", taskParams);
+        this.dinkyParameters = JSONUtils.parseObject(taskParams, DinkyParameters.class);
+        if (this.dinkyParameters == null || !this.dinkyParameters.checkParameters()) {
+            throw new DinkyTaskException("dinky task params is not valid");
+        }
+    }
+
+    @Override
+    public void handle() throws Exception {
+        try {
+            String address = this.dinkyParameters.getAddress();
+            String taskId = this.dinkyParameters.getTaskId();
+            boolean isOnline = this.dinkyParameters.isOnline();
+            JsonNode result;
+            if (isOnline) {
+                // Online dinky task, and only one job is allowed to execute
+                result = onlineTask(address, taskId);
+            } else {
+                // Submit dinky task
+                result = submitTask(address, taskId);
+            }
+            if (checkResult(result)) {
+                boolean status = result.get(DinkyTaskConstants.API_RESULT_DATAS).get("success").asBoolean();
+                String jobInstanceId = result.get(DinkyTaskConstants.API_RESULT_DATAS).get("jobInstanceId").asText();
+                boolean finishFlag = false;
+                while (!finishFlag) {
+                    JsonNode jobInstanceInfoResult = getJobInstanceInfo(address, jobInstanceId);
+                    if (!checkResult(jobInstanceInfoResult)) {
+                        break;
+                    }
+                    String jobInstanceStatus = jobInstanceInfoResult.get(DinkyTaskConstants.API_RESULT_DATAS).get("status").asText();
+                    switch (jobInstanceStatus) {
+                        case DinkyTaskConstants.STATUS_FINISHED:
+                            final int exitStatusCode = mapStatusToExitCode(status);
+                            // Use address-taskId as app id
+                            setAppIds(String.format("%s-%s", address, taskId));
+                            setExitStatusCode(exitStatusCode);
+                            logger.info("dinky task finished with results: {}", result.get(DinkyTaskConstants.API_RESULT_DATAS));
+                            finishFlag = true;
+                            break;
+                        case DinkyTaskConstants.STATUS_FAILED:
+                        case DinkyTaskConstants.STATUS_CANCELED:
+                        case DinkyTaskConstants.STATUS_UNKNOWN:
+                            errorHandle(jobInstanceInfoResult.get(DinkyTaskConstants.API_RESULT_DATAS).get("error").asText());
+                            finishFlag = true;
+                            break;
+                        default:
+                            Thread.sleep(DinkyTaskConstants.SLEEP_MILLIS);
+                    }
+                }
+            }
+        } catch (Exception e) {
+            errorHandle(e);

Review Comment:
   Hi. Do you mean to remove 'try catch' ?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@dolphinscheduler.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [dolphinscheduler] SbloodyS closed pull request #10640: [Feature-10444][Task] Add Dinky task to better support the development and execution of FlinkSQL

Posted by GitBox <gi...@apache.org>.
SbloodyS closed pull request #10640: [Feature-10444][Task] Add Dinky task to better support the development and execution of FlinkSQL
URL: https://github.com/apache/dolphinscheduler/pull/10640


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@dolphinscheduler.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [dolphinscheduler] sonarcloud[bot] commented on pull request #10640: [Feature-10444][Task] Add Dinky task to better support the development and execution of FlinkSQL

Posted by GitBox <gi...@apache.org>.
sonarcloud[bot] commented on PR #10640:
URL: https://github.com/apache/dolphinscheduler/pull/10640#issuecomment-1170700800

   SonarCloud Quality Gate failed.&nbsp; &nbsp; [![Quality Gate failed](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/QualityGateBadge/failed-16px.png 'Quality Gate failed')](https://sonarcloud.io/dashboard?id=apache-dolphinscheduler&pullRequest=10640)
   
   [![Bug](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/bug-16px.png 'Bug')](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=10640&resolved=false&types=BUG) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=10640&resolved=false&types=BUG) [0 Bugs](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=10640&resolved=false&types=BUG)  
   [![Vulnerability](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/vulnerability-16px.png 'Vulnerability')](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=10640&resolved=false&types=VULNERABILITY) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=10640&resolved=false&types=VULNERABILITY) [0 Vulnerabilities](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=10640&resolved=false&types=VULNERABILITY)  
   [![Security Hotspot](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/security_hotspot-16px.png 'Security Hotspot')](https://sonarcloud.io/project/security_hotspots?id=apache-dolphinscheduler&pullRequest=10640&resolved=false&types=SECURITY_HOTSPOT) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/security_hotspots?id=apache-dolphinscheduler&pullRequest=10640&resolved=false&types=SECURITY_HOTSPOT) [0 Security Hotspots](https://sonarcloud.io/project/security_hotspots?id=apache-dolphinscheduler&pullRequest=10640&resolved=false&types=SECURITY_HOTSPOT)  
   [![Code Smell](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/code_smell-16px.png 'Code Smell')](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=10640&resolved=false&types=CODE_SMELL) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=10640&resolved=false&types=CODE_SMELL) [0 Code Smells](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=10640&resolved=false&types=CODE_SMELL)
   
   [![0.0%](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/CoverageChart/0-16px.png '0.0%')](https://sonarcloud.io/component_measures?id=apache-dolphinscheduler&pullRequest=10640&metric=new_coverage&view=list) [0.0% Coverage](https://sonarcloud.io/component_measures?id=apache-dolphinscheduler&pullRequest=10640&metric=new_coverage&view=list)  
   [![0.0%](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/Duplications/3-16px.png '0.0%')](https://sonarcloud.io/component_measures?id=apache-dolphinscheduler&pullRequest=10640&metric=new_duplicated_lines_density&view=list) [0.0% Duplication](https://sonarcloud.io/component_measures?id=apache-dolphinscheduler&pullRequest=10640&metric=new_duplicated_lines_density&view=list)
   
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@dolphinscheduler.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [dolphinscheduler] aiwenmo commented on pull request #10640: [Feature-10444][Task] Add Dinky task to better support the development and execution of FlinkSQL

Posted by GitBox <gi...@apache.org>.
aiwenmo commented on PR #10640:
URL: https://github.com/apache/dolphinscheduler/pull/10640#issuecomment-1173819823

   Hi. Please review the new docs. thx. @zhongjiajie 


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@dolphinscheduler.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [dolphinscheduler] sonarcloud[bot] commented on pull request #10640: [Feature-10444][Task] Add Dinky task to better support the development and execution of FlinkSQL

Posted by GitBox <gi...@apache.org>.
sonarcloud[bot] commented on PR #10640:
URL: https://github.com/apache/dolphinscheduler/pull/10640#issuecomment-1172434017

   SonarCloud Quality Gate failed.&nbsp; &nbsp; [![Quality Gate failed](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/QualityGateBadge/failed-16px.png 'Quality Gate failed')](https://sonarcloud.io/dashboard?id=apache-dolphinscheduler&pullRequest=10640)
   
   [![Bug](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/bug-16px.png 'Bug')](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=10640&resolved=false&types=BUG) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=10640&resolved=false&types=BUG) [0 Bugs](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=10640&resolved=false&types=BUG)  
   [![Vulnerability](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/vulnerability-16px.png 'Vulnerability')](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=10640&resolved=false&types=VULNERABILITY) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=10640&resolved=false&types=VULNERABILITY) [0 Vulnerabilities](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=10640&resolved=false&types=VULNERABILITY)  
   [![Security Hotspot](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/security_hotspot-16px.png 'Security Hotspot')](https://sonarcloud.io/project/security_hotspots?id=apache-dolphinscheduler&pullRequest=10640&resolved=false&types=SECURITY_HOTSPOT) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/security_hotspots?id=apache-dolphinscheduler&pullRequest=10640&resolved=false&types=SECURITY_HOTSPOT) [0 Security Hotspots](https://sonarcloud.io/project/security_hotspots?id=apache-dolphinscheduler&pullRequest=10640&resolved=false&types=SECURITY_HOTSPOT)  
   [![Code Smell](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/code_smell-16px.png 'Code Smell')](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=10640&resolved=false&types=CODE_SMELL) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=10640&resolved=false&types=CODE_SMELL) [0 Code Smells](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=10640&resolved=false&types=CODE_SMELL)
   
   [![0.0%](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/CoverageChart/0-16px.png '0.0%')](https://sonarcloud.io/component_measures?id=apache-dolphinscheduler&pullRequest=10640&metric=new_coverage&view=list) [0.0% Coverage](https://sonarcloud.io/component_measures?id=apache-dolphinscheduler&pullRequest=10640&metric=new_coverage&view=list)  
   [![0.0%](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/Duplications/3-16px.png '0.0%')](https://sonarcloud.io/component_measures?id=apache-dolphinscheduler&pullRequest=10640&metric=new_duplicated_lines_density&view=list) [0.0% Duplication](https://sonarcloud.io/component_measures?id=apache-dolphinscheduler&pullRequest=10640&metric=new_duplicated_lines_density&view=list)
   
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@dolphinscheduler.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [dolphinscheduler] aiwenmo commented on a diff in pull request #10640: [Feature-10444][Task] Add Dinky task to better support the development and execution of FlinkSQL

Posted by GitBox <gi...@apache.org>.
aiwenmo commented on code in PR #10640:
URL: https://github.com/apache/dolphinscheduler/pull/10640#discussion_r912366997


##########
dolphinscheduler-ui/src/views/projects/task/components/node/tasks/use-dinky.ts:
##########
@@ -0,0 +1,80 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import {reactive} from 'vue'
+import * as Fields from '../fields/index'
+import type {IJsonItem, INodeData, ITaskData} from '../types'
+
+export function useDinky({
+                             projectCode,
+                             from = 0,
+                             readonly,
+                             data
+                         }: {
+    projectCode: number
+    from?: number
+    readonly?: boolean
+    data?: ITaskData
+}) {
+    const model = reactive({
+        name: '',
+        taskType: 'DINKY',
+        flag: 'YES',
+        description: '',
+        timeoutFlag: false,
+        localParams: [],
+        environmentCode: null,
+        failRetryInterval: 1,
+        failRetryTimes: 0,
+        workerGroup: 'default',
+        delayTime: 0,
+        timeout: 30

Review Comment:
   Hi. I have changed the code.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@dolphinscheduler.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [dolphinscheduler] aiwenmo commented on a diff in pull request #10640: [Feature-10444][Task] Add Dinky task to better support the development and execution of FlinkSQL

Posted by GitBox <gi...@apache.org>.
aiwenmo commented on code in PR #10640:
URL: https://github.com/apache/dolphinscheduler/pull/10640#discussion_r909481261


##########
dolphinscheduler-task-plugin/dolphinscheduler-task-dinky/src/main/java/org/apache/dolphinscheduler/plugin/task/dinky/DinkyTask.java:
##########
@@ -0,0 +1,272 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.dolphinscheduler.plugin.task.dinky;
+
+import org.apache.dolphinscheduler.plugin.task.api.AbstractTaskExecutor;
+import org.apache.dolphinscheduler.plugin.task.api.TaskConstants;
+import org.apache.dolphinscheduler.plugin.task.api.TaskExecutionContext;
+import org.apache.dolphinscheduler.plugin.task.api.parameters.AbstractParameters;
+import org.apache.dolphinscheduler.spi.utils.JSONUtils;
+import org.apache.dolphinscheduler.spi.utils.StringUtils;
+
+import org.apache.http.HttpResponse;
+import org.apache.http.HttpStatus;
+import org.apache.http.client.HttpClient;
+import org.apache.http.client.methods.HttpGet;
+import org.apache.http.client.methods.HttpPost;
+import org.apache.http.client.utils.URIBuilder;
+import org.apache.http.entity.StringEntity;
+import org.apache.http.impl.client.HttpClientBuilder;
+import org.apache.http.util.EntityUtils;
+
+import java.net.URI;
+import java.nio.charset.StandardCharsets;
+import java.util.HashMap;
+import java.util.Map;
+
+import com.fasterxml.jackson.core.JsonProcessingException;
+import com.fasterxml.jackson.databind.JsonNode;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.databind.node.MissingNode;
+
+public class DinkyTask extends AbstractTaskExecutor {
+
+    /**
+     * taskExecutionContext
+     */
+    private final TaskExecutionContext taskExecutionContext;
+
+    /**
+     * dinky parameters
+     */
+    private DinkyParameters dinkyParameters;
+
+    /**
+     * constructor
+     *
+     * @param taskExecutionContext taskExecutionContext
+     */
+    protected DinkyTask(TaskExecutionContext taskExecutionContext) {
+        super(taskExecutionContext);
+        this.taskExecutionContext = taskExecutionContext;
+    }
+
+    @Override
+    public void init() {
+        final String taskParams = taskExecutionContext.getTaskParams();
+        logger.info("dinky task params:{}", taskParams);
+        this.dinkyParameters = JSONUtils.parseObject(taskParams, DinkyParameters.class);
+        if (this.dinkyParameters == null || !this.dinkyParameters.checkParameters()) {
+            throw new DinkyTaskException("dinky task params is not valid");
+        }
+    }
+
+    @Override
+    public void handle() throws Exception {
+        try {
+            String address = this.dinkyParameters.getAddress();
+            String taskId = this.dinkyParameters.getTaskId();
+            boolean isOnline = this.dinkyParameters.isOnline();
+            JsonNode result;
+            if (isOnline) {
+                // Online dinky task, and only one job is allowed to execute
+                result = onlineTask(address, taskId);
+            } else {
+                // Submit dinky task
+                result = submitTask(address, taskId);
+            }
+            if (checkResult(result)) {
+                boolean status = result.get(DinkyTaskConstants.API_RESULT_DATAS).get("success").asBoolean();
+                String jobInstanceId = result.get(DinkyTaskConstants.API_RESULT_DATAS).get("jobInstanceId").asText();
+                boolean finishFlag = false;
+                while (!finishFlag) {
+                    JsonNode jobInstanceInfoResult = getJobInstanceInfo(address, jobInstanceId);
+                    if (!checkResult(jobInstanceInfoResult)) {
+                        break;
+                    }
+                    String jobInstanceStatus = jobInstanceInfoResult.get(DinkyTaskConstants.API_RESULT_DATAS).get("status").asText();
+                    switch (jobInstanceStatus) {
+                        case DinkyTaskConstants.STATUS_FINISHED:
+                            final int exitStatusCode = mapStatusToExitCode(status);
+                            // Use address-taskId as app id
+                            setAppIds(String.format("%s-%s", address, taskId));
+                            setExitStatusCode(exitStatusCode);
+                            logger.info("dinky task finished with results: {}", result.get(DinkyTaskConstants.API_RESULT_DATAS));
+                            finishFlag = true;
+                            break;
+                        case DinkyTaskConstants.STATUS_FAILED:
+                        case DinkyTaskConstants.STATUS_CANCELED:
+                        case DinkyTaskConstants.STATUS_UNKNOWN:
+                            errorHandle(jobInstanceInfoResult.get(DinkyTaskConstants.API_RESULT_DATAS).get("error").asText());
+                            finishFlag = true;
+                            break;
+                        default:
+                            Thread.sleep(DinkyTaskConstants.SLEEP_MILLIS);

Review Comment:
   Hi. Need to poll for task status.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@dolphinscheduler.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [dolphinscheduler] zhongjiajie merged pull request #10640: [Feature-10444][Task] Add Dinky task to better support the development and execution of FlinkSQL

Posted by GitBox <gi...@apache.org>.
zhongjiajie merged PR #10640:
URL: https://github.com/apache/dolphinscheduler/pull/10640


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@dolphinscheduler.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [dolphinscheduler] aiwenmo commented on pull request #10640: [Feature-10444][Task] Add Dinky task to better support the development and execution of FlinkSQL

Posted by GitBox <gi...@apache.org>.
aiwenmo commented on PR #10640:
URL: https://github.com/apache/dolphinscheduler/pull/10640#issuecomment-1173272516

   Hi. Please review the new code. thx. @Amy0104 


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@dolphinscheduler.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [dolphinscheduler] SbloodyS commented on pull request #10640: [Feature-10444][Task] Add Dinky task to better support the development and execution of FlinkSQL

Posted by GitBox <gi...@apache.org>.
SbloodyS commented on PR #10640:
URL: https://github.com/apache/dolphinscheduler/pull/10640#issuecomment-1173305191

   @zhongjiajie PTAL.
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@dolphinscheduler.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [dolphinscheduler] sonarcloud[bot] commented on pull request #10640: [Feature-10444][Task] Add Dinky task to better support the development and execution of FlinkSQL

Posted by GitBox <gi...@apache.org>.
sonarcloud[bot] commented on PR #10640:
URL: https://github.com/apache/dolphinscheduler/pull/10640#issuecomment-1169935429

   SonarCloud Quality Gate failed.&nbsp; &nbsp; [![Quality Gate failed](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/QualityGateBadge/failed-16px.png 'Quality Gate failed')](https://sonarcloud.io/dashboard?id=apache-dolphinscheduler&pullRequest=10640)
   
   [![Bug](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/bug-16px.png 'Bug')](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=10640&resolved=false&types=BUG) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=10640&resolved=false&types=BUG) [0 Bugs](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=10640&resolved=false&types=BUG)  
   [![Vulnerability](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/vulnerability-16px.png 'Vulnerability')](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=10640&resolved=false&types=VULNERABILITY) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=10640&resolved=false&types=VULNERABILITY) [0 Vulnerabilities](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=10640&resolved=false&types=VULNERABILITY)  
   [![Security Hotspot](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/security_hotspot-16px.png 'Security Hotspot')](https://sonarcloud.io/project/security_hotspots?id=apache-dolphinscheduler&pullRequest=10640&resolved=false&types=SECURITY_HOTSPOT) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/security_hotspots?id=apache-dolphinscheduler&pullRequest=10640&resolved=false&types=SECURITY_HOTSPOT) [0 Security Hotspots](https://sonarcloud.io/project/security_hotspots?id=apache-dolphinscheduler&pullRequest=10640&resolved=false&types=SECURITY_HOTSPOT)  
   [![Code Smell](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/code_smell-16px.png 'Code Smell')](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=10640&resolved=false&types=CODE_SMELL) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=10640&resolved=false&types=CODE_SMELL) [0 Code Smells](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=10640&resolved=false&types=CODE_SMELL)
   
   [![0.0%](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/CoverageChart/0-16px.png '0.0%')](https://sonarcloud.io/component_measures?id=apache-dolphinscheduler&pullRequest=10640&metric=new_coverage&view=list) [0.0% Coverage](https://sonarcloud.io/component_measures?id=apache-dolphinscheduler&pullRequest=10640&metric=new_coverage&view=list)  
   [![0.0%](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/Duplications/3-16px.png '0.0%')](https://sonarcloud.io/component_measures?id=apache-dolphinscheduler&pullRequest=10640&metric=new_duplicated_lines_density&view=list) [0.0% Duplication](https://sonarcloud.io/component_measures?id=apache-dolphinscheduler&pullRequest=10640&metric=new_duplicated_lines_density&view=list)
   
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@dolphinscheduler.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [dolphinscheduler] caishunfeng commented on a diff in pull request #10640: [Feature-10444][Task] Add Dinky task to better support the development and execution of FlinkSQL

Posted by GitBox <gi...@apache.org>.
caishunfeng commented on code in PR #10640:
URL: https://github.com/apache/dolphinscheduler/pull/10640#discussion_r910558567


##########
dolphinscheduler-task-plugin/dolphinscheduler-task-dinky/src/main/java/org/apache/dolphinscheduler/plugin/task/dinky/DinkyTask.java:
##########
@@ -0,0 +1,272 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.dolphinscheduler.plugin.task.dinky;
+
+import org.apache.dolphinscheduler.plugin.task.api.AbstractTaskExecutor;
+import org.apache.dolphinscheduler.plugin.task.api.TaskConstants;
+import org.apache.dolphinscheduler.plugin.task.api.TaskExecutionContext;
+import org.apache.dolphinscheduler.plugin.task.api.parameters.AbstractParameters;
+import org.apache.dolphinscheduler.spi.utils.JSONUtils;
+import org.apache.dolphinscheduler.spi.utils.StringUtils;
+
+import org.apache.http.HttpResponse;
+import org.apache.http.HttpStatus;
+import org.apache.http.client.HttpClient;
+import org.apache.http.client.methods.HttpGet;
+import org.apache.http.client.methods.HttpPost;
+import org.apache.http.client.utils.URIBuilder;
+import org.apache.http.entity.StringEntity;
+import org.apache.http.impl.client.HttpClientBuilder;
+import org.apache.http.util.EntityUtils;
+
+import java.net.URI;
+import java.nio.charset.StandardCharsets;
+import java.util.HashMap;
+import java.util.Map;
+
+import com.fasterxml.jackson.core.JsonProcessingException;
+import com.fasterxml.jackson.databind.JsonNode;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.databind.node.MissingNode;
+
+public class DinkyTask extends AbstractTaskExecutor {
+
+    /**
+     * taskExecutionContext
+     */
+    private final TaskExecutionContext taskExecutionContext;
+
+    /**
+     * dinky parameters
+     */
+    private DinkyParameters dinkyParameters;
+
+    /**
+     * constructor
+     *
+     * @param taskExecutionContext taskExecutionContext
+     */
+    protected DinkyTask(TaskExecutionContext taskExecutionContext) {
+        super(taskExecutionContext);
+        this.taskExecutionContext = taskExecutionContext;
+    }
+
+    @Override
+    public void init() {
+        final String taskParams = taskExecutionContext.getTaskParams();
+        logger.info("dinky task params:{}", taskParams);
+        this.dinkyParameters = JSONUtils.parseObject(taskParams, DinkyParameters.class);
+        if (this.dinkyParameters == null || !this.dinkyParameters.checkParameters()) {
+            throw new DinkyTaskException("dinky task params is not valid");
+        }
+    }
+
+    @Override
+    public void handle() throws Exception {
+        try {
+            String address = this.dinkyParameters.getAddress();
+            String taskId = this.dinkyParameters.getTaskId();
+            boolean isOnline = this.dinkyParameters.isOnline();
+            JsonNode result;
+            if (isOnline) {
+                // Online dinky task, and only one job is allowed to execute
+                result = onlineTask(address, taskId);
+            } else {
+                // Submit dinky task
+                result = submitTask(address, taskId);
+            }
+            if (checkResult(result)) {
+                boolean status = result.get(DinkyTaskConstants.API_RESULT_DATAS).get("success").asBoolean();
+                String jobInstanceId = result.get(DinkyTaskConstants.API_RESULT_DATAS).get("jobInstanceId").asText();
+                boolean finishFlag = false;
+                while (!finishFlag) {
+                    JsonNode jobInstanceInfoResult = getJobInstanceInfo(address, jobInstanceId);
+                    if (!checkResult(jobInstanceInfoResult)) {
+                        break;
+                    }
+                    String jobInstanceStatus = jobInstanceInfoResult.get(DinkyTaskConstants.API_RESULT_DATAS).get("status").asText();
+                    switch (jobInstanceStatus) {
+                        case DinkyTaskConstants.STATUS_FINISHED:
+                            final int exitStatusCode = mapStatusToExitCode(status);
+                            // Use address-taskId as app id
+                            setAppIds(String.format("%s-%s", address, taskId));
+                            setExitStatusCode(exitStatusCode);
+                            logger.info("dinky task finished with results: {}", result.get(DinkyTaskConstants.API_RESULT_DATAS));
+                            finishFlag = true;
+                            break;
+                        case DinkyTaskConstants.STATUS_FAILED:
+                        case DinkyTaskConstants.STATUS_CANCELED:
+                        case DinkyTaskConstants.STATUS_UNKNOWN:
+                            errorHandle(jobInstanceInfoResult.get(DinkyTaskConstants.API_RESULT_DATAS).get("error").asText());
+                            finishFlag = true;
+                            break;
+                        default:
+                            Thread.sleep(DinkyTaskConstants.SLEEP_MILLIS);
+                    }
+                }
+            }
+        } catch (Exception e) {
+            errorHandle(e);

Review Comment:
   > Hi. Do you mean to remove 'try catch' ?
   
   No, I mean remove the `errorHandle` method and handle error directly. 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@dolphinscheduler.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [dolphinscheduler] Amy0104 commented on a diff in pull request #10640: [Feature-10444][Task] Add Dinky task to better support the development and execution of FlinkSQL

Posted by GitBox <gi...@apache.org>.
Amy0104 commented on code in PR #10640:
URL: https://github.com/apache/dolphinscheduler/pull/10640#discussion_r911585938


##########
dolphinscheduler-ui/src/views/projects/task/components/node/tasks/use-dinky.ts:
##########
@@ -0,0 +1,80 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import {reactive} from 'vue'
+import * as Fields from '../fields/index'
+import type {IJsonItem, INodeData, ITaskData} from '../types'
+
+export function useDinky({
+                             projectCode,
+                             from = 0,
+                             readonly,
+                             data
+                         }: {
+    projectCode: number
+    from?: number
+    readonly?: boolean
+    data?: ITaskData
+}) {
+    const model = reactive({
+        name: '',
+        taskType: 'DINKY',
+        flag: 'YES',
+        description: '',
+        timeoutFlag: false,
+        localParams: [],
+        environmentCode: null,
+        failRetryInterval: 1,
+        failRetryTimes: 0,
+        workerGroup: 'default',
+        delayTime: 0,
+        timeout: 30

Review Comment:
   Here should set the timeoutNotifyStrategy default value to ['WARN'].
   plx check this #10259 .



##########
dolphinscheduler-ui/src/views/projects/task/components/node/tasks/use-dinky.ts:
##########
@@ -0,0 +1,80 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import {reactive} from 'vue'
+import * as Fields from '../fields/index'
+import type {IJsonItem, INodeData, ITaskData} from '../types'
+
+export function useDinky({
+                             projectCode,
+                             from = 0,
+                             readonly,
+                             data
+                         }: {

Review Comment:
   It's better to format the codes by running 'npm run prettier'.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@dolphinscheduler.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [dolphinscheduler] sonarcloud[bot] commented on pull request #10640: [Feature-10444][Task] Add Dinky task to better support the development and execution of FlinkSQL

Posted by GitBox <gi...@apache.org>.
sonarcloud[bot] commented on PR #10640:
URL: https://github.com/apache/dolphinscheduler/pull/10640#issuecomment-1170700518

   SonarCloud Quality Gate failed.&nbsp; &nbsp; [![Quality Gate failed](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/QualityGateBadge/failed-16px.png 'Quality Gate failed')](https://sonarcloud.io/dashboard?id=apache-dolphinscheduler&pullRequest=10640)
   
   [![Bug](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/bug-16px.png 'Bug')](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=10640&resolved=false&types=BUG) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=10640&resolved=false&types=BUG) [0 Bugs](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=10640&resolved=false&types=BUG)  
   [![Vulnerability](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/vulnerability-16px.png 'Vulnerability')](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=10640&resolved=false&types=VULNERABILITY) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=10640&resolved=false&types=VULNERABILITY) [0 Vulnerabilities](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=10640&resolved=false&types=VULNERABILITY)  
   [![Security Hotspot](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/security_hotspot-16px.png 'Security Hotspot')](https://sonarcloud.io/project/security_hotspots?id=apache-dolphinscheduler&pullRequest=10640&resolved=false&types=SECURITY_HOTSPOT) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/security_hotspots?id=apache-dolphinscheduler&pullRequest=10640&resolved=false&types=SECURITY_HOTSPOT) [0 Security Hotspots](https://sonarcloud.io/project/security_hotspots?id=apache-dolphinscheduler&pullRequest=10640&resolved=false&types=SECURITY_HOTSPOT)  
   [![Code Smell](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/code_smell-16px.png 'Code Smell')](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=10640&resolved=false&types=CODE_SMELL) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=10640&resolved=false&types=CODE_SMELL) [0 Code Smells](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=10640&resolved=false&types=CODE_SMELL)
   
   [![0.0%](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/CoverageChart/0-16px.png '0.0%')](https://sonarcloud.io/component_measures?id=apache-dolphinscheduler&pullRequest=10640&metric=new_coverage&view=list) [0.0% Coverage](https://sonarcloud.io/component_measures?id=apache-dolphinscheduler&pullRequest=10640&metric=new_coverage&view=list)  
   [![0.0%](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/Duplications/3-16px.png '0.0%')](https://sonarcloud.io/component_measures?id=apache-dolphinscheduler&pullRequest=10640&metric=new_duplicated_lines_density&view=list) [0.0% Duplication](https://sonarcloud.io/component_measures?id=apache-dolphinscheduler&pullRequest=10640&metric=new_duplicated_lines_density&view=list)
   
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@dolphinscheduler.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [dolphinscheduler] sonarcloud[bot] commented on pull request #10640: [Feature-10444][Task] Add Dinky task to better support the development and execution of FlinkSQL

Posted by GitBox <gi...@apache.org>.
sonarcloud[bot] commented on PR #10640:
URL: https://github.com/apache/dolphinscheduler/pull/10640#issuecomment-1172435680

   SonarCloud Quality Gate failed.&nbsp; &nbsp; [![Quality Gate failed](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/QualityGateBadge/failed-16px.png 'Quality Gate failed')](https://sonarcloud.io/dashboard?id=apache-dolphinscheduler&pullRequest=10640)
   
   [![Bug](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/bug-16px.png 'Bug')](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=10640&resolved=false&types=BUG) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=10640&resolved=false&types=BUG) [0 Bugs](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=10640&resolved=false&types=BUG)  
   [![Vulnerability](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/vulnerability-16px.png 'Vulnerability')](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=10640&resolved=false&types=VULNERABILITY) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=10640&resolved=false&types=VULNERABILITY) [0 Vulnerabilities](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=10640&resolved=false&types=VULNERABILITY)  
   [![Security Hotspot](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/security_hotspot-16px.png 'Security Hotspot')](https://sonarcloud.io/project/security_hotspots?id=apache-dolphinscheduler&pullRequest=10640&resolved=false&types=SECURITY_HOTSPOT) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/security_hotspots?id=apache-dolphinscheduler&pullRequest=10640&resolved=false&types=SECURITY_HOTSPOT) [0 Security Hotspots](https://sonarcloud.io/project/security_hotspots?id=apache-dolphinscheduler&pullRequest=10640&resolved=false&types=SECURITY_HOTSPOT)  
   [![Code Smell](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/code_smell-16px.png 'Code Smell')](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=10640&resolved=false&types=CODE_SMELL) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=10640&resolved=false&types=CODE_SMELL) [0 Code Smells](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=10640&resolved=false&types=CODE_SMELL)
   
   [![0.0%](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/CoverageChart/0-16px.png '0.0%')](https://sonarcloud.io/component_measures?id=apache-dolphinscheduler&pullRequest=10640&metric=new_coverage&view=list) [0.0% Coverage](https://sonarcloud.io/component_measures?id=apache-dolphinscheduler&pullRequest=10640&metric=new_coverage&view=list)  
   [![0.0%](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/Duplications/3-16px.png '0.0%')](https://sonarcloud.io/component_measures?id=apache-dolphinscheduler&pullRequest=10640&metric=new_duplicated_lines_density&view=list) [0.0% Duplication](https://sonarcloud.io/component_measures?id=apache-dolphinscheduler&pullRequest=10640&metric=new_duplicated_lines_density&view=list)
   
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@dolphinscheduler.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [dolphinscheduler] sonarcloud[bot] commented on pull request #10640: [Feature-10444][Task] Add Dinky task to better support the development and execution of FlinkSQL

Posted by GitBox <gi...@apache.org>.
sonarcloud[bot] commented on PR #10640:
URL: https://github.com/apache/dolphinscheduler/pull/10640#issuecomment-1173797323

   SonarCloud Quality Gate failed.&nbsp; &nbsp; [![Quality Gate failed](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/QualityGateBadge/failed-16px.png 'Quality Gate failed')](https://sonarcloud.io/dashboard?id=apache-dolphinscheduler&pullRequest=10640)
   
   [![Bug](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/bug-16px.png 'Bug')](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=10640&resolved=false&types=BUG) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=10640&resolved=false&types=BUG) [0 Bugs](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=10640&resolved=false&types=BUG)  
   [![Vulnerability](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/vulnerability-16px.png 'Vulnerability')](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=10640&resolved=false&types=VULNERABILITY) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=10640&resolved=false&types=VULNERABILITY) [0 Vulnerabilities](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=10640&resolved=false&types=VULNERABILITY)  
   [![Security Hotspot](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/security_hotspot-16px.png 'Security Hotspot')](https://sonarcloud.io/project/security_hotspots?id=apache-dolphinscheduler&pullRequest=10640&resolved=false&types=SECURITY_HOTSPOT) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/security_hotspots?id=apache-dolphinscheduler&pullRequest=10640&resolved=false&types=SECURITY_HOTSPOT) [0 Security Hotspots](https://sonarcloud.io/project/security_hotspots?id=apache-dolphinscheduler&pullRequest=10640&resolved=false&types=SECURITY_HOTSPOT)  
   [![Code Smell](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/code_smell-16px.png 'Code Smell')](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=10640&resolved=false&types=CODE_SMELL) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=10640&resolved=false&types=CODE_SMELL) [0 Code Smells](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=10640&resolved=false&types=CODE_SMELL)
   
   [![0.0%](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/CoverageChart/0-16px.png '0.0%')](https://sonarcloud.io/component_measures?id=apache-dolphinscheduler&pullRequest=10640&metric=new_coverage&view=list) [0.0% Coverage](https://sonarcloud.io/component_measures?id=apache-dolphinscheduler&pullRequest=10640&metric=new_coverage&view=list)  
   [![0.0%](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/Duplications/3-16px.png '0.0%')](https://sonarcloud.io/component_measures?id=apache-dolphinscheduler&pullRequest=10640&metric=new_duplicated_lines_density&view=list) [0.0% Duplication](https://sonarcloud.io/component_measures?id=apache-dolphinscheduler&pullRequest=10640&metric=new_duplicated_lines_density&view=list)
   
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@dolphinscheduler.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [dolphinscheduler] sonarcloud[bot] commented on pull request #10640: [Feature-10444][Task] Add Dinky task to better support the development and execution of FlinkSQL

Posted by GitBox <gi...@apache.org>.
sonarcloud[bot] commented on PR #10640:
URL: https://github.com/apache/dolphinscheduler/pull/10640#issuecomment-1169848074

   SonarCloud Quality Gate failed.&nbsp; &nbsp; [![Quality Gate failed](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/QualityGateBadge/failed-16px.png 'Quality Gate failed')](https://sonarcloud.io/dashboard?id=apache-dolphinscheduler&pullRequest=10640)
   
   [![Bug](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/bug-16px.png 'Bug')](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=10640&resolved=false&types=BUG) [![C](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/C-16px.png 'C')](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=10640&resolved=false&types=BUG) [1 Bug](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=10640&resolved=false&types=BUG)  
   [![Vulnerability](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/vulnerability-16px.png 'Vulnerability')](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=10640&resolved=false&types=VULNERABILITY) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=10640&resolved=false&types=VULNERABILITY) [0 Vulnerabilities](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=10640&resolved=false&types=VULNERABILITY)  
   [![Security Hotspot](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/security_hotspot-16px.png 'Security Hotspot')](https://sonarcloud.io/project/security_hotspots?id=apache-dolphinscheduler&pullRequest=10640&resolved=false&types=SECURITY_HOTSPOT) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/security_hotspots?id=apache-dolphinscheduler&pullRequest=10640&resolved=false&types=SECURITY_HOTSPOT) [0 Security Hotspots](https://sonarcloud.io/project/security_hotspots?id=apache-dolphinscheduler&pullRequest=10640&resolved=false&types=SECURITY_HOTSPOT)  
   [![Code Smell](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/code_smell-16px.png 'Code Smell')](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=10640&resolved=false&types=CODE_SMELL) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=10640&resolved=false&types=CODE_SMELL) [0 Code Smells](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=10640&resolved=false&types=CODE_SMELL)
   
   [![0.0%](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/CoverageChart/0-16px.png '0.0%')](https://sonarcloud.io/component_measures?id=apache-dolphinscheduler&pullRequest=10640&metric=new_coverage&view=list) [0.0% Coverage](https://sonarcloud.io/component_measures?id=apache-dolphinscheduler&pullRequest=10640&metric=new_coverage&view=list)  
   [![0.0%](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/Duplications/3-16px.png '0.0%')](https://sonarcloud.io/component_measures?id=apache-dolphinscheduler&pullRequest=10640&metric=new_duplicated_lines_density&view=list) [0.0% Duplication](https://sonarcloud.io/component_measures?id=apache-dolphinscheduler&pullRequest=10640&metric=new_duplicated_lines_density&view=list)
   
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@dolphinscheduler.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [dolphinscheduler] sonarcloud[bot] commented on pull request #10640: [Feature-10444][Task] Add Dinky task to better support the development and execution of FlinkSQL

Posted by GitBox <gi...@apache.org>.
sonarcloud[bot] commented on PR #10640:
URL: https://github.com/apache/dolphinscheduler/pull/10640#issuecomment-1169852560

   SonarCloud Quality Gate failed.&nbsp; &nbsp; [![Quality Gate failed](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/QualityGateBadge/failed-16px.png 'Quality Gate failed')](https://sonarcloud.io/dashboard?id=apache-dolphinscheduler&pullRequest=10640)
   
   [![Bug](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/bug-16px.png 'Bug')](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=10640&resolved=false&types=BUG) [![C](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/C-16px.png 'C')](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=10640&resolved=false&types=BUG) [1 Bug](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=10640&resolved=false&types=BUG)  
   [![Vulnerability](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/vulnerability-16px.png 'Vulnerability')](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=10640&resolved=false&types=VULNERABILITY) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=10640&resolved=false&types=VULNERABILITY) [0 Vulnerabilities](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=10640&resolved=false&types=VULNERABILITY)  
   [![Security Hotspot](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/security_hotspot-16px.png 'Security Hotspot')](https://sonarcloud.io/project/security_hotspots?id=apache-dolphinscheduler&pullRequest=10640&resolved=false&types=SECURITY_HOTSPOT) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/security_hotspots?id=apache-dolphinscheduler&pullRequest=10640&resolved=false&types=SECURITY_HOTSPOT) [0 Security Hotspots](https://sonarcloud.io/project/security_hotspots?id=apache-dolphinscheduler&pullRequest=10640&resolved=false&types=SECURITY_HOTSPOT)  
   [![Code Smell](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/code_smell-16px.png 'Code Smell')](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=10640&resolved=false&types=CODE_SMELL) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=10640&resolved=false&types=CODE_SMELL) [0 Code Smells](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=10640&resolved=false&types=CODE_SMELL)
   
   [![0.0%](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/CoverageChart/0-16px.png '0.0%')](https://sonarcloud.io/component_measures?id=apache-dolphinscheduler&pullRequest=10640&metric=new_coverage&view=list) [0.0% Coverage](https://sonarcloud.io/component_measures?id=apache-dolphinscheduler&pullRequest=10640&metric=new_coverage&view=list)  
   [![0.0%](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/Duplications/3-16px.png '0.0%')](https://sonarcloud.io/component_measures?id=apache-dolphinscheduler&pullRequest=10640&metric=new_duplicated_lines_density&view=list) [0.0% Duplication](https://sonarcloud.io/component_measures?id=apache-dolphinscheduler&pullRequest=10640&metric=new_duplicated_lines_density&view=list)
   
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@dolphinscheduler.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [dolphinscheduler] SbloodyS commented on a diff in pull request #10640: [Feature-10444][Task] Add Dinky task to better support the development and execution of FlinkSQL

Posted by GitBox <gi...@apache.org>.
SbloodyS commented on code in PR #10640:
URL: https://github.com/apache/dolphinscheduler/pull/10640#discussion_r911062412


##########
dolphinscheduler-task-plugin/dolphinscheduler-task-dinky/src/main/java/org/apache/dolphinscheduler/plugin/task/dinky/DinkyTaskChannel.java:
##########
@@ -0,0 +1,49 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.dolphinscheduler.plugin.task.dinky;
+
+import org.apache.dolphinscheduler.plugin.task.api.AbstractTask;
+import org.apache.dolphinscheduler.plugin.task.api.TaskChannel;
+import org.apache.dolphinscheduler.plugin.task.api.TaskExecutionContext;
+import org.apache.dolphinscheduler.plugin.task.api.parameters.AbstractParameters;
+import org.apache.dolphinscheduler.plugin.task.api.parameters.ParametersNode;
+import org.apache.dolphinscheduler.plugin.task.api.parameters.resource.ResourceParametersHelper;
+import org.apache.dolphinscheduler.spi.utils.JSONUtils;
+
+public class DinkyTaskChannel implements TaskChannel {
+
+    @Override
+    public void cancelApplication(boolean status) {
+        // nothing to do

Review Comment:
   Why not implement this?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@dolphinscheduler.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [dolphinscheduler] sonarcloud[bot] commented on pull request #10640: [Feature-10444][Task] Add Dinky task to better support the development and execution of FlinkSQL

Posted by GitBox <gi...@apache.org>.
sonarcloud[bot] commented on PR #10640:
URL: https://github.com/apache/dolphinscheduler/pull/10640#issuecomment-1168753468

   SonarCloud Quality Gate failed.&nbsp; &nbsp; [![Quality Gate failed](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/QualityGateBadge/failed-16px.png 'Quality Gate failed')](https://sonarcloud.io/dashboard?id=apache-dolphinscheduler&pullRequest=10640)
   
   [![Bug](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/bug-16px.png 'Bug')](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=10640&resolved=false&types=BUG) [![C](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/C-16px.png 'C')](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=10640&resolved=false&types=BUG) [3 Bugs](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=10640&resolved=false&types=BUG)  
   [![Vulnerability](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/vulnerability-16px.png 'Vulnerability')](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=10640&resolved=false&types=VULNERABILITY) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=10640&resolved=false&types=VULNERABILITY) [0 Vulnerabilities](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=10640&resolved=false&types=VULNERABILITY)  
   [![Security Hotspot](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/security_hotspot-16px.png 'Security Hotspot')](https://sonarcloud.io/project/security_hotspots?id=apache-dolphinscheduler&pullRequest=10640&resolved=false&types=SECURITY_HOTSPOT) [![E](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/E-16px.png 'E')](https://sonarcloud.io/project/security_hotspots?id=apache-dolphinscheduler&pullRequest=10640&resolved=false&types=SECURITY_HOTSPOT) [1 Security Hotspot](https://sonarcloud.io/project/security_hotspots?id=apache-dolphinscheduler&pullRequest=10640&resolved=false&types=SECURITY_HOTSPOT)  
   [![Code Smell](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/code_smell-16px.png 'Code Smell')](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=10640&resolved=false&types=CODE_SMELL) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=10640&resolved=false&types=CODE_SMELL) [14 Code Smells](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=10640&resolved=false&types=CODE_SMELL)
   
   [![0.0%](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/CoverageChart/0-16px.png '0.0%')](https://sonarcloud.io/component_measures?id=apache-dolphinscheduler&pullRequest=10640&metric=new_coverage&view=list) [0.0% Coverage](https://sonarcloud.io/component_measures?id=apache-dolphinscheduler&pullRequest=10640&metric=new_coverage&view=list)  
   [![3.1%](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/Duplications/5-16px.png '3.1%')](https://sonarcloud.io/component_measures?id=apache-dolphinscheduler&pullRequest=10640&metric=new_duplicated_lines_density&view=list) [3.1% Duplication](https://sonarcloud.io/component_measures?id=apache-dolphinscheduler&pullRequest=10640&metric=new_duplicated_lines_density&view=list)
   
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@dolphinscheduler.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [dolphinscheduler] sonarcloud[bot] commented on pull request #10640: [Feature-10444][Task] Add Dinky task to better support the development and execution of FlinkSQL

Posted by GitBox <gi...@apache.org>.
sonarcloud[bot] commented on PR #10640:
URL: https://github.com/apache/dolphinscheduler/pull/10640#issuecomment-1168930600

   SonarCloud Quality Gate failed.&nbsp; &nbsp; [![Quality Gate failed](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/QualityGateBadge/failed-16px.png 'Quality Gate failed')](https://sonarcloud.io/dashboard?id=apache-dolphinscheduler&pullRequest=10640)
   
   [![Bug](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/bug-16px.png 'Bug')](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=10640&resolved=false&types=BUG) [![C](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/C-16px.png 'C')](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=10640&resolved=false&types=BUG) [1 Bug](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=10640&resolved=false&types=BUG)  
   [![Vulnerability](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/vulnerability-16px.png 'Vulnerability')](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=10640&resolved=false&types=VULNERABILITY) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=10640&resolved=false&types=VULNERABILITY) [0 Vulnerabilities](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=10640&resolved=false&types=VULNERABILITY)  
   [![Security Hotspot](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/security_hotspot-16px.png 'Security Hotspot')](https://sonarcloud.io/project/security_hotspots?id=apache-dolphinscheduler&pullRequest=10640&resolved=false&types=SECURITY_HOTSPOT) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/security_hotspots?id=apache-dolphinscheduler&pullRequest=10640&resolved=false&types=SECURITY_HOTSPOT) [0 Security Hotspots](https://sonarcloud.io/project/security_hotspots?id=apache-dolphinscheduler&pullRequest=10640&resolved=false&types=SECURITY_HOTSPOT)  
   [![Code Smell](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/code_smell-16px.png 'Code Smell')](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=10640&resolved=false&types=CODE_SMELL) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=10640&resolved=false&types=CODE_SMELL) [1 Code Smell](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=10640&resolved=false&types=CODE_SMELL)
   
   [![0.0%](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/CoverageChart/0-16px.png '0.0%')](https://sonarcloud.io/component_measures?id=apache-dolphinscheduler&pullRequest=10640&metric=new_coverage&view=list) [0.0% Coverage](https://sonarcloud.io/component_measures?id=apache-dolphinscheduler&pullRequest=10640&metric=new_coverage&view=list)  
   [![0.0%](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/Duplications/3-16px.png '0.0%')](https://sonarcloud.io/component_measures?id=apache-dolphinscheduler&pullRequest=10640&metric=new_duplicated_lines_density&view=list) [0.0% Duplication](https://sonarcloud.io/component_measures?id=apache-dolphinscheduler&pullRequest=10640&metric=new_duplicated_lines_density&view=list)
   
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@dolphinscheduler.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [dolphinscheduler] aiwenmo commented on a diff in pull request #10640: [Feature-10444][Task] Add Dinky task to better support the development and execution of FlinkSQL

Posted by GitBox <gi...@apache.org>.
aiwenmo commented on code in PR #10640:
URL: https://github.com/apache/dolphinscheduler/pull/10640#discussion_r909486384


##########
dolphinscheduler-task-plugin/dolphinscheduler-task-dinky/src/main/java/org/apache/dolphinscheduler/plugin/task/dinky/DinkyTask.java:
##########
@@ -0,0 +1,272 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.dolphinscheduler.plugin.task.dinky;
+
+import org.apache.dolphinscheduler.plugin.task.api.AbstractTaskExecutor;
+import org.apache.dolphinscheduler.plugin.task.api.TaskConstants;
+import org.apache.dolphinscheduler.plugin.task.api.TaskExecutionContext;
+import org.apache.dolphinscheduler.plugin.task.api.parameters.AbstractParameters;
+import org.apache.dolphinscheduler.spi.utils.JSONUtils;
+import org.apache.dolphinscheduler.spi.utils.StringUtils;
+
+import org.apache.http.HttpResponse;
+import org.apache.http.HttpStatus;
+import org.apache.http.client.HttpClient;
+import org.apache.http.client.methods.HttpGet;
+import org.apache.http.client.methods.HttpPost;
+import org.apache.http.client.utils.URIBuilder;
+import org.apache.http.entity.StringEntity;
+import org.apache.http.impl.client.HttpClientBuilder;
+import org.apache.http.util.EntityUtils;
+
+import java.net.URI;
+import java.nio.charset.StandardCharsets;
+import java.util.HashMap;
+import java.util.Map;
+
+import com.fasterxml.jackson.core.JsonProcessingException;
+import com.fasterxml.jackson.databind.JsonNode;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.databind.node.MissingNode;
+
+public class DinkyTask extends AbstractTaskExecutor {
+
+    /**
+     * taskExecutionContext
+     */
+    private final TaskExecutionContext taskExecutionContext;
+
+    /**
+     * dinky parameters
+     */
+    private DinkyParameters dinkyParameters;
+
+    /**
+     * constructor
+     *
+     * @param taskExecutionContext taskExecutionContext
+     */
+    protected DinkyTask(TaskExecutionContext taskExecutionContext) {
+        super(taskExecutionContext);
+        this.taskExecutionContext = taskExecutionContext;
+    }
+
+    @Override
+    public void init() {
+        final String taskParams = taskExecutionContext.getTaskParams();
+        logger.info("dinky task params:{}", taskParams);
+        this.dinkyParameters = JSONUtils.parseObject(taskParams, DinkyParameters.class);
+        if (this.dinkyParameters == null || !this.dinkyParameters.checkParameters()) {
+            throw new DinkyTaskException("dinky task params is not valid");
+        }
+    }
+
+    @Override
+    public void handle() throws Exception {
+        try {
+            String address = this.dinkyParameters.getAddress();
+            String taskId = this.dinkyParameters.getTaskId();
+            boolean isOnline = this.dinkyParameters.isOnline();
+            JsonNode result;
+            if (isOnline) {
+                // Online dinky task, and only one job is allowed to execute
+                result = onlineTask(address, taskId);
+            } else {
+                // Submit dinky task
+                result = submitTask(address, taskId);
+            }
+            if (checkResult(result)) {
+                boolean status = result.get(DinkyTaskConstants.API_RESULT_DATAS).get("success").asBoolean();
+                String jobInstanceId = result.get(DinkyTaskConstants.API_RESULT_DATAS).get("jobInstanceId").asText();
+                boolean finishFlag = false;
+                while (!finishFlag) {
+                    JsonNode jobInstanceInfoResult = getJobInstanceInfo(address, jobInstanceId);
+                    if (!checkResult(jobInstanceInfoResult)) {
+                        break;
+                    }
+                    String jobInstanceStatus = jobInstanceInfoResult.get(DinkyTaskConstants.API_RESULT_DATAS).get("status").asText();
+                    switch (jobInstanceStatus) {
+                        case DinkyTaskConstants.STATUS_FINISHED:
+                            final int exitStatusCode = mapStatusToExitCode(status);
+                            // Use address-taskId as app id
+                            setAppIds(String.format("%s-%s", address, taskId));
+                            setExitStatusCode(exitStatusCode);
+                            logger.info("dinky task finished with results: {}", result.get(DinkyTaskConstants.API_RESULT_DATAS));
+                            finishFlag = true;
+                            break;
+                        case DinkyTaskConstants.STATUS_FAILED:
+                        case DinkyTaskConstants.STATUS_CANCELED:
+                        case DinkyTaskConstants.STATUS_UNKNOWN:
+                            errorHandle(jobInstanceInfoResult.get(DinkyTaskConstants.API_RESULT_DATAS).get("error").asText());
+                            finishFlag = true;
+                            break;
+                        default:
+                            Thread.sleep(DinkyTaskConstants.SLEEP_MILLIS);
+                    }
+                }
+            }
+        } catch (Exception e) {
+            errorHandle(e);
+        }
+    }
+
+    /**
+     * map dinky task status to exitStatusCode
+     *
+     * @param status dinky job status
+     * @return exitStatusCode
+     */
+    private int mapStatusToExitCode(boolean status) {
+        if (status) {
+            return TaskConstants.EXIT_CODE_SUCCESS;
+        } else {
+            return TaskConstants.EXIT_CODE_FAILURE;
+        }
+    }
+
+    private boolean checkResult(JsonNode result) {
+        if (result instanceof MissingNode || result == null) {
+            errorHandle(DinkyTaskConstants.API_VERSION_ERROR_TIPS);
+            return false;
+        } else if (result.get("code").asInt() == DinkyTaskConstants.API_ERROR) {
+            errorHandle(result.get("msg"));
+            return false;
+        }
+        return true;
+    }
+
+    private void errorHandle(Object msg) {
+        setExitStatusCode(TaskConstants.EXIT_CODE_FAILURE);
+        logger.error("dinky task submit failed with error: {}", msg);
+    }
+
+    @Override
+    public AbstractParameters getParameters() {
+        return dinkyParameters;
+    }
+
+    @Override
+    public void cancelApplication(boolean status) throws Exception {
+        super.cancelApplication(status);
+        String address = this.dinkyParameters.getAddress();
+        String taskId = this.dinkyParameters.getTaskId();
+        logger.info("trying terminate dinky task, taskId: {}, address: {}, taskId: {}",
+            this.taskExecutionContext.getTaskInstanceId(),
+            address,
+            taskId);
+        cancelTask(address, taskId);
+        logger.info("dinky task terminated, taskId: {}, address: {}, taskId: {}",
+            this.taskExecutionContext.getTaskInstanceId(),
+            address,
+            taskId);
+    }
+
+    private JsonNode submitTask(String address, String taskId) {
+        Map<String, String> params = new HashMap<>();
+        params.put(DinkyTaskConstants.PARAM_TASK_ID, taskId);
+        return parse(doGet(address + DinkyTaskConstants.SUBMIT_TASK, params));
+    }
+
+    private JsonNode onlineTask(String address, String taskId) {
+        Map<String, String> params = new HashMap<>();
+        params.put(DinkyTaskConstants.PARAM_TASK_ID, taskId);
+        return parse(doGet(address + DinkyTaskConstants.ONLINE_TASK, params));
+    }
+
+    private JsonNode cancelTask(String address, String taskId) {
+        Map<String, String> params = new HashMap<>();
+        params.put(DinkyTaskConstants.PARAM_JSON_TASK_ID, taskId);
+        params.put(DinkyTaskConstants.PARAM_SAVEPOINT_TYPE, DinkyTaskConstants.SAVEPOINT_CANCEL);
+        return parse(sendJsonStr(address + DinkyTaskConstants.SAVEPOINT_TASK, JSONUtils.toJsonString(params)));
+    }
+
+    private JsonNode getJobInstanceInfo(String address, String taskId) {
+        Map<String, String> params = new HashMap<>();
+        params.put(DinkyTaskConstants.PARAM_JOB_INSTANCE_ID, taskId);
+        return parse(doGet(address + DinkyTaskConstants.GET_JOB_INFO, params));
+    }
+
+    private JsonNode parse(String res) {
+        ObjectMapper mapper = new ObjectMapper();
+        JsonNode result = null;
+        try {
+            result = mapper.readTree(res);
+        } catch (JsonProcessingException e) {
+            logger.error("dinky task submit failed with error", e);
+        }
+        return result;
+    }
+
+    private String doGet(String url, Map<String, String> params) {
+        String result = "";
+        HttpClient httpClient = HttpClientBuilder.create().build();
+        HttpGet httpGet = null;
+        try {
+            URIBuilder uriBuilder = new URIBuilder(url);
+            if (null != params && !params.isEmpty()) {
+                for (Map.Entry<String, String> entry : params.entrySet()) {
+                    uriBuilder.addParameter(entry.getKey(), entry.getValue());
+                }
+            }
+            URI uri = uriBuilder.build();
+            httpGet = new HttpGet(uri);
+            logger.info("access url: {}", uri);
+            HttpResponse response = httpClient.execute(httpGet);
+            if (response.getStatusLine().getStatusCode() == HttpStatus.SC_OK) {
+                result = EntityUtils.toString(response.getEntity());
+                logger.info("dinky task succeed with results: {}", result);
+            } else {
+                logger.info("dinky task terminated,response: {}", response);

Review Comment:
   It's a good advice. thx



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@dolphinscheduler.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [dolphinscheduler] zhongjiajie commented on a diff in pull request #10640: [Feature-10444][Task] Add Dinky task to better support the development and execution of FlinkSQL

Posted by GitBox <gi...@apache.org>.
zhongjiajie commented on code in PR #10640:
URL: https://github.com/apache/dolphinscheduler/pull/10640#discussion_r912646958


##########
docs/docs/en/guide/task/dinky.md:
##########
@@ -0,0 +1,36 @@
+# Dinky
+
+## Overview
+
+Use `Dinky Task` to create a dinky-type task and support one-stop development, debugging, operation and maintenance of FlinkSql, Flink jar and SQL. When the worker executes `Dinky Task`,
+it will call `Dinky API` to trigger dinky task. Click [here](http://www.dlink.top/) for details about `Dinky`.
+
+## Create Task
+
+- Click Project Management-Project Name-Workflow Definition, and click the "Create Workflow" button to enter the DAG editing page.
+- Drag <img src="../../../../img/tasks/icons/dinky.png" width="15"/> from the toolbar to the canvas.
+
+## Task Parameter
+
+- Node name: The node name in a workflow definition is unique.
+- Run flag: Identifies whether this node can be scheduled normally, if it does not need to be executed, you can turn on the prohibition switch.
+- Descriptive information: Describe the function of the node.
+- Task priority: When the number of worker threads is insufficient, execute in the order of priority from high to low, and tasks with the same priority will execute in a first-in first-out order.
+- Worker grouping: Assign tasks to the machines of the worker group to execute. If `Default` is selected, randomly select a worker machine for execution.
+- Number of failed retry attempts: The failure task resubmitting times. It supports drop-down and hand-filling.
+- Failed retry interval: The time interval for resubmitting the task after a failed task. It supports drop-down and hand-filling.
+- Timeout alarm: Check the timeout alarm and timeout failure. When the task exceeds the "timeout period", an alarm email will send and the task execution will fail.
+- Dinky Address: The url for a dinky server.
+- Dinky Task ID: The unique task id for a dinky task.
+- Online Task: Specify whether the current dinky job is online. If yes, the submitted job can only be submitted successfully when it is published and there is no corresponding Flink job instance running.

Review Comment:
   currently, all our English version tasks parameters convert from list style to table format style, which you can see example in https://github.com/apache/dolphinscheduler/blob/dev/docs/docs/en/guide/task/shell.md.  So please change to a table format in English version of docs. And it is up to you whether change Chinese version of docs



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@dolphinscheduler.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [dolphinscheduler] sonarcloud[bot] commented on pull request #10640: [Feature-10444][Task] Add Dinky task to better support the development and execution of FlinkSQL

Posted by GitBox <gi...@apache.org>.
sonarcloud[bot] commented on PR #10640:
URL: https://github.com/apache/dolphinscheduler/pull/10640#issuecomment-1168925370

   SonarCloud Quality Gate failed.&nbsp; &nbsp; [![Quality Gate failed](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/QualityGateBadge/failed-16px.png 'Quality Gate failed')](https://sonarcloud.io/dashboard?id=apache-dolphinscheduler&pullRequest=10640)
   
   [![Bug](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/bug-16px.png 'Bug')](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=10640&resolved=false&types=BUG) [![C](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/C-16px.png 'C')](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=10640&resolved=false&types=BUG) [1 Bug](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=10640&resolved=false&types=BUG)  
   [![Vulnerability](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/vulnerability-16px.png 'Vulnerability')](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=10640&resolved=false&types=VULNERABILITY) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=10640&resolved=false&types=VULNERABILITY) [0 Vulnerabilities](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=10640&resolved=false&types=VULNERABILITY)  
   [![Security Hotspot](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/security_hotspot-16px.png 'Security Hotspot')](https://sonarcloud.io/project/security_hotspots?id=apache-dolphinscheduler&pullRequest=10640&resolved=false&types=SECURITY_HOTSPOT) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/security_hotspots?id=apache-dolphinscheduler&pullRequest=10640&resolved=false&types=SECURITY_HOTSPOT) [0 Security Hotspots](https://sonarcloud.io/project/security_hotspots?id=apache-dolphinscheduler&pullRequest=10640&resolved=false&types=SECURITY_HOTSPOT)  
   [![Code Smell](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/code_smell-16px.png 'Code Smell')](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=10640&resolved=false&types=CODE_SMELL) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=10640&resolved=false&types=CODE_SMELL) [1 Code Smell](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=10640&resolved=false&types=CODE_SMELL)
   
   [![0.0%](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/CoverageChart/0-16px.png '0.0%')](https://sonarcloud.io/component_measures?id=apache-dolphinscheduler&pullRequest=10640&metric=new_coverage&view=list) [0.0% Coverage](https://sonarcloud.io/component_measures?id=apache-dolphinscheduler&pullRequest=10640&metric=new_coverage&view=list)  
   [![0.0%](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/Duplications/3-16px.png '0.0%')](https://sonarcloud.io/component_measures?id=apache-dolphinscheduler&pullRequest=10640&metric=new_duplicated_lines_density&view=list) [0.0% Duplication](https://sonarcloud.io/component_measures?id=apache-dolphinscheduler&pullRequest=10640&metric=new_duplicated_lines_density&view=list)
   
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@dolphinscheduler.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [dolphinscheduler] codecov-commenter commented on pull request #10640: [Feature-10444][Task] Add Dinky task to better support the development and execution of FlinkSQL

Posted by GitBox <gi...@apache.org>.
codecov-commenter commented on PR #10640:
URL: https://github.com/apache/dolphinscheduler/pull/10640#issuecomment-1168927532

   # [Codecov](https://codecov.io/gh/apache/dolphinscheduler/pull/10640?src=pr&el=h1&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) Report
   > Merging [#10640](https://codecov.io/gh/apache/dolphinscheduler/pull/10640?src=pr&el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (464ac6e) into [dev](https://codecov.io/gh/apache/dolphinscheduler/commit/719a9d4532733c0d7b4a54ed52b15dca7982ca8d?el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (719a9d4) will **decrease** coverage by `0.04%`.
   > The diff coverage is `n/a`.
   
   ```diff
   @@             Coverage Diff              @@
   ##                dev   #10640      +/-   ##
   ============================================
   - Coverage     40.94%   40.90%   -0.05%     
   + Complexity     4876     4830      -46     
   ============================================
     Files           895      897       +2     
     Lines         36212    36158      -54     
     Branches       3987     4006      +19     
   ============================================
   - Hits          14826    14789      -37     
   + Misses        19921    19898      -23     
   - Partials       1465     1471       +6     
   ```
   
   
   | [Impacted Files](https://codecov.io/gh/apache/dolphinscheduler/pull/10640?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | Coverage Δ | |
   |---|---|---|
   | [...inscheduler/server/master/config/MasterConfig.java](https://codecov.io/gh/apache/dolphinscheduler/pull/10640/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-ZG9scGhpbnNjaGVkdWxlci1tYXN0ZXIvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2RvbHBoaW5zY2hlZHVsZXIvc2VydmVyL21hc3Rlci9jb25maWcvTWFzdGVyQ29uZmlnLmphdmE=) | `35.00% <0.00%> (-41.75%)` | :arrow_down: |
   | [...inscheduler/server/worker/config/WorkerConfig.java](https://codecov.io/gh/apache/dolphinscheduler/pull/10640/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-ZG9scGhpbnNjaGVkdWxlci13b3JrZXIvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2RvbHBoaW5zY2hlZHVsZXIvc2VydmVyL3dvcmtlci9jb25maWcvV29ya2VyQ29uZmlnLmphdmE=) | `52.38% <0.00%> (-24.09%)` | :arrow_down: |
   | [.../dolphinscheduler/common/utils/ParameterUtils.java](https://codecov.io/gh/apache/dolphinscheduler/pull/10640/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-ZG9scGhpbnNjaGVkdWxlci1jb21tb24vc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2RvbHBoaW5zY2hlZHVsZXIvY29tbW9uL3V0aWxzL1BhcmFtZXRlclV0aWxzLmphdmE=) | `58.53% <0.00%> (-14.60%)` | :arrow_down: |
   | [...cheduler/plugin/task/api/TaskExecutionContext.java](https://codecov.io/gh/apache/dolphinscheduler/pull/10640/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-ZG9scGhpbnNjaGVkdWxlci10YXNrLXBsdWdpbi9kb2xwaGluc2NoZWR1bGVyLXRhc2stYXBpL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9kb2xwaGluc2NoZWR1bGVyL3BsdWdpbi90YXNrL2FwaS9UYXNrRXhlY3V0aW9uQ29udGV4dC5qYXZh) | `93.87% <0.00%> (-5.38%)` | :arrow_down: |
   | [...ache/impl/ProcessInstanceExecCacheManagerImpl.java](https://codecov.io/gh/apache/dolphinscheduler/pull/10640/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-ZG9scGhpbnNjaGVkdWxlci1tYXN0ZXIvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2RvbHBoaW5zY2hlZHVsZXIvc2VydmVyL21hc3Rlci9jYWNoZS9pbXBsL1Byb2Nlc3NJbnN0YW5jZUV4ZWNDYWNoZU1hbmFnZXJJbXBsLmphdmE=) | `83.33% <0.00%> (-1.29%)` | :arrow_down: |
   | [...heduler/server/master/service/FailoverService.java](https://codecov.io/gh/apache/dolphinscheduler/pull/10640/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-ZG9scGhpbnNjaGVkdWxlci1tYXN0ZXIvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2RvbHBoaW5zY2hlZHVsZXIvc2VydmVyL21hc3Rlci9zZXJ2aWNlL0ZhaWxvdmVyU2VydmljZS5qYXZh) | `63.80% <0.00%> (-1.10%)` | :arrow_down: |
   | [...he/dolphinscheduler/server/utils/ProcessUtils.java](https://codecov.io/gh/apache/dolphinscheduler/pull/10640/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-ZG9scGhpbnNjaGVkdWxlci1zZXJ2ZXIvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2RvbHBoaW5zY2hlZHVsZXIvc2VydmVyL3V0aWxzL1Byb2Nlc3NVdGlscy5qYXZh) | `28.23% <0.00%> (-1.04%)` | :arrow_down: |
   | [...olphinscheduler/plugin/alert/email/MailSender.java](https://codecov.io/gh/apache/dolphinscheduler/pull/10640/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-ZG9scGhpbnNjaGVkdWxlci1hbGVydC9kb2xwaGluc2NoZWR1bGVyLWFsZXJ0LXBsdWdpbnMvZG9scGhpbnNjaGVkdWxlci1hbGVydC1lbWFpbC9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvZG9scGhpbnNjaGVkdWxlci9wbHVnaW4vYWxlcnQvZW1haWwvTWFpbFNlbmRlci5qYXZh) | `51.03% <0.00%> (-0.36%)` | :arrow_down: |
   | [...r/api/service/impl/WorkFlowLineageServiceImpl.java](https://codecov.io/gh/apache/dolphinscheduler/pull/10640/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-ZG9scGhpbnNjaGVkdWxlci1hcGkvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2RvbHBoaW5zY2hlZHVsZXIvYXBpL3NlcnZpY2UvaW1wbC9Xb3JrRmxvd0xpbmVhZ2VTZXJ2aWNlSW1wbC5qYXZh) | `33.01% <0.00%> (-0.32%)` | :arrow_down: |
   | [...nscheduler/service/process/ProcessServiceImpl.java](https://codecov.io/gh/apache/dolphinscheduler/pull/10640/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-ZG9scGhpbnNjaGVkdWxlci1zZXJ2aWNlL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9kb2xwaGluc2NoZWR1bGVyL3NlcnZpY2UvcHJvY2Vzcy9Qcm9jZXNzU2VydmljZUltcGwuamF2YQ==) | `30.28% <0.00%> (-0.30%)` | :arrow_down: |
   | ... and [34 more](https://codecov.io/gh/apache/dolphinscheduler/pull/10640/diff?src=pr&el=tree-more&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/dolphinscheduler/pull/10640?src=pr&el=continue&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/dolphinscheduler/pull/10640?src=pr&el=footer&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Last update [719a9d4...464ac6e](https://codecov.io/gh/apache/dolphinscheduler/pull/10640?src=pr&el=lastupdated&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation).
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@dolphinscheduler.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [dolphinscheduler] sonarcloud[bot] commented on pull request #10640: [Feature-10444][Task] Add Dinky task to better support the development and execution of FlinkSQL

Posted by GitBox <gi...@apache.org>.
sonarcloud[bot] commented on PR #10640:
URL: https://github.com/apache/dolphinscheduler/pull/10640#issuecomment-1168568664

   SonarCloud Quality Gate failed.&nbsp; &nbsp; [![Quality Gate failed](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/QualityGateBadge/failed-16px.png 'Quality Gate failed')](https://sonarcloud.io/dashboard?id=apache-dolphinscheduler&pullRequest=10640)
   
   [![Bug](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/bug-16px.png 'Bug')](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=10640&resolved=false&types=BUG) [![C](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/C-16px.png 'C')](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=10640&resolved=false&types=BUG) [3 Bugs](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=10640&resolved=false&types=BUG)  
   [![Vulnerability](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/vulnerability-16px.png 'Vulnerability')](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=10640&resolved=false&types=VULNERABILITY) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=10640&resolved=false&types=VULNERABILITY) [0 Vulnerabilities](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=10640&resolved=false&types=VULNERABILITY)  
   [![Security Hotspot](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/security_hotspot-16px.png 'Security Hotspot')](https://sonarcloud.io/project/security_hotspots?id=apache-dolphinscheduler&pullRequest=10640&resolved=false&types=SECURITY_HOTSPOT) [![E](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/E-16px.png 'E')](https://sonarcloud.io/project/security_hotspots?id=apache-dolphinscheduler&pullRequest=10640&resolved=false&types=SECURITY_HOTSPOT) [1 Security Hotspot](https://sonarcloud.io/project/security_hotspots?id=apache-dolphinscheduler&pullRequest=10640&resolved=false&types=SECURITY_HOTSPOT)  
   [![Code Smell](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/code_smell-16px.png 'Code Smell')](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=10640&resolved=false&types=CODE_SMELL) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=10640&resolved=false&types=CODE_SMELL) [14 Code Smells](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=10640&resolved=false&types=CODE_SMELL)
   
   [![0.0%](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/CoverageChart/0-16px.png '0.0%')](https://sonarcloud.io/component_measures?id=apache-dolphinscheduler&pullRequest=10640&metric=new_coverage&view=list) [0.0% Coverage](https://sonarcloud.io/component_measures?id=apache-dolphinscheduler&pullRequest=10640&metric=new_coverage&view=list)  
   [![3.1%](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/Duplications/5-16px.png '3.1%')](https://sonarcloud.io/component_measures?id=apache-dolphinscheduler&pullRequest=10640&metric=new_duplicated_lines_density&view=list) [3.1% Duplication](https://sonarcloud.io/component_measures?id=apache-dolphinscheduler&pullRequest=10640&metric=new_duplicated_lines_density&view=list)
   
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@dolphinscheduler.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [dolphinscheduler] sonarcloud[bot] commented on pull request #10640: [Feature-10444][Task] Add Dinky task to better support the development and execution of FlinkSQL

Posted by GitBox <gi...@apache.org>.
sonarcloud[bot] commented on PR #10640:
URL: https://github.com/apache/dolphinscheduler/pull/10640#issuecomment-1168750996

   SonarCloud Quality Gate failed.&nbsp; &nbsp; [![Quality Gate failed](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/QualityGateBadge/failed-16px.png 'Quality Gate failed')](https://sonarcloud.io/dashboard?id=apache-dolphinscheduler&pullRequest=10640)
   
   [![Bug](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/bug-16px.png 'Bug')](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=10640&resolved=false&types=BUG) [![C](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/C-16px.png 'C')](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=10640&resolved=false&types=BUG) [3 Bugs](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=10640&resolved=false&types=BUG)  
   [![Vulnerability](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/vulnerability-16px.png 'Vulnerability')](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=10640&resolved=false&types=VULNERABILITY) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=10640&resolved=false&types=VULNERABILITY) [0 Vulnerabilities](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=10640&resolved=false&types=VULNERABILITY)  
   [![Security Hotspot](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/security_hotspot-16px.png 'Security Hotspot')](https://sonarcloud.io/project/security_hotspots?id=apache-dolphinscheduler&pullRequest=10640&resolved=false&types=SECURITY_HOTSPOT) [![E](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/E-16px.png 'E')](https://sonarcloud.io/project/security_hotspots?id=apache-dolphinscheduler&pullRequest=10640&resolved=false&types=SECURITY_HOTSPOT) [1 Security Hotspot](https://sonarcloud.io/project/security_hotspots?id=apache-dolphinscheduler&pullRequest=10640&resolved=false&types=SECURITY_HOTSPOT)  
   [![Code Smell](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/code_smell-16px.png 'Code Smell')](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=10640&resolved=false&types=CODE_SMELL) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=10640&resolved=false&types=CODE_SMELL) [14 Code Smells](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=10640&resolved=false&types=CODE_SMELL)
   
   [![0.0%](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/CoverageChart/0-16px.png '0.0%')](https://sonarcloud.io/component_measures?id=apache-dolphinscheduler&pullRequest=10640&metric=new_coverage&view=list) [0.0% Coverage](https://sonarcloud.io/component_measures?id=apache-dolphinscheduler&pullRequest=10640&metric=new_coverage&view=list)  
   [![3.1%](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/Duplications/5-16px.png '3.1%')](https://sonarcloud.io/component_measures?id=apache-dolphinscheduler&pullRequest=10640&metric=new_duplicated_lines_density&view=list) [3.1% Duplication](https://sonarcloud.io/component_measures?id=apache-dolphinscheduler&pullRequest=10640&metric=new_duplicated_lines_density&view=list)
   
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@dolphinscheduler.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [dolphinscheduler] sonarcloud[bot] commented on pull request #10640: [Feature-10444][Task] Add Dinky task to better support the development and execution of FlinkSQL

Posted by GitBox <gi...@apache.org>.
sonarcloud[bot] commented on PR #10640:
URL: https://github.com/apache/dolphinscheduler/pull/10640#issuecomment-1168565753

   SonarCloud Quality Gate failed.&nbsp; &nbsp; [![Quality Gate failed](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/QualityGateBadge/failed-16px.png 'Quality Gate failed')](https://sonarcloud.io/dashboard?id=apache-dolphinscheduler&pullRequest=10640)
   
   [![Bug](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/bug-16px.png 'Bug')](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=10640&resolved=false&types=BUG) [![C](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/C-16px.png 'C')](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=10640&resolved=false&types=BUG) [3 Bugs](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=10640&resolved=false&types=BUG)  
   [![Vulnerability](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/vulnerability-16px.png 'Vulnerability')](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=10640&resolved=false&types=VULNERABILITY) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=10640&resolved=false&types=VULNERABILITY) [0 Vulnerabilities](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=10640&resolved=false&types=VULNERABILITY)  
   [![Security Hotspot](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/security_hotspot-16px.png 'Security Hotspot')](https://sonarcloud.io/project/security_hotspots?id=apache-dolphinscheduler&pullRequest=10640&resolved=false&types=SECURITY_HOTSPOT) [![E](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/E-16px.png 'E')](https://sonarcloud.io/project/security_hotspots?id=apache-dolphinscheduler&pullRequest=10640&resolved=false&types=SECURITY_HOTSPOT) [1 Security Hotspot](https://sonarcloud.io/project/security_hotspots?id=apache-dolphinscheduler&pullRequest=10640&resolved=false&types=SECURITY_HOTSPOT)  
   [![Code Smell](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/code_smell-16px.png 'Code Smell')](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=10640&resolved=false&types=CODE_SMELL) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=10640&resolved=false&types=CODE_SMELL) [14 Code Smells](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=10640&resolved=false&types=CODE_SMELL)
   
   [![0.0%](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/CoverageChart/0-16px.png '0.0%')](https://sonarcloud.io/component_measures?id=apache-dolphinscheduler&pullRequest=10640&metric=new_coverage&view=list) [0.0% Coverage](https://sonarcloud.io/component_measures?id=apache-dolphinscheduler&pullRequest=10640&metric=new_coverage&view=list)  
   [![3.1%](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/Duplications/5-16px.png '3.1%')](https://sonarcloud.io/component_measures?id=apache-dolphinscheduler&pullRequest=10640&metric=new_duplicated_lines_density&view=list) [3.1% Duplication](https://sonarcloud.io/component_measures?id=apache-dolphinscheduler&pullRequest=10640&metric=new_duplicated_lines_density&view=list)
   
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@dolphinscheduler.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [dolphinscheduler] aiwenmo commented on pull request #10640: [Feature-10444][Task] Add Dinky task to better support the development and execution of FlinkSQL

Posted by GitBox <gi...@apache.org>.
aiwenmo commented on PR #10640:
URL: https://github.com/apache/dolphinscheduler/pull/10640#issuecomment-1168730003

   demo-get-dinky-task-id(../../../../img/tasks/demo/dinky_task_id.png)
   At present, Dinky is not internationalized, and English pictures will be added later.
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@dolphinscheduler.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [dolphinscheduler] sonarcloud[bot] commented on pull request #10640: [Feature-10444][Task] Add Dinky task to better support the development and execution of FlinkSQL

Posted by GitBox <gi...@apache.org>.
sonarcloud[bot] commented on PR #10640:
URL: https://github.com/apache/dolphinscheduler/pull/10640#issuecomment-1169935534

   Please retry analysis of this Pull-Request directly on [SonarCloud](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=10640).


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@dolphinscheduler.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [dolphinscheduler] caishunfeng commented on a diff in pull request #10640: [Feature-10444][Task] Add Dinky task to better support the development and execution of FlinkSQL

Posted by GitBox <gi...@apache.org>.
caishunfeng commented on code in PR #10640:
URL: https://github.com/apache/dolphinscheduler/pull/10640#discussion_r910558567


##########
dolphinscheduler-task-plugin/dolphinscheduler-task-dinky/src/main/java/org/apache/dolphinscheduler/plugin/task/dinky/DinkyTask.java:
##########
@@ -0,0 +1,272 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.dolphinscheduler.plugin.task.dinky;
+
+import org.apache.dolphinscheduler.plugin.task.api.AbstractTaskExecutor;
+import org.apache.dolphinscheduler.plugin.task.api.TaskConstants;
+import org.apache.dolphinscheduler.plugin.task.api.TaskExecutionContext;
+import org.apache.dolphinscheduler.plugin.task.api.parameters.AbstractParameters;
+import org.apache.dolphinscheduler.spi.utils.JSONUtils;
+import org.apache.dolphinscheduler.spi.utils.StringUtils;
+
+import org.apache.http.HttpResponse;
+import org.apache.http.HttpStatus;
+import org.apache.http.client.HttpClient;
+import org.apache.http.client.methods.HttpGet;
+import org.apache.http.client.methods.HttpPost;
+import org.apache.http.client.utils.URIBuilder;
+import org.apache.http.entity.StringEntity;
+import org.apache.http.impl.client.HttpClientBuilder;
+import org.apache.http.util.EntityUtils;
+
+import java.net.URI;
+import java.nio.charset.StandardCharsets;
+import java.util.HashMap;
+import java.util.Map;
+
+import com.fasterxml.jackson.core.JsonProcessingException;
+import com.fasterxml.jackson.databind.JsonNode;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.databind.node.MissingNode;
+
+public class DinkyTask extends AbstractTaskExecutor {
+
+    /**
+     * taskExecutionContext
+     */
+    private final TaskExecutionContext taskExecutionContext;
+
+    /**
+     * dinky parameters
+     */
+    private DinkyParameters dinkyParameters;
+
+    /**
+     * constructor
+     *
+     * @param taskExecutionContext taskExecutionContext
+     */
+    protected DinkyTask(TaskExecutionContext taskExecutionContext) {
+        super(taskExecutionContext);
+        this.taskExecutionContext = taskExecutionContext;
+    }
+
+    @Override
+    public void init() {
+        final String taskParams = taskExecutionContext.getTaskParams();
+        logger.info("dinky task params:{}", taskParams);
+        this.dinkyParameters = JSONUtils.parseObject(taskParams, DinkyParameters.class);
+        if (this.dinkyParameters == null || !this.dinkyParameters.checkParameters()) {
+            throw new DinkyTaskException("dinky task params is not valid");
+        }
+    }
+
+    @Override
+    public void handle() throws Exception {
+        try {
+            String address = this.dinkyParameters.getAddress();
+            String taskId = this.dinkyParameters.getTaskId();
+            boolean isOnline = this.dinkyParameters.isOnline();
+            JsonNode result;
+            if (isOnline) {
+                // Online dinky task, and only one job is allowed to execute
+                result = onlineTask(address, taskId);
+            } else {
+                // Submit dinky task
+                result = submitTask(address, taskId);
+            }
+            if (checkResult(result)) {
+                boolean status = result.get(DinkyTaskConstants.API_RESULT_DATAS).get("success").asBoolean();
+                String jobInstanceId = result.get(DinkyTaskConstants.API_RESULT_DATAS).get("jobInstanceId").asText();
+                boolean finishFlag = false;
+                while (!finishFlag) {
+                    JsonNode jobInstanceInfoResult = getJobInstanceInfo(address, jobInstanceId);
+                    if (!checkResult(jobInstanceInfoResult)) {
+                        break;
+                    }
+                    String jobInstanceStatus = jobInstanceInfoResult.get(DinkyTaskConstants.API_RESULT_DATAS).get("status").asText();
+                    switch (jobInstanceStatus) {
+                        case DinkyTaskConstants.STATUS_FINISHED:
+                            final int exitStatusCode = mapStatusToExitCode(status);
+                            // Use address-taskId as app id
+                            setAppIds(String.format("%s-%s", address, taskId));
+                            setExitStatusCode(exitStatusCode);
+                            logger.info("dinky task finished with results: {}", result.get(DinkyTaskConstants.API_RESULT_DATAS));
+                            finishFlag = true;
+                            break;
+                        case DinkyTaskConstants.STATUS_FAILED:
+                        case DinkyTaskConstants.STATUS_CANCELED:
+                        case DinkyTaskConstants.STATUS_UNKNOWN:
+                            errorHandle(jobInstanceInfoResult.get(DinkyTaskConstants.API_RESULT_DATAS).get("error").asText());
+                            finishFlag = true;
+                            break;
+                        default:
+                            Thread.sleep(DinkyTaskConstants.SLEEP_MILLIS);
+                    }
+                }
+            }
+        } catch (Exception e) {
+            errorHandle(e);

Review Comment:
   > Hi. Do you mean to remove 'try catch' ?
   
   No, I mean remove the `errorHandle` method and handle error directly.  Just a suggestion, nip.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@dolphinscheduler.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [dolphinscheduler] sonarcloud[bot] commented on pull request #10640: [Feature-10444][Task] Add Dinky task to better support the development and execution of FlinkSQL

Posted by GitBox <gi...@apache.org>.
sonarcloud[bot] commented on PR #10640:
URL: https://github.com/apache/dolphinscheduler/pull/10640#issuecomment-1173797799

   SonarCloud Quality Gate failed.&nbsp; &nbsp; [![Quality Gate failed](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/QualityGateBadge/failed-16px.png 'Quality Gate failed')](https://sonarcloud.io/dashboard?id=apache-dolphinscheduler&pullRequest=10640)
   
   [![Bug](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/bug-16px.png 'Bug')](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=10640&resolved=false&types=BUG) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=10640&resolved=false&types=BUG) [0 Bugs](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=10640&resolved=false&types=BUG)  
   [![Vulnerability](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/vulnerability-16px.png 'Vulnerability')](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=10640&resolved=false&types=VULNERABILITY) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=10640&resolved=false&types=VULNERABILITY) [0 Vulnerabilities](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=10640&resolved=false&types=VULNERABILITY)  
   [![Security Hotspot](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/security_hotspot-16px.png 'Security Hotspot')](https://sonarcloud.io/project/security_hotspots?id=apache-dolphinscheduler&pullRequest=10640&resolved=false&types=SECURITY_HOTSPOT) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/security_hotspots?id=apache-dolphinscheduler&pullRequest=10640&resolved=false&types=SECURITY_HOTSPOT) [0 Security Hotspots](https://sonarcloud.io/project/security_hotspots?id=apache-dolphinscheduler&pullRequest=10640&resolved=false&types=SECURITY_HOTSPOT)  
   [![Code Smell](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/code_smell-16px.png 'Code Smell')](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=10640&resolved=false&types=CODE_SMELL) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=10640&resolved=false&types=CODE_SMELL) [0 Code Smells](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=10640&resolved=false&types=CODE_SMELL)
   
   [![0.0%](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/CoverageChart/0-16px.png '0.0%')](https://sonarcloud.io/component_measures?id=apache-dolphinscheduler&pullRequest=10640&metric=new_coverage&view=list) [0.0% Coverage](https://sonarcloud.io/component_measures?id=apache-dolphinscheduler&pullRequest=10640&metric=new_coverage&view=list)  
   [![0.0%](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/Duplications/3-16px.png '0.0%')](https://sonarcloud.io/component_measures?id=apache-dolphinscheduler&pullRequest=10640&metric=new_duplicated_lines_density&view=list) [0.0% Duplication](https://sonarcloud.io/component_measures?id=apache-dolphinscheduler&pullRequest=10640&metric=new_duplicated_lines_density&view=list)
   
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@dolphinscheduler.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [dolphinscheduler] zhongjiajie commented on pull request #10640: [Task] Add Dinky task to better support the development and execution of FlinkSQL

Posted by GitBox <gi...@apache.org>.
zhongjiajie commented on PR #10640:
URL: https://github.com/apache/dolphinscheduler/pull/10640#issuecomment-1174523974

   Thanks for bringing this up


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@dolphinscheduler.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org