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/08/22 10:49:13 UTC

[GitHub] [dolphinscheduler] fengjian1129 opened a new pull request, #11598: [Improvement-#9680][task-plugin] add hive sql log listener, print MapReduce progress log and obtain application_id allows developers to better track tasks

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

   <!--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
   
   <!--(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.* -->
   
   (or)
   
   If your pull request contain incompatible change, you should also add it to `docs/docs/en/guide/upgrede/incompatible.md`
   


-- 
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 #11598: [Improvement-#9680][task-plugin] add hive sql log listener, print MapReduce progress log and obtain application_id allows developers to better track tasks

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

   To target specific issue from PR, we recommend use keyword fix: #issue_id or close: #issue_id or closes: #issue_id in your PR describe(not title, just desc). It would not only connect issue to PR but also close issue automatically when PR is be closed. @fengjian1129 


-- 
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] EricGao888 commented on a diff in pull request #11598: [Improvement-#9680][task-plugin] add hive sql log listener, print MapReduce progress log and obtain application_id allows developers to better track tasks

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


##########
dolphinscheduler-task-plugin/dolphinscheduler-task-sql/src/test/java/org/apache/dolphinscheduler/plugin/task/sql/HiveSqlLogThreadTest.java:
##########
@@ -0,0 +1,68 @@
+/*
+ * 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.sql;
+
+import org.apache.dolphinscheduler.plugin.task.api.TaskExecutionContext;
+
+import org.apache.hive.jdbc.HiveStatement;
+import java.sql.SQLException;
+import java.util.ArrayList;
+import java.util.List;
+import org.junit.Assert;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.powermock.api.mockito.PowerMockito;
+import org.powermock.core.classloader.annotations.PowerMockIgnore;
+import org.powermock.modules.junit4.PowerMockRunner;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+@RunWith(PowerMockRunner.class)

Review Comment:
   Better avoid using `Powermock`



##########
dolphinscheduler-task-plugin/dolphinscheduler-task-sql/src/main/java/org/apache/dolphinscheduler/plugin/task/sql/HiveSqlLogThread.java:
##########
@@ -0,0 +1,71 @@
+/*
+ * 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.sql;
+
+import org.apache.dolphinscheduler.common.utils.LoggerUtils;
+import org.apache.dolphinscheduler.plugin.task.api.TaskExecutionContext;
+
+import org.apache.hive.jdbc.HiveStatement;
+
+import java.sql.SQLException;
+import java.sql.Statement;
+import java.util.List;
+
+import org.slf4j.Logger;
+
+
+public class HiveSqlLogThread extends Thread {
+
+    private final HiveStatement statement;
+
+    private final Logger hiveMapReduceLogger;
+
+    private final TaskExecutionContext taskExecutionContext;
+
+    public HiveSqlLogThread(Statement statement, Logger logger, TaskExecutionContext taskExecutionContext) {
+        this.statement = (HiveStatement) statement;
+        this.hiveMapReduceLogger = logger;
+        this.taskExecutionContext = taskExecutionContext;
+    }
+
+    @Override
+    public void run() {
+        if (statement == null) {
+            hiveMapReduceLogger.info("hive statement is null, end this log query!");
+            return;
+        }
+        try {
+            while (!statement.isClosed() && statement.hasMoreLogs()) {
+                for (String log : statement.getQueryLog(true, 500)) {
+
+                    hiveMapReduceLogger.info(log);
+
+                    List<String> appIds = LoggerUtils.getAppIds(log, hiveMapReduceLogger);
+                    //get sql task yarn's application_id

Review Comment:
   ```suggestion
                       // get sql task yarn's application_id
   ```



-- 
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] fengjian1129 commented on a diff in pull request #11598: [Improvement-#9680][task-plugin] #9680 add hive sql log listener, print MapReduce progress log and obtain application_id allows developers to better track tasks

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


##########
dolphinscheduler-task-plugin/dolphinscheduler-task-sql/src/main/java/org/apache/dolphinscheduler/plugin/task/sql/HiveSqlLogThread.java:
##########
@@ -0,0 +1,82 @@
+/*
+ * 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.sql;
+
+import org.apache.dolphinscheduler.common.utils.LoggerUtils;
+import org.apache.dolphinscheduler.plugin.task.api.TaskExecutionContext;
+import org.apache.dolphinscheduler.plugin.task.api.utils.LogUtils;
+
+import org.apache.hive.jdbc.HivePreparedStatement;
+
+import java.sql.PreparedStatement;
+import java.sql.SQLException;
+import java.util.List;
+
+import org.slf4j.Logger;
+
+
+public class HiveSqlLogThread extends Thread {
+
+    private final HivePreparedStatement statement;
+
+    private final Logger hiveMapReduceLogger;
+
+    private final TaskExecutionContext taskExecutionContext;
+
+    public HiveSqlLogThread(PreparedStatement statement, Logger logger, TaskExecutionContext taskExecutionContext) throws SQLException {
+        this.statement = statement.unwrap(HivePreparedStatement.class);
+        this.hiveMapReduceLogger = logger;
+        this.taskExecutionContext = taskExecutionContext;
+    }
+
+    @Override
+    public void run() {
+        String taskLogName = LoggerUtils.buildTaskId(taskExecutionContext.getFirstSubmitTime(),
+            taskExecutionContext.getProcessDefineCode(),
+            taskExecutionContext.getProcessDefineVersion(),
+            taskExecutionContext.getProcessInstanceId(),
+            taskExecutionContext.getTaskInstanceId());
+        taskExecutionContext.setTaskLogName(taskLogName);
+
+        // set the name of the current thread
+        Thread.currentThread().setName(taskLogName);

Review Comment:
   <img width="782" alt="image" src="https://user-images.githubusercontent.com/35831367/189047832-a7ddbe6b-b667-4328-b35a-c9972983e283.png">
   This is printed with the task log,If you do not set a unique taskLogName for this thread, the log content cannot be displayed on the web page



-- 
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] ruanwenjun commented on a diff in pull request #11598: [Improvement-#9680][task-plugin] #9680 add hive sql log listener, print MapReduce progress log and obtain application_id allows developers to better track tasks

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


##########
dolphinscheduler-task-plugin/dolphinscheduler-task-sql/src/main/java/org/apache/dolphinscheduler/plugin/task/sql/SqlTask.java:
##########
@@ -306,13 +319,31 @@ private String executeUpdate(Connection connection, List<SqlBinds> statementsBin
         int result = 0;
         for (SqlBinds sqlBind : statementsBinds) {
             try (PreparedStatement statement = prepareStatementAndBind(connection, sqlBind)) {
+
+                sqlLogListener(statement);
+
                 result = statement.executeUpdate();
                 logger.info("{} statement execute update result: {}, for sql: {}", handlerType, result, sqlBind.getSql());
             }
         }
         return String.valueOf(result);
     }
 
+    /**
+     * sql log listener
+     *
+     * @param statement statement
+     * @throws SQLException sqlException
+     */
+    private void sqlLogListener(PreparedStatement statement) throws SQLException {
+        if (DbType.HIVE == DbType.valueOf(sqlParameters.getType())) {

Review Comment:
   ```suggestion
           if (DbType.HIVE.name() == sqlParameters.getType()) {
   ```



##########
dolphinscheduler-task-plugin/dolphinscheduler-task-sql/src/main/java/org/apache/dolphinscheduler/plugin/task/sql/HiveSqlLogThread.java:
##########
@@ -0,0 +1,75 @@
+/*
+ * 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.sql;
+
+import org.apache.dolphinscheduler.plugin.task.api.TaskExecutionContext;
+import org.apache.dolphinscheduler.plugin.task.api.utils.LogUtils;
+
+import org.apache.hive.jdbc.HivePreparedStatement;
+
+import java.sql.PreparedStatement;
+import java.sql.SQLException;
+import java.util.List;
+
+import org.slf4j.Logger;
+
+
+public class HiveSqlLogThread extends Thread {
+
+    private final HivePreparedStatement statement;
+
+    private final Logger hiveMapReduceLogger;
+
+    private final TaskExecutionContext taskExecutionContext;
+
+    public HiveSqlLogThread(PreparedStatement statement, Logger logger, TaskExecutionContext taskExecutionContext) throws SQLException {
+        super.setDaemon(true);
+        this.statement = statement.unwrap(HivePreparedStatement.class);
+        this.hiveMapReduceLogger = logger;
+        this.taskExecutionContext = taskExecutionContext;
+    }
+
+    @Override
+    public void run() {
+
+        Thread.currentThread().setName(taskExecutionContext.getTaskLogName());
+
+        if (statement == null) {
+            hiveMapReduceLogger.info("hive statement is null, end this log query!");
+            return;
+        }
+        try {
+            while (!statement.isClosed() && statement.hasMoreLogs()) {
+                for (String log : statement.getQueryLog(true, 500)) {
+
+                    hiveMapReduceLogger.info(log);
+
+                    List<String> appIds = LogUtils.getAppIds(log, hiveMapReduceLogger);
+                    //get sql task yarn's application_id
+                    if (!appIds.isEmpty()) {
+                        hiveMapReduceLogger.info("yarn application_id is {}",appIds);
+                        taskExecutionContext.setAppIds(String.join(",", appIds));
+                    }
+                }
+            }
+        } catch (SQLException e) {
+            hiveMapReduceLogger.error("Failed to view hive log,exception:[{}]", e.getMessage());

Review Comment:
   ```suggestion
               hiveMapReduceLogger.error("Failed to view hive log,exception:[{}]");
   ```



##########
dolphinscheduler-task-plugin/dolphinscheduler-task-sql/src/main/java/org/apache/dolphinscheduler/plugin/task/sql/SqlTask.java:
##########
@@ -306,13 +319,31 @@ private String executeUpdate(Connection connection, List<SqlBinds> statementsBin
         int result = 0;
         for (SqlBinds sqlBind : statementsBinds) {
             try (PreparedStatement statement = prepareStatementAndBind(connection, sqlBind)) {
+
+                sqlLogListener(statement);
+
                 result = statement.executeUpdate();
                 logger.info("{} statement execute update result: {}, for sql: {}", handlerType, result, sqlBind.getSql());
             }
         }
         return String.valueOf(result);
     }
 
+    /**
+     * sql log listener
+     *
+     * @param statement statement
+     * @throws SQLException sqlException
+     */
+    private void sqlLogListener(PreparedStatement statement) throws SQLException {
+        if (DbType.HIVE == DbType.valueOf(sqlParameters.getType())) {
+            logger.info("The current sql task type is [{}], will start a demon thread to resolve the appIds", DbType.HIVE);

Review Comment:
   ```suggestion
               logger.info("The current sql task type is HIVE, will start a daemon thread to resolve the appIds");
   ```



##########
dolphinscheduler-task-plugin/dolphinscheduler-task-sql/src/main/java/org/apache/dolphinscheduler/plugin/task/sql/SqlTask.java:
##########
@@ -156,6 +158,14 @@ public void handle(TaskCallBack taskCallBack) throws TaskException {
             // execute sql task
             executeFuncAndSql(mainStatementSqlBinds, preStatementSqlBinds, postStatementSqlBinds, createFuncs);
 
+            // get appIds
+            Set<String> appIds = LogUtils.getAppIdsFromLogFile(taskExecutionContext.getLogPath());
+            logger.info("Resolve appIds: [{}] from task log: [{}]", appIds, taskExecutionContext.getLogPath());
+            if (!appIds.isEmpty()) {
+                taskExecutionContext.setAppIds(String.join(",", appIds));
+                setAppIds(String.join(",", appIds));

Review Comment:
   ```suggestion
                   String appIdString = String.join(",", appIds);
                   taskExecutionContext.setAppIds(appIdString);
                   setAppIds(appIdString);
   ```



##########
dolphinscheduler-task-plugin/dolphinscheduler-task-sql/src/main/java/org/apache/dolphinscheduler/plugin/task/sql/HiveSqlLogThread.java:
##########
@@ -0,0 +1,75 @@
+/*
+ * 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.sql;
+
+import org.apache.dolphinscheduler.plugin.task.api.TaskExecutionContext;
+import org.apache.dolphinscheduler.plugin.task.api.utils.LogUtils;
+
+import org.apache.hive.jdbc.HivePreparedStatement;
+
+import java.sql.PreparedStatement;
+import java.sql.SQLException;
+import java.util.List;
+
+import org.slf4j.Logger;
+
+
+public class HiveSqlLogThread extends Thread {
+
+    private final HivePreparedStatement statement;
+
+    private final Logger hiveMapReduceLogger;
+
+    private final TaskExecutionContext taskExecutionContext;
+
+    public HiveSqlLogThread(PreparedStatement statement, Logger logger, TaskExecutionContext taskExecutionContext) throws SQLException {
+        super.setDaemon(true);
+        this.statement = statement.unwrap(HivePreparedStatement.class);
+        this.hiveMapReduceLogger = logger;
+        this.taskExecutionContext = taskExecutionContext;
+    }
+
+    @Override
+    public void run() {
+
+        Thread.currentThread().setName(taskExecutionContext.getTaskLogName());
+
+        if (statement == null) {
+            hiveMapReduceLogger.info("hive statement is null, end this log query!");
+            return;
+        }
+        try {
+            while (!statement.isClosed() && statement.hasMoreLogs()) {
+                for (String log : statement.getQueryLog(true, 500)) {
+
+                    hiveMapReduceLogger.info(log);
+
+                    List<String> appIds = LogUtils.getAppIds(log, hiveMapReduceLogger);
+                    //get sql task yarn's application_id
+                    if (!appIds.isEmpty()) {
+                        hiveMapReduceLogger.info("yarn application_id is {}",appIds);
+                        taskExecutionContext.setAppIds(String.join(",", appIds));

Review Comment:
   Why you don't stop this loop here? you want to rewrite the appId in next loop?



-- 
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] fengjian1129 commented on a diff in pull request #11598: [Improvement-#9680][task-plugin] #9680 add hive sql log listener, print MapReduce progress log and obtain application_id allows developers to better track tasks

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


##########
dolphinscheduler-task-plugin/dolphinscheduler-task-sql/src/main/java/org/apache/dolphinscheduler/plugin/task/sql/SqlTask.java:
##########
@@ -306,13 +319,31 @@ private String executeUpdate(Connection connection, List<SqlBinds> statementsBin
         int result = 0;
         for (SqlBinds sqlBind : statementsBinds) {
             try (PreparedStatement statement = prepareStatementAndBind(connection, sqlBind)) {
+
+                sqlLogListener(statement);
+
                 result = statement.executeUpdate();
                 logger.info("{} statement execute update result: {}, for sql: {}", handlerType, result, sqlBind.getSql());
             }
         }
         return String.valueOf(result);
     }
 
+    /**
+     * sql log listener
+     *
+     * @param statement statement
+     * @throws SQLException sqlException
+     */
+    private void sqlLogListener(PreparedStatement statement) throws SQLException {
+        if (DbType.HIVE == DbType.valueOf(sqlParameters.getType())) {
+            logger.info("The current sql task type is [{}], will start a demon thread to resolve the appIds", DbType.HIVE);
+
+            HiveSqlLogThread queryThread = new HiveSqlLogThread(statement, logger,taskExecutionContext);

Review Comment:
   The purpose here is to start the thread to print the progress log of MR, so that there is a progress log of HIVE SQL execution in the worker log file, and then we can obtain appids using the LogUtils.getAppids() method.



-- 
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] MonsterChenzhuo commented on pull request #11598: [Improvement-#9680][task-plugin] #9680 add hive sql log listener, print MapReduce progress log and obtain application_id allows developers to better track tasks

Posted by "MonsterChenzhuo (via GitHub)" <gi...@apache.org>.
MonsterChenzhuo commented on PR #11598:
URL: https://github.com/apache/dolphinscheduler/pull/11598#issuecomment-1421886022

   Hi @fengjian1129  , I would like to ask if the current pr is still going on, if you have other things to do there and have no time to take care of, I hope to finish the follow-up content


-- 
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] MonsterChenzhuo commented on pull request #11598: [Improvement-#9680][task-plugin] #9680 add hive sql log listener, print MapReduce progress log and obtain application_id allows developers to better track tasks

Posted by "MonsterChenzhuo (via GitHub)" <gi...@apache.org>.
MonsterChenzhuo commented on PR #11598:
URL: https://github.com/apache/dolphinscheduler/pull/11598#issuecomment-1421887991

   Hi, I would like to ask if the current pr is still going on


-- 
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] fengjian1129 commented on a diff in pull request #11598: [Improvement-#9680][task-plugin] #9680 add hive sql log listener, print MapReduce progress log and obtain application_id allows developers to better track tasks

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


##########
dolphinscheduler-task-plugin/dolphinscheduler-task-sql/src/test/java/org/apache/dolphinscheduler/plugin/task/sql/HiveSqlLogThreadTest.java:
##########
@@ -0,0 +1,68 @@
+/*
+ * 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.sql;
+
+import org.apache.dolphinscheduler.plugin.task.api.TaskExecutionContext;
+
+import org.apache.hive.jdbc.HiveStatement;
+import java.sql.SQLException;
+import java.util.ArrayList;
+import java.util.List;
+import org.junit.Assert;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.powermock.api.mockito.PowerMockito;

Review Comment:
   OK



-- 
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 #11598: [Improvement-#9680][task-plugin] add hive sql log listener, print MapReduce progress log and obtain application_id allows developers to better track tasks

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

   Kudos, SonarCloud Quality Gate passed!&nbsp; &nbsp; [![Quality Gate passed](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/QualityGateBadge/passed-16px.png 'Quality Gate passed')](https://sonarcloud.io/dashboard?id=apache-dolphinscheduler&pullRequest=11598)
   
   [![Bug](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/bug-16px.png 'Bug')](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=11598&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=11598&resolved=false&types=BUG) [0 Bugs](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=11598&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=11598&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=11598&resolved=false&types=VULNERABILITY) [0 Vulnerabilities](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=11598&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=11598&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=11598&resolved=false&types=SECURITY_HOTSPOT) [0 Security Hotspots](https://sonarcloud.io/project/security_hotspots?id=apache-dolphinscheduler&pullRequest=11598&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=11598&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=11598&resolved=false&types=CODE_SMELL) [1 Code Smell](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=11598&resolved=false&types=CODE_SMELL)
   
   [![46.3%](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/CoverageChart/40-16px.png '46.3%')](https://sonarcloud.io/component_measures?id=apache-dolphinscheduler&pullRequest=11598&metric=new_coverage&view=list) [46.3% Coverage](https://sonarcloud.io/component_measures?id=apache-dolphinscheduler&pullRequest=11598&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=11598&metric=new_duplicated_lines_density&view=list) [0.0% Duplication](https://sonarcloud.io/component_measures?id=apache-dolphinscheduler&pullRequest=11598&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 #11598: [Improvement-#9680][task-plugin] #9680 add hive sql log listener, print MapReduce progress log and obtain application_id allows developers to better track tasks

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


##########
dolphinscheduler-task-plugin/dolphinscheduler-task-sql/src/main/java/org/apache/dolphinscheduler/plugin/task/sql/SqlTask.java:
##########
@@ -306,13 +319,31 @@ private String executeUpdate(Connection connection, List<SqlBinds> statementsBin
         int result = 0;
         for (SqlBinds sqlBind : statementsBinds) {
             try (PreparedStatement statement = prepareStatementAndBind(connection, sqlBind)) {
+
+                sqlLogListener(statement);
+
                 result = statement.executeUpdate();
                 logger.info("{} statement execute update result: {}, for sql: {}", handlerType, result, sqlBind.getSql());
             }
         }
         return String.valueOf(result);
     }
 
+    /**
+     * sql log listener
+     *
+     * @param statement statement
+     * @throws SQLException sqlException
+     */
+    private void sqlLogListener(PreparedStatement statement) throws SQLException {
+        if (DbType.HIVE == DbType.valueOf(sqlParameters.getType())) {
+            logger.info("The current sql task type is [{}], will start a demon thread to resolve the appIds", DbType.HIVE);
+
+            HiveSqlLogThread queryThread = new HiveSqlLogThread(statement, logger,taskExecutionContext);

Review Comment:
   Is it possible to parse all the logs at once after execution?



-- 
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] EricGao888 commented on a diff in pull request #11598: [Improvement-#9680][task-plugin] #9680 add hive sql log listener, print MapReduce progress log and obtain application_id allows developers to better track tasks

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


##########
dolphinscheduler-task-plugin/dolphinscheduler-task-sql/src/test/java/org/apache/dolphinscheduler/plugin/task/sql/HiveSqlLogThreadTest.java:
##########
@@ -0,0 +1,68 @@
+/*
+ * 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.sql;
+
+import org.apache.dolphinscheduler.plugin.task.api.TaskExecutionContext;
+
+import org.apache.hive.jdbc.HiveStatement;
+import java.sql.SQLException;
+import java.util.ArrayList;
+import java.util.List;
+import org.junit.Assert;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.powermock.api.mockito.PowerMockito;

Review Comment:
   For clarification, what I meant was we should avoid using anything which comes from  `org.powermock`. You could use `org.mockito.xxx` instead.



-- 
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 #11598: [Improvement-#9680][task-plugin] add hive sql log listener, print MapReduce progress log and obtain application_id allows developers to better track tasks

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

   # [Codecov](https://codecov.io/gh/apache/dolphinscheduler/pull/11598?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 [#11598](https://codecov.io/gh/apache/dolphinscheduler/pull/11598?src=pr&el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (9b412a2) into [dev](https://codecov.io/gh/apache/dolphinscheduler/commit/17a9dd25fa0e80b048394f79db130f56eb8ef72f?el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (17a9dd2) will **increase** coverage by `0.00%`.
   > The diff coverage is `n/a`.
   
   > :exclamation: Current head 9b412a2 differs from pull request most recent head 5d1316b. Consider uploading reports for the commit 5d1316b to get more accurate results
   
   ```diff
   @@            Coverage Diff            @@
   ##                dev   #11598   +/-   ##
   =========================================
     Coverage     39.79%   39.79%           
   + Complexity     4697     4696    -1     
   =========================================
     Files           993      993           
     Lines         37662    37662           
     Branches       4186     4187    +1     
   =========================================
   + Hits          14986    14987    +1     
     Misses        21109    21109           
   + Partials       1567     1566    -1     
   ```
   
   
   | [Impacted Files](https://codecov.io/gh/apache/dolphinscheduler/pull/11598?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | Coverage Δ | |
   |---|---|---|
   | [...e/dolphinscheduler/remote/NettyRemotingClient.java](https://codecov.io/gh/apache/dolphinscheduler/pull/11598/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-ZG9scGhpbnNjaGVkdWxlci1yZW1vdGUvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2RvbHBoaW5zY2hlZHVsZXIvcmVtb3RlL05ldHR5UmVtb3RpbmdDbGllbnQuamF2YQ==) | `52.77% <0.00%> (+0.69%)` | :arrow_up: |
   
   :mega: We’re building smart automated test selection to slash your CI/CD build times. [Learn more](https://about.codecov.io/iterative-testing/?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] github-code-scanning[bot] commented on a diff in pull request #11598: [Improvement-#9680][task-plugin] #9680 add hive sql log listener, print MapReduce progress log and obtain application_id allows developers to better track tasks

Posted by GitBox <gi...@apache.org>.
github-code-scanning[bot] commented on code in PR #11598:
URL: https://github.com/apache/dolphinscheduler/pull/11598#discussion_r969443590


##########
dolphinscheduler-task-plugin/dolphinscheduler-task-sql/src/test/java/org/apache/dolphinscheduler/plugin/task/sql/HiveSqlLogThreadTest.java:
##########
@@ -0,0 +1,65 @@
+/*
+ * 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.sql;
+
+import org.apache.dolphinscheduler.plugin.task.api.TaskExecutionContext;
+import org.apache.dolphinscheduler.spi.enums.DbType;
+
+import org.apache.hive.jdbc.HivePreparedStatement;
+
+import java.sql.PreparedStatement;
+import java.sql.SQLException;
+import java.util.ArrayList;
+import java.util.List;
+import org.junit.Assert;
+import org.junit.Test;
+import org.mockito.Mockito;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+public class HiveSqlLogThreadTest {
+    private static final Logger LOGGER = LoggerFactory.getLogger(HiveSqlLogThreadTest.class);
+
+    private static volatile TaskExecutionContext taskExecutionContext;
+
+    @Test
+    public void testHiveSql() throws Exception {
+        taskExecutionContext = new TaskExecutionContext();
+        taskExecutionContext.setTaskType("hive");
+        taskExecutionContext.setTaskLogName("1-1-1-1-1");
+
+        String sql = "select count(*) from test.table";

Review Comment:
   ## Unread local variable
   
   Variable 'String sql' is never read.
   
   [Show more details](https://github.com/apache/dolphinscheduler/security/code-scanning/1225)



-- 
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 #11598: [Improvement-#9680][task-plugin] #9680 add hive sql log listener, print MapReduce progress log and obtain application_id allows developers to better track tasks

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

   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=11598)
   
   [![Bug](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/bug-16px.png 'Bug')](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=11598&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=11598&resolved=false&types=BUG) [0 Bugs](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=11598&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=11598&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=11598&resolved=false&types=VULNERABILITY) [0 Vulnerabilities](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=11598&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=11598&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=11598&resolved=false&types=SECURITY_HOTSPOT) [0 Security Hotspots](https://sonarcloud.io/project/security_hotspots?id=apache-dolphinscheduler&pullRequest=11598&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=11598&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=11598&resolved=false&types=CODE_SMELL) [0 Code Smells](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=11598&resolved=false&types=CODE_SMELL)
   
   [![30.4%](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/CoverageChart/25-16px.png '30.4%')](https://sonarcloud.io/component_measures?id=apache-dolphinscheduler&pullRequest=11598&metric=new_coverage&view=list) [30.4% Coverage](https://sonarcloud.io/component_measures?id=apache-dolphinscheduler&pullRequest=11598&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=11598&metric=new_duplicated_lines_density&view=list) [0.0% Duplication](https://sonarcloud.io/component_measures?id=apache-dolphinscheduler&pullRequest=11598&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] fengjian1129 commented on a diff in pull request #11598: [Improvement-#9680][task-plugin] #9680 add hive sql log listener, print MapReduce progress log and obtain application_id allows developers to better track tasks

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


##########
dolphinscheduler-task-plugin/dolphinscheduler-task-sql/pom.xml:
##########
@@ -55,5 +55,10 @@
             <scope>provided</scope>
         </dependency>
 
+        <dependency>
+            <groupId>org.apache.dolphinscheduler</groupId>
+            <artifactId>dolphinscheduler-common</artifactId>
+        </dependency>
+

Review Comment:
   I need to use the LoggerUtils.buildTaskId() method,Can I write the same method in the dolphinscheduler-task-api



-- 
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] EricGao888 commented on pull request #11598: [Improvement-#9680][task-plugin] add hive sql log listener, print MapReduce progress log and obtain application_id allows developers to better track tasks

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

   @SbloodyS @zhongjiajie PTAL when available, thanks : )


-- 
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] fengjian1129 commented on a diff in pull request #11598: [Improvement-#9680][task-plugin] #9680 add hive sql log listener, print MapReduce progress log and obtain application_id allows developers to better track tasks

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


##########
dolphinscheduler-task-plugin/dolphinscheduler-task-sql/src/main/java/org/apache/dolphinscheduler/plugin/task/sql/SqlTask.java:
##########
@@ -306,13 +319,31 @@ private String executeUpdate(Connection connection, List<SqlBinds> statementsBin
         int result = 0;
         for (SqlBinds sqlBind : statementsBinds) {
             try (PreparedStatement statement = prepareStatementAndBind(connection, sqlBind)) {
+
+                sqlLogListener(statement);
+
                 result = statement.executeUpdate();
                 logger.info("{} statement execute update result: {}, for sql: {}", handlerType, result, sqlBind.getSql());
             }
         }
         return String.valueOf(result);
     }
 
+    /**
+     * sql log listener
+     *
+     * @param statement statement
+     * @throws SQLException sqlException
+     */
+    private void sqlLogListener(PreparedStatement statement) throws SQLException {
+        if (DbType.HIVE == DbType.valueOf(sqlParameters.getType())) {
+            logger.info("The current sql task type is [{}], will start a demon thread to resolve the appIds", DbType.HIVE);
+
+            HiveSqlLogThread queryThread = new HiveSqlLogThread(statement, logger,taskExecutionContext);

Review Comment:
   The screenshot of the print log is as follows:
   <img width="821" alt="image" src="https://user-images.githubusercontent.com/35831367/189870359-16ba2b67-ece2-405a-97a7-4ffd7c8d37da.png">
   
   In SqlTask.class ,Parse all logs and get appid as follows:
   <img width="869" alt="image" src="https://user-images.githubusercontent.com/35831367/189877917-fe6f4e98-2bc3-406e-8e2b-009b5e2df76d.png">
   



-- 
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] fengjian1129 commented on a diff in pull request #11598: [Improvement-#9680][task-plugin] #9680 add hive sql log listener, print MapReduce progress log and obtain application_id allows developers to better track tasks

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


##########
dolphinscheduler-task-plugin/dolphinscheduler-task-sql/src/main/java/org/apache/dolphinscheduler/plugin/task/sql/SqlTask.java:
##########
@@ -306,13 +319,31 @@ private String executeUpdate(Connection connection, List<SqlBinds> statementsBin
         int result = 0;
         for (SqlBinds sqlBind : statementsBinds) {
             try (PreparedStatement statement = prepareStatementAndBind(connection, sqlBind)) {
+
+                sqlLogListener(statement);
+
                 result = statement.executeUpdate();
                 logger.info("{} statement execute update result: {}, for sql: {}", handlerType, result, sqlBind.getSql());
             }
         }
         return String.valueOf(result);
     }
 
+    /**
+     * sql log listener
+     *
+     * @param statement statement
+     * @throws SQLException sqlException
+     */
+    private void sqlLogListener(PreparedStatement statement) throws SQLException {
+        if (DbType.HIVE == DbType.valueOf(sqlParameters.getType())) {
+            logger.info("The current sql task type is [{}], will start a demon thread to resolve the appIds", DbType.HIVE);
+
+            HiveSqlLogThread queryThread = new HiveSqlLogThread(statement, logger,taskExecutionContext);

Review Comment:
   @caishunfeng You can review my latest 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


Re: [PR] [Improvement-#9680][task-plugin] #9680 add hive sql log listener, print MapReduce progress log and obtain application_id allows developers to better track tasks [dolphinscheduler]

Posted by "github-actions[bot] (via GitHub)" <gi...@apache.org>.
github-actions[bot] closed pull request #11598: [Improvement-#9680][task-plugin] #9680 add hive sql log listener, print MapReduce progress log and obtain application_id allows developers to better track tasks
URL: https://github.com/apache/dolphinscheduler/pull/11598


-- 
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 #11598: [Improvement-#9680][task-plugin] add hive sql log listener, print MapReduce progress log and obtain application_id allows developers to better track tasks

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

   Kudos, SonarCloud Quality Gate passed!&nbsp; &nbsp; [![Quality Gate passed](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/QualityGateBadge/passed-16px.png 'Quality Gate passed')](https://sonarcloud.io/dashboard?id=apache-dolphinscheduler&pullRequest=11598)
   
   [![Bug](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/bug-16px.png 'Bug')](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=11598&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=11598&resolved=false&types=BUG) [0 Bugs](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=11598&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=11598&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=11598&resolved=false&types=VULNERABILITY) [0 Vulnerabilities](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=11598&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=11598&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=11598&resolved=false&types=SECURITY_HOTSPOT) [0 Security Hotspots](https://sonarcloud.io/project/security_hotspots?id=apache-dolphinscheduler&pullRequest=11598&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=11598&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=11598&resolved=false&types=CODE_SMELL) [2 Code Smells](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=11598&resolved=false&types=CODE_SMELL)
   
   [![47.6%](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/CoverageChart/40-16px.png '47.6%')](https://sonarcloud.io/component_measures?id=apache-dolphinscheduler&pullRequest=11598&metric=new_coverage&view=list) [47.6% Coverage](https://sonarcloud.io/component_measures?id=apache-dolphinscheduler&pullRequest=11598&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=11598&metric=new_duplicated_lines_density&view=list) [0.0% Duplication](https://sonarcloud.io/component_measures?id=apache-dolphinscheduler&pullRequest=11598&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 #11598: [Improvement-#9680][task-plugin] add hive sql log listener, print MapReduce progress log and obtain application_id allows developers to better track tasks

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

   Kudos, SonarCloud Quality Gate passed!&nbsp; &nbsp; [![Quality Gate passed](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/QualityGateBadge/passed-16px.png 'Quality Gate passed')](https://sonarcloud.io/dashboard?id=apache-dolphinscheduler&pullRequest=11598)
   
   [![Bug](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/bug-16px.png 'Bug')](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=11598&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=11598&resolved=false&types=BUG) [0 Bugs](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=11598&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=11598&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=11598&resolved=false&types=VULNERABILITY) [0 Vulnerabilities](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=11598&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=11598&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=11598&resolved=false&types=SECURITY_HOTSPOT) [0 Security Hotspots](https://sonarcloud.io/project/security_hotspots?id=apache-dolphinscheduler&pullRequest=11598&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=11598&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=11598&resolved=false&types=CODE_SMELL) [2 Code Smells](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=11598&resolved=false&types=CODE_SMELL)
   
   [![47.6%](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/CoverageChart/40-16px.png '47.6%')](https://sonarcloud.io/component_measures?id=apache-dolphinscheduler&pullRequest=11598&metric=new_coverage&view=list) [47.6% Coverage](https://sonarcloud.io/component_measures?id=apache-dolphinscheduler&pullRequest=11598&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=11598&metric=new_duplicated_lines_density&view=list) [0.0% Duplication](https://sonarcloud.io/component_measures?id=apache-dolphinscheduler&pullRequest=11598&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] fengjian1129 commented on pull request #11598: [Improvement-#9680][task-plugin] add hive sql log listener, print MapReduce progress log and obtain application_id allows developers to better track tasks

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

   @EricGao888 hi bro,I have modified the test class, and the local test is available. Please check it。


-- 
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 #11598: [Improvement-#9680][task-plugin] #9680 add hive sql log listener, print MapReduce progress log and obtain application_id allows developers to better track tasks

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

   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=11598)
   
   [![Bug](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/bug-16px.png 'Bug')](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=11598&resolved=false&types=BUG) [![D](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/D-16px.png 'D')](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=11598&resolved=false&types=BUG) [2 Bugs](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=11598&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=11598&resolved=false&types=VULNERABILITY) [![B](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/B-16px.png 'B')](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=11598&resolved=false&types=VULNERABILITY) [1 Vulnerability](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=11598&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=11598&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=11598&resolved=false&types=SECURITY_HOTSPOT) [0 Security Hotspots](https://sonarcloud.io/project/security_hotspots?id=apache-dolphinscheduler&pullRequest=11598&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=11598&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=11598&resolved=false&types=CODE_SMELL) [58 Code Smells](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=11598&resolved=false&types=CODE_SMELL)
   
   [![57.8%](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/CoverageChart/50-16px.png '57.8%')](https://sonarcloud.io/component_measures?id=apache-dolphinscheduler&pullRequest=11598&metric=new_coverage&view=list) [57.8% Coverage](https://sonarcloud.io/component_measures?id=apache-dolphinscheduler&pullRequest=11598&metric=new_coverage&view=list)  
   [![32.9%](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/Duplications/20plus-16px.png '32.9%')](https://sonarcloud.io/component_measures?id=apache-dolphinscheduler&pullRequest=11598&metric=new_duplicated_lines_density&view=list) [32.9% Duplication](https://sonarcloud.io/component_measures?id=apache-dolphinscheduler&pullRequest=11598&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] EricGao888 commented on a diff in pull request #11598: [Improvement-#9680][task-plugin] #9680 add hive sql log listener, print MapReduce progress log and obtain application_id allows developers to better track tasks

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


##########
dolphinscheduler-task-plugin/dolphinscheduler-task-sql/src/test/java/org/apache/dolphinscheduler/plugin/task/sql/HiveSqlLogThreadTest.java:
##########
@@ -0,0 +1,68 @@
+/*
+ * 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.sql;
+
+import org.apache.dolphinscheduler.plugin.task.api.TaskExecutionContext;
+
+import org.apache.hive.jdbc.HiveStatement;
+import java.sql.SQLException;
+import java.util.ArrayList;
+import java.util.List;
+import org.junit.Assert;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.powermock.api.mockito.PowerMockito;
+import org.powermock.core.classloader.annotations.PowerMockIgnore;
+import org.powermock.modules.junit4.PowerMockRunner;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+@RunWith(PowerMockRunner.class)
+@PowerMockIgnore({"javax.*"})
+public class HiveSqlLogThreadTest {
+    private static final Logger LOGGER = LoggerFactory.getLogger(HiveSqlLogThreadTest.class);
+
+    private static volatile TaskExecutionContext taskExecutionContext;
+
+    @Test
+    public void testHiveSql() throws SQLException {
+        taskExecutionContext = new TaskExecutionContext();
+        taskExecutionContext.setTaskType("hive");
+
+        String sql = "select count(*) from test.table";
+
+        List<String> mockLog = new ArrayList<>();
+        mockLog.add("1start hive sql log\napplication_1231_2323");
+        HiveStatement statement = PowerMockito.mock(HiveStatement.class);
+        PowerMockito.when(statement.isClosed()).thenReturn(false);
+        PowerMockito.when(statement.hasMoreLogs()).thenReturn(true);
+        PowerMockito.when(statement.getQueryLog(true, 500)).thenReturn(mockLog);
+        try {
+            HiveSqlLogThread queryThread = PowerMockito.spy(new HiveSqlLogThread(statement, LOGGER, taskExecutionContext));
+            queryThread.start();
+
+            Thread.sleep(5000);
+            Assert.assertEquals(taskExecutionContext.getAppIds(), "application_1231_2323");
+
+        } catch (Exception e) {

Review Comment:
   You shouldn't catch `Exception e` here. If anything fails and throws exception in the `try` block, the exception will get caught and the UT will pass, which is not expected. If you want to verify a specific exception, you could use `@Test(expected = XXXException.class)` in `jUnit4`.



##########
dolphinscheduler-task-plugin/dolphinscheduler-task-sql/src/test/java/org/apache/dolphinscheduler/plugin/task/sql/HiveSqlLogThreadTest.java:
##########
@@ -0,0 +1,68 @@
+/*
+ * 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.sql;
+
+import org.apache.dolphinscheduler.plugin.task.api.TaskExecutionContext;
+
+import org.apache.hive.jdbc.HiveStatement;
+import java.sql.SQLException;
+import java.util.ArrayList;
+import java.util.List;
+import org.junit.Assert;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.powermock.api.mockito.PowerMockito;

Review Comment:
   There is no need to use `Powermock` in this UT. `Mockito` will be enough.



-- 
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] fengjian1129 commented on pull request #11598: [Improvement-#9680][task-plugin] #9680 add hive sql log listener, print MapReduce progress log and obtain application_id allows developers to better track tasks

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

   @EricGao888 I use the result chart to show this change
   1.appids write t_ ds_ task_ Instance table and print MR processing log
   <img width="956" alt="image" src="https://user-images.githubusercontent.com/35831367/188057628-4f5e57c2-93e8-445f-afaa-20092c440b8b.png">
   <img width="1001" alt="image" src="https://user-images.githubusercontent.com/35831367/188057666-0036e739-9d7c-4757-93bf-69994d0da502.png">
   <img width="1364" alt="image" src="https://user-images.githubusercontent.com/35831367/188057930-4931d850-6ca3-4186-aab3-356f02be6101.png">
   
   


-- 
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] fengjian1129 commented on pull request #11598: [Improvement-#9680][task-plugin] add hive sql log listener, print MapReduce progress log and obtain application_id allows developers to better track tasks

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

   @EricGao888 hi bro, please check


-- 
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 #11598: [Improvement-#9680][task-plugin] #9680 add hive sql log listener, print MapReduce progress log and obtain application_id allows developers to better track tasks

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

   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=11598)
   
   [![Bug](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/bug-16px.png 'Bug')](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=11598&resolved=false&types=BUG) [![D](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/D-16px.png 'D')](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=11598&resolved=false&types=BUG) [2 Bugs](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=11598&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=11598&resolved=false&types=VULNERABILITY) [![B](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/B-16px.png 'B')](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=11598&resolved=false&types=VULNERABILITY) [1 Vulnerability](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=11598&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=11598&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=11598&resolved=false&types=SECURITY_HOTSPOT) [0 Security Hotspots](https://sonarcloud.io/project/security_hotspots?id=apache-dolphinscheduler&pullRequest=11598&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=11598&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=11598&resolved=false&types=CODE_SMELL) [58 Code Smells](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=11598&resolved=false&types=CODE_SMELL)
   
   [![57.8%](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/CoverageChart/50-16px.png '57.8%')](https://sonarcloud.io/component_measures?id=apache-dolphinscheduler&pullRequest=11598&metric=new_coverage&view=list) [57.8% Coverage](https://sonarcloud.io/component_measures?id=apache-dolphinscheduler&pullRequest=11598&metric=new_coverage&view=list)  
   [![32.9%](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/Duplications/20plus-16px.png '32.9%')](https://sonarcloud.io/component_measures?id=apache-dolphinscheduler&pullRequest=11598&metric=new_duplicated_lines_density&view=list) [32.9% Duplication](https://sonarcloud.io/component_measures?id=apache-dolphinscheduler&pullRequest=11598&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] ruanwenjun commented on pull request #11598: [Improvement-#9680][task-plugin] #9680 add hive sql log listener, print MapReduce progress log and obtain application_id allows developers to better track tasks

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

   @caishunfeng Please take a look.


-- 
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 #11598: [Improvement-#9680][task-plugin] #9680 add hive sql log listener, print MapReduce progress log and obtain application_id allows developers to better track tasks

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


##########
dolphinscheduler-task-plugin/dolphinscheduler-task-sql/src/main/java/org/apache/dolphinscheduler/plugin/task/sql/SqlTask.java:
##########
@@ -306,13 +319,31 @@ private String executeUpdate(Connection connection, List<SqlBinds> statementsBin
         int result = 0;
         for (SqlBinds sqlBind : statementsBinds) {
             try (PreparedStatement statement = prepareStatementAndBind(connection, sqlBind)) {
+
+                sqlLogListener(statement);
+
                 result = statement.executeUpdate();
                 logger.info("{} statement execute update result: {}, for sql: {}", handlerType, result, sqlBind.getSql());
             }
         }
         return String.valueOf(result);
     }
 
+    /**
+     * sql log listener
+     *
+     * @param statement statement
+     * @throws SQLException sqlException
+     */
+    private void sqlLogListener(PreparedStatement statement) throws SQLException {
+        if (DbType.HIVE == DbType.valueOf(sqlParameters.getType())) {
+            logger.info("The current sql task type is [{}], will start a demon thread to resolve the appIds", DbType.HIVE);
+
+            HiveSqlLogThread queryThread = new HiveSqlLogThread(statement, logger,taskExecutionContext);

Review Comment:
   I don't think it's a good way to create thread for log parsing, you can just get appIds after executing sql.



-- 
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 #11598: [Improvement-#9680][task-plugin] #9680 add hive sql log listener, print MapReduce progress log and obtain application_id allows developers to better track tasks

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

   Kudos, SonarCloud Quality Gate passed!&nbsp; &nbsp; [![Quality Gate passed](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/QualityGateBadge/passed-16px.png 'Quality Gate passed')](https://sonarcloud.io/dashboard?id=apache-dolphinscheduler&pullRequest=11598)
   
   [![Bug](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/bug-16px.png 'Bug')](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=11598&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=11598&resolved=false&types=BUG) [0 Bugs](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=11598&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=11598&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=11598&resolved=false&types=VULNERABILITY) [0 Vulnerabilities](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=11598&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=11598&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=11598&resolved=false&types=SECURITY_HOTSPOT) [0 Security Hotspots](https://sonarcloud.io/project/security_hotspots?id=apache-dolphinscheduler&pullRequest=11598&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=11598&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=11598&resolved=false&types=CODE_SMELL) [2 Code Smells](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=11598&resolved=false&types=CODE_SMELL)
   
   [![36.1%](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/CoverageChart/25-16px.png '36.1%')](https://sonarcloud.io/component_measures?id=apache-dolphinscheduler&pullRequest=11598&metric=new_coverage&view=list) [36.1% Coverage](https://sonarcloud.io/component_measures?id=apache-dolphinscheduler&pullRequest=11598&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=11598&metric=new_duplicated_lines_density&view=list) [0.0% Duplication](https://sonarcloud.io/component_measures?id=apache-dolphinscheduler&pullRequest=11598&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 #11598: [Improvement-#9680][task-plugin] #9680 add hive sql log listener, print MapReduce progress log and obtain application_id allows developers to better track tasks

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

   Kudos, SonarCloud Quality Gate passed!&nbsp; &nbsp; [![Quality Gate passed](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/QualityGateBadge/passed-16px.png 'Quality Gate passed')](https://sonarcloud.io/dashboard?id=apache-dolphinscheduler&pullRequest=11598)
   
   [![Bug](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/bug-16px.png 'Bug')](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=11598&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=11598&resolved=false&types=BUG) [0 Bugs](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=11598&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=11598&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=11598&resolved=false&types=VULNERABILITY) [0 Vulnerabilities](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=11598&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=11598&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=11598&resolved=false&types=SECURITY_HOTSPOT) [0 Security Hotspots](https://sonarcloud.io/project/security_hotspots?id=apache-dolphinscheduler&pullRequest=11598&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=11598&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=11598&resolved=false&types=CODE_SMELL) [2 Code Smells](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=11598&resolved=false&types=CODE_SMELL)
   
   [![36.1%](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/CoverageChart/25-16px.png '36.1%')](https://sonarcloud.io/component_measures?id=apache-dolphinscheduler&pullRequest=11598&metric=new_coverage&view=list) [36.1% Coverage](https://sonarcloud.io/component_measures?id=apache-dolphinscheduler&pullRequest=11598&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=11598&metric=new_duplicated_lines_density&view=list) [0.0% Duplication](https://sonarcloud.io/component_measures?id=apache-dolphinscheduler&pullRequest=11598&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] fengjian1129 commented on pull request #11598: [Improvement-#9680][task-plugin] #9680 add hive sql log listener, print MapReduce progress log and obtain application_id allows developers to better track tasks

Posted by "fengjian1129 (via GitHub)" <gi...@apache.org>.
fengjian1129 commented on PR #11598:
URL: https://github.com/apache/dolphinscheduler/pull/11598#issuecomment-1434399655

   > Hi, I would like to ask if the current pr is still going on
   
   hi bro.
   Recently, I have been busy. Can you help me to modify this PR and merge it?  Because this PR is too far behind.


-- 
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


Re: [PR] [Improvement-#9680][task-plugin] #9680 add hive sql log listener, print MapReduce progress log and obtain application_id allows developers to better track tasks [dolphinscheduler]

Posted by "github-actions[bot] (via GitHub)" <gi...@apache.org>.
github-actions[bot] commented on PR #11598:
URL: https://github.com/apache/dolphinscheduler/pull/11598#issuecomment-1928550833

   This pull request has been automatically marked as stale because it has not had recent activity for 120 days. It will be closed in 7 days if no further activity occurs.


-- 
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] kezhenxu94 commented on a diff in pull request #11598: [Improvement-#9680][task-plugin] #9680 add hive sql log listener, print MapReduce progress log and obtain application_id allows developers to better track tasks

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


##########
dolphinscheduler-dist/release-docs/licenses/LISCENSE-perfmark-api.txt:
##########
@@ -0,0 +1,201 @@
+                                 Apache License

Review Comment:
   You didn't add any new dependencies, this file is unexpected, please remove or confirm where this dependency is brought



##########
dolphinscheduler-bom/pom.xml:
##########
@@ -258,6 +258,26 @@
                 <scope>import</scope>
             </dependency>
 
+            <!-- Etcd -->
+            <dependency>
+                <groupId>io.etcd</groupId>
+                <artifactId>jetcd-core</artifactId>
+                <version>${jetcd.version}</version>
+            </dependency>
+            <dependency>
+                <groupId>io.etcd</groupId>
+                <artifactId>jetcd-test</artifactId>
+                <version>${jetcd.test.version}</version>
+                <scope>test</scope>
+            </dependency>
+            <dependency>
+                <groupId>io.grpc</groupId>
+                <artifactId>grpc-bom</artifactId>
+                <version>${io.grpc.version}</version>
+                <scope>import</scope>
+                <type>pom</type>
+            </dependency>
+

Review Comment:
   Remove these lines



##########
dolphinscheduler-task-plugin/dolphinscheduler-task-sql/src/main/java/org/apache/dolphinscheduler/plugin/task/sql/SqlTask.java:
##########
@@ -365,13 +379,31 @@ private String executeUpdate(Connection connection, List<SqlBinds> statementsBin
         int result = 0;
         for (SqlBinds sqlBind : statementsBinds) {
             try (PreparedStatement statement = prepareStatementAndBind(connection, sqlBind)) {
+
+                sqlLogListener(statement);
+
                 result = statement.executeUpdate();
                 logger.info("{} statement execute update result: {}, for sql: {}", handlerType, result, sqlBind.getSql());
             }
         }
         return String.valueOf(result);
     }
 
+    /**
+     * sql log listener
+     *
+     * @param statement statement
+     * @throws SQLException sqlException
+     */
+    private void sqlLogListener(PreparedStatement statement) throws SQLException {
+        if (DbType.HIVE.name().equals(sqlParameters.getType())) {
+            logger.info("The current sql task type is HIVE, will start a demon thread to resolve the appIds");

Review Comment:
   ```suggestion
               logger.info("The current sql task type is HIVE, will start a daemon thread to resolve the appIds");
   ```



##########
dolphinscheduler-api/pom.xml:
##########
@@ -195,6 +195,11 @@
             <artifactId>hbase-noop-htrace</artifactId>
         </dependency>
 
+        <dependency>
+            <groupId>org.springframework.cloud</groupId>
+            <artifactId>spring-cloud-starter-kubernetes-fabric8-config</artifactId>
+        </dependency>
+

Review Comment:
   Remove this, perhaps you do it wrong when resolving conflicts



-- 
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] fengjian1129 commented on pull request #11598: [Improvement-#9680][task-plugin] #9680 add hive sql log listener, print MapReduce progress log and obtain application_id allows developers to better track tasks

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

   @EricGao888 t_ds_task_instance table
   <img width="884" alt="image" src="https://user-images.githubusercontent.com/35831367/188058211-38b66409-9584-4621-8cd8-013eb25cc308.png">
   


-- 
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 #11598: [Improvement-#9680][task-plugin] #9680 add hive sql log listener, print MapReduce progress log and obtain application_id allows developers to better track tasks

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

   Kudos, SonarCloud Quality Gate passed!&nbsp; &nbsp; [![Quality Gate passed](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/QualityGateBadge/passed-16px.png 'Quality Gate passed')](https://sonarcloud.io/dashboard?id=apache-dolphinscheduler&pullRequest=11598)
   
   [![Bug](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/bug-16px.png 'Bug')](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=11598&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=11598&resolved=false&types=BUG) [0 Bugs](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=11598&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=11598&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=11598&resolved=false&types=VULNERABILITY) [0 Vulnerabilities](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=11598&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=11598&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=11598&resolved=false&types=SECURITY_HOTSPOT) [0 Security Hotspots](https://sonarcloud.io/project/security_hotspots?id=apache-dolphinscheduler&pullRequest=11598&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=11598&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=11598&resolved=false&types=CODE_SMELL) [1 Code Smell](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=11598&resolved=false&types=CODE_SMELL)
   
   [![34.4%](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/CoverageChart/25-16px.png '34.4%')](https://sonarcloud.io/component_measures?id=apache-dolphinscheduler&pullRequest=11598&metric=new_coverage&view=list) [34.4% Coverage](https://sonarcloud.io/component_measures?id=apache-dolphinscheduler&pullRequest=11598&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=11598&metric=new_duplicated_lines_density&view=list) [0.0% Duplication](https://sonarcloud.io/component_measures?id=apache-dolphinscheduler&pullRequest=11598&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 #11598: [Improvement-#9680][task-plugin] #9680 add hive sql log listener, print MapReduce progress log and obtain application_id allows developers to better track tasks

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

   Kudos, SonarCloud Quality Gate passed!&nbsp; &nbsp; [![Quality Gate passed](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/QualityGateBadge/passed-16px.png 'Quality Gate passed')](https://sonarcloud.io/dashboard?id=apache-dolphinscheduler&pullRequest=11598)
   
   [![Bug](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/bug-16px.png 'Bug')](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=11598&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=11598&resolved=false&types=BUG) [0 Bugs](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=11598&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=11598&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=11598&resolved=false&types=VULNERABILITY) [0 Vulnerabilities](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=11598&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=11598&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=11598&resolved=false&types=SECURITY_HOTSPOT) [0 Security Hotspots](https://sonarcloud.io/project/security_hotspots?id=apache-dolphinscheduler&pullRequest=11598&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=11598&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=11598&resolved=false&types=CODE_SMELL) [1 Code Smell](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=11598&resolved=false&types=CODE_SMELL)
   
   [![34.4%](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/CoverageChart/25-16px.png '34.4%')](https://sonarcloud.io/component_measures?id=apache-dolphinscheduler&pullRequest=11598&metric=new_coverage&view=list) [34.4% Coverage](https://sonarcloud.io/component_measures?id=apache-dolphinscheduler&pullRequest=11598&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=11598&metric=new_duplicated_lines_density&view=list) [0.0% Duplication](https://sonarcloud.io/component_measures?id=apache-dolphinscheduler&pullRequest=11598&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] fengjian1129 commented on pull request #11598: [Improvement-#9680][task-plugin] #9680 add hive sql log listener, print MapReduce progress log and obtain application_id allows developers to better track tasks

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

   @ruanwenjun hi bro,review the code for me. thanks


-- 
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] ruanwenjun commented on a diff in pull request #11598: [Improvement-#9680][task-plugin] #9680 add hive sql log listener, print MapReduce progress log and obtain application_id allows developers to better track tasks

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


##########
dolphinscheduler-task-plugin/dolphinscheduler-task-sql/src/main/java/org/apache/dolphinscheduler/plugin/task/sql/SqlTask.java:
##########
@@ -155,6 +156,14 @@ public void handle() throws TaskException {
             // execute sql task
             executeFuncAndSql(mainStatementSqlBinds, preStatementSqlBinds, postStatementSqlBinds, createFuncs);
 
+            // get appIds
+            List<String> appIds = LogUtils.getAppIdsFromLogFile(taskExecutionContext.getLogPath());
+            logger.info("task log path:[{}],yarn appIds:[{}]", taskExecutionContext.getLogPath(), appIds);

Review Comment:
   ```suggestion
               logger.info("Resolve appIds from task log: [{}]", appIds, taskExecutionContext.getLogPath());
   ```



##########
dolphinscheduler-task-plugin/dolphinscheduler-task-sql/pom.xml:
##########
@@ -55,5 +55,10 @@
             <scope>provided</scope>
         </dependency>
 
+        <dependency>
+            <groupId>org.apache.dolphinscheduler</groupId>
+            <artifactId>dolphinscheduler-common</artifactId>
+        </dependency>
+

Review Comment:
   Please don't add `dolphinscheduler-common` into task plugin module.



##########
dolphinscheduler-task-plugin/dolphinscheduler-task-sql/src/main/java/org/apache/dolphinscheduler/plugin/task/sql/SqlTask.java:
##########
@@ -291,6 +300,12 @@ private void sendAttachment(int groupId, String title, String content) {
     private String executeQuery(Connection connection, SqlBinds sqlBinds, String handlerType) throws Exception {
         try (PreparedStatement statement = prepareStatementAndBind(connection, sqlBinds)) {
             logger.info("{} statement execute query, for sql: {}", handlerType, sqlBinds.getSql());
+
+            if (DbType.HIVE == DbType.valueOf(sqlParameters.getType())) {
+                logger.info("execute sql type is [{}]",DbType.HIVE.getDescp());

Review Comment:
   ```suggestion
                   logger.info("The current sql task type is [{}], will start a demon thread to resolve the appIds", DbType.HIVE);
   ```



##########
dolphinscheduler-task-plugin/dolphinscheduler-task-sql/src/main/java/org/apache/dolphinscheduler/plugin/task/sql/SqlTask.java:
##########
@@ -300,6 +315,13 @@ private String executeUpdate(Connection connection, List<SqlBinds> statementsBin
         int result = 0;
         for (SqlBinds sqlBind : statementsBinds) {
             try (PreparedStatement statement = prepareStatementAndBind(connection, sqlBind)) {
+
+                if (DbType.HIVE == DbType.valueOf(sqlParameters.getType())) {
+                    logger.info("execute sql type is [{}]",DbType.HIVE.getDescp());
+
+                    HiveSqlLogThread queryThread = new HiveSqlLogThread(statement, logger,taskExecutionContext);
+                    queryThread.start();
+                }

Review Comment:
   Can we extract this into a private method?



##########
dolphinscheduler-task-plugin/dolphinscheduler-task-sql/src/main/java/org/apache/dolphinscheduler/plugin/task/sql/HiveSqlLogThread.java:
##########
@@ -0,0 +1,82 @@
+/*
+ * 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.sql;
+
+import org.apache.dolphinscheduler.common.utils.LoggerUtils;
+import org.apache.dolphinscheduler.plugin.task.api.TaskExecutionContext;
+import org.apache.dolphinscheduler.plugin.task.api.utils.LogUtils;
+
+import org.apache.hive.jdbc.HivePreparedStatement;
+
+import java.sql.PreparedStatement;
+import java.sql.SQLException;
+import java.util.List;
+
+import org.slf4j.Logger;
+
+
+public class HiveSqlLogThread extends Thread {
+
+    private final HivePreparedStatement statement;
+
+    private final Logger hiveMapReduceLogger;
+
+    private final TaskExecutionContext taskExecutionContext;
+
+    public HiveSqlLogThread(PreparedStatement statement, Logger logger, TaskExecutionContext taskExecutionContext) throws SQLException {
+        this.statement = statement.unwrap(HivePreparedStatement.class);

Review Comment:
   super.setDeamon(true)



##########
dolphinscheduler-task-plugin/dolphinscheduler-task-sql/src/main/java/org/apache/dolphinscheduler/plugin/task/sql/HiveSqlLogThread.java:
##########
@@ -0,0 +1,82 @@
+/*
+ * 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.sql;
+
+import org.apache.dolphinscheduler.common.utils.LoggerUtils;
+import org.apache.dolphinscheduler.plugin.task.api.TaskExecutionContext;
+import org.apache.dolphinscheduler.plugin.task.api.utils.LogUtils;
+
+import org.apache.hive.jdbc.HivePreparedStatement;
+
+import java.sql.PreparedStatement;
+import java.sql.SQLException;
+import java.util.List;
+
+import org.slf4j.Logger;
+
+
+public class HiveSqlLogThread extends Thread {
+
+    private final HivePreparedStatement statement;
+
+    private final Logger hiveMapReduceLogger;
+
+    private final TaskExecutionContext taskExecutionContext;
+
+    public HiveSqlLogThread(PreparedStatement statement, Logger logger, TaskExecutionContext taskExecutionContext) throws SQLException {
+        this.statement = statement.unwrap(HivePreparedStatement.class);
+        this.hiveMapReduceLogger = logger;
+        this.taskExecutionContext = taskExecutionContext;
+    }
+
+    @Override
+    public void run() {
+        String taskLogName = LoggerUtils.buildTaskId(taskExecutionContext.getFirstSubmitTime(),
+            taskExecutionContext.getProcessDefineCode(),
+            taskExecutionContext.getProcessDefineVersion(),
+            taskExecutionContext.getProcessInstanceId(),
+            taskExecutionContext.getTaskInstanceId());
+        taskExecutionContext.setTaskLogName(taskLogName);
+
+        // set the name of the current thread
+        Thread.currentThread().setName(taskLogName);

Review Comment:
   You can directly use the sql task's logger to print the log.



-- 
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] fengjian1129 commented on pull request #11598: [Improvement-#9680][task-plugin] #9680 add hive sql log listener, print MapReduce progress log and obtain application_id allows developers to better track tasks

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

   @EricGao888 I fixed 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] sonarcloud[bot] commented on pull request #11598: [Improvement-#9680][task-plugin] #9680 add hive sql log listener, print MapReduce progress log and obtain application_id allows developers to better track tasks

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

   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=11598)
   
   [![Bug](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/bug-16px.png 'Bug')](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=11598&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=11598&resolved=false&types=BUG) [0 Bugs](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=11598&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=11598&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=11598&resolved=false&types=VULNERABILITY) [0 Vulnerabilities](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=11598&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=11598&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=11598&resolved=false&types=SECURITY_HOTSPOT) [0 Security Hotspots](https://sonarcloud.io/project/security_hotspots?id=apache-dolphinscheduler&pullRequest=11598&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=11598&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=11598&resolved=false&types=CODE_SMELL) [0 Code Smells](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=11598&resolved=false&types=CODE_SMELL)
   
   [![30.4%](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/CoverageChart/25-16px.png '30.4%')](https://sonarcloud.io/component_measures?id=apache-dolphinscheduler&pullRequest=11598&metric=new_coverage&view=list) [30.4% Coverage](https://sonarcloud.io/component_measures?id=apache-dolphinscheduler&pullRequest=11598&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=11598&metric=new_duplicated_lines_density&view=list) [0.0% Duplication](https://sonarcloud.io/component_measures?id=apache-dolphinscheduler&pullRequest=11598&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] fengjian1129 commented on pull request #11598: [Improvement-#9680][task-plugin] #9680 add hive sql log listener, print MapReduce progress log and obtain application_id allows developers to better track tasks

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

   @ruanwenjun hi bro, thank you for checking it for me. I have modified the code as required


-- 
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] fengjian1129 commented on a diff in pull request #11598: [Improvement-#9680][task-plugin] #9680 add hive sql log listener, print MapReduce progress log and obtain application_id allows developers to better track tasks

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


##########
dolphinscheduler-task-plugin/dolphinscheduler-task-sql/src/main/java/org/apache/dolphinscheduler/plugin/task/sql/SqlTask.java:
##########
@@ -365,13 +379,31 @@ private String executeUpdate(Connection connection, List<SqlBinds> statementsBin
         int result = 0;
         for (SqlBinds sqlBind : statementsBinds) {
             try (PreparedStatement statement = prepareStatementAndBind(connection, sqlBind)) {
+
+                sqlLogListener(statement);
+
                 result = statement.executeUpdate();
                 logger.info("{} statement execute update result: {}, for sql: {}", handlerType, result, sqlBind.getSql());
             }
         }
         return String.valueOf(result);
     }
 
+    /**
+     * sql log listener
+     *
+     * @param statement statement
+     * @throws SQLException sqlException
+     */
+    private void sqlLogListener(PreparedStatement statement) throws SQLException {
+        if (DbType.HIVE.name().equals(sqlParameters.getType())) {
+            logger.info("The current sql task type is HIVE, will start a demon thread to resolve the appIds");

Review Comment:
   ok, I will modify it according to your suggestions. Can you help me merge the code after modification



##########
dolphinscheduler-task-plugin/dolphinscheduler-task-sql/src/main/java/org/apache/dolphinscheduler/plugin/task/sql/SqlTask.java:
##########
@@ -365,13 +379,31 @@ private String executeUpdate(Connection connection, List<SqlBinds> statementsBin
         int result = 0;
         for (SqlBinds sqlBind : statementsBinds) {
             try (PreparedStatement statement = prepareStatementAndBind(connection, sqlBind)) {
+
+                sqlLogListener(statement);
+
                 result = statement.executeUpdate();
                 logger.info("{} statement execute update result: {}, for sql: {}", handlerType, result, sqlBind.getSql());
             }
         }
         return String.valueOf(result);
     }
 
+    /**
+     * sql log listener
+     *
+     * @param statement statement
+     * @throws SQLException sqlException
+     */
+    private void sqlLogListener(PreparedStatement statement) throws SQLException {
+        if (DbType.HIVE.name().equals(sqlParameters.getType())) {
+            logger.info("The current sql task type is HIVE, will start a demon thread to resolve the appIds");

Review Comment:
   ok, I will modify it according to your suggestions. Can you help me merge the code after modification



-- 
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] EricGao888 commented on a diff in pull request #11598: [Improvement-#9680][task-plugin] add hive sql log listener, print MapReduce progress log and obtain application_id allows developers to better track tasks

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


##########
dolphinscheduler-task-plugin/dolphinscheduler-task-sql/src/main/java/org/apache/dolphinscheduler/plugin/task/sql/HiveSqlLogThread.java:
##########
@@ -0,0 +1,76 @@
+/*
+ * 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.sql;
+
+import org.apache.dolphinscheduler.common.utils.LoggerUtils;
+import org.apache.dolphinscheduler.plugin.task.api.TaskExecutionContext;
+
+import org.apache.hive.jdbc.HiveStatement;
+
+import java.sql.SQLException;
+import java.sql.Statement;
+import java.util.List;
+
+import org.slf4j.Logger;
+
+/**
+ * hive log listener thread
+ */
+public class HiveSqlLogThread extends Thread {
+    /**
+     * hive statement
+     */
+    private final HiveStatement statement;
+    /**
+     * logger
+     */
+    private final Logger threadLogger;
+
+    private final TaskExecutionContext taskExecutionContext;
+
+    public HiveSqlLogThread(Statement statement, Logger logger, TaskExecutionContext taskExecutionContext) {
+        this.statement = (HiveStatement) statement;
+        this.threadLogger = logger;
+        this.taskExecutionContext = taskExecutionContext;
+    }
+
+    @Override
+    public void run() {
+        if (statement == null) {
+            threadLogger.info("hive statement is null,end this log query!");
+            return;
+        }
+        try {
+            while (!statement.isClosed() && statement.hasMoreLogs()) {
+                for (String log : statement.getQueryLog(true, 500)) {
+                    //hive mapreduce log
+                    threadLogger.info(log);

Review Comment:
   What about changing the name to `hiveMapReduceLogger` and remove the comment?



-- 
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] fengjian1129 commented on pull request #11598: [Improvement-#9680][task-plugin] add hive sql log listener, print MapReduce progress log and obtain application_id allows developers to better track tasks

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

   @EricGao888  look 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 #11598: [Improvement-#9680][task-plugin] add hive sql log listener, print MapReduce progress log and obtain application_id allows developers to better track tasks

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

   Kudos, SonarCloud Quality Gate passed!&nbsp; &nbsp; [![Quality Gate passed](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/QualityGateBadge/passed-16px.png 'Quality Gate passed')](https://sonarcloud.io/dashboard?id=apache-dolphinscheduler&pullRequest=11598)
   
   [![Bug](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/bug-16px.png 'Bug')](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=11598&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=11598&resolved=false&types=BUG) [0 Bugs](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=11598&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=11598&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=11598&resolved=false&types=VULNERABILITY) [0 Vulnerabilities](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=11598&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=11598&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=11598&resolved=false&types=SECURITY_HOTSPOT) [0 Security Hotspots](https://sonarcloud.io/project/security_hotspots?id=apache-dolphinscheduler&pullRequest=11598&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=11598&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=11598&resolved=false&types=CODE_SMELL) [1 Code Smell](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=11598&resolved=false&types=CODE_SMELL)
   
   [![46.3%](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/CoverageChart/40-16px.png '46.3%')](https://sonarcloud.io/component_measures?id=apache-dolphinscheduler&pullRequest=11598&metric=new_coverage&view=list) [46.3% Coverage](https://sonarcloud.io/component_measures?id=apache-dolphinscheduler&pullRequest=11598&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=11598&metric=new_duplicated_lines_density&view=list) [0.0% Duplication](https://sonarcloud.io/component_measures?id=apache-dolphinscheduler&pullRequest=11598&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] EricGao888 commented on a diff in pull request #11598: [Improvement-#9680][task-plugin] add hive sql log listener, print MapReduce progress log and obtain application_id allows developers to better track tasks

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


##########
dolphinscheduler-task-plugin/dolphinscheduler-task-sql/src/main/java/org/apache/dolphinscheduler/plugin/task/sql/HiveSqlLogThread.java:
##########
@@ -0,0 +1,76 @@
+/*
+ * 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.sql;
+
+import org.apache.dolphinscheduler.common.utils.LoggerUtils;
+import org.apache.dolphinscheduler.plugin.task.api.TaskExecutionContext;
+
+import org.apache.hive.jdbc.HiveStatement;
+
+import java.sql.SQLException;
+import java.sql.Statement;
+import java.util.List;
+
+import org.slf4j.Logger;
+
+/**
+ * hive log listener thread
+ */
+public class HiveSqlLogThread extends Thread {
+    /**
+     * hive statement
+     */
+    private final HiveStatement statement;
+    /**
+     * logger
+     */
+    private final Logger threadLogger;
+
+    private final TaskExecutionContext taskExecutionContext;
+
+    public HiveSqlLogThread(Statement statement, Logger logger, TaskExecutionContext taskExecutionContext) {
+        this.statement = (HiveStatement) statement;
+        this.threadLogger = logger;
+        this.taskExecutionContext = taskExecutionContext;
+    }
+
+    @Override
+    public void run() {
+        if (statement == null) {
+            threadLogger.info("hive statement is null,end this log query!");

Review Comment:
   ```suggestion
               threadLogger.info("hive statement is null, end this log query!");
   ```



##########
dolphinscheduler-task-plugin/dolphinscheduler-task-sql/src/main/java/org/apache/dolphinscheduler/plugin/task/sql/HiveSqlLogThread.java:
##########
@@ -0,0 +1,76 @@
+/*
+ * 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.sql;
+
+import org.apache.dolphinscheduler.common.utils.LoggerUtils;
+import org.apache.dolphinscheduler.plugin.task.api.TaskExecutionContext;
+
+import org.apache.hive.jdbc.HiveStatement;
+
+import java.sql.SQLException;
+import java.sql.Statement;
+import java.util.List;
+
+import org.slf4j.Logger;
+
+/**
+ * hive log listener thread
+ */
+public class HiveSqlLogThread extends Thread {
+    /**
+     * hive statement
+     */
+    private final HiveStatement statement;
+    /**
+     * logger

Review Comment:
   We do not need to comment if people could easily understand it from field name.



##########
dolphinscheduler-task-plugin/dolphinscheduler-task-sql/src/test/java/org/apache/dolphinscheduler/plugin/task/sql/HiveSqlLogThreadTest.java:
##########
@@ -0,0 +1,68 @@
+/*
+ * 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.sql;
+
+import org.apache.dolphinscheduler.plugin.task.api.TaskExecutionContext;
+
+import org.apache.hive.jdbc.HiveStatement;
+import java.sql.SQLException;
+import java.util.ArrayList;
+import java.util.List;
+import org.junit.Assert;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.powermock.api.mockito.PowerMockito;
+import org.powermock.core.classloader.annotations.PowerMockIgnore;
+import org.powermock.modules.junit4.PowerMockRunner;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * hive sql listener test
+ */
+@RunWith(PowerMockRunner.class)
+@PowerMockIgnore({"javax.*"})
+public class HiveSqlLogThreadTest {
+    private static final Logger LOGGER = LoggerFactory.getLogger(HiveSqlLogThreadTest.class);
+
+    @Test
+    public void testHiveSql() throws SQLException {
+        TaskExecutionContext taskExecutionContext = PowerMockito.mock(TaskExecutionContext.class);
+        PowerMockito.when(taskExecutionContext.getTaskType()).thenReturn("hive");
+
+        String sql = "select count(*) from test.table";
+
+        List<String> mockLog = new ArrayList<>();
+        mockLog.add("start hive sql log\napplication_1231_2323");
+        HiveStatement statement = PowerMockito.mock(HiveStatement.class);
+        PowerMockito.when(statement.isClosed()).thenReturn(false);
+        PowerMockito.when(statement.hasMoreLogs()).thenReturn(true);
+        PowerMockito.when(statement.getQueryLog(true,500)).thenReturn(mockLog);
+        try {
+            //print process log
+            HiveSqlLogThread queryThread = PowerMockito.spy(new HiveSqlLogThread(statement,LOGGER,taskExecutionContext));

Review Comment:
   ```suggestion
               HiveSqlLogThread queryThread = PowerMockito.spy(new HiveSqlLogThread(statement, LOGGER, taskExecutionContext));
   ```



##########
dolphinscheduler-task-plugin/dolphinscheduler-task-sql/src/main/java/org/apache/dolphinscheduler/plugin/task/sql/HiveSqlLogThread.java:
##########
@@ -0,0 +1,76 @@
+/*
+ * 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.sql;
+
+import org.apache.dolphinscheduler.common.utils.LoggerUtils;
+import org.apache.dolphinscheduler.plugin.task.api.TaskExecutionContext;
+
+import org.apache.hive.jdbc.HiveStatement;
+
+import java.sql.SQLException;
+import java.sql.Statement;
+import java.util.List;
+
+import org.slf4j.Logger;
+
+/**

Review Comment:
   We do not need the comment here. People know it from the class name.



##########
dolphinscheduler-task-plugin/dolphinscheduler-task-sql/src/main/java/org/apache/dolphinscheduler/plugin/task/sql/SqlTask.java:
##########
@@ -303,6 +310,13 @@ private String executeUpdate(Connection connection, List<SqlBinds> statementsBin
         int result = 0;
         for (SqlBinds sqlBind : statementsBinds) {
             try (PreparedStatement statement = prepareStatementAndBind(connection, sqlBind)) {
+                //hive log listener

Review Comment:
   The same as above.



##########
dolphinscheduler-task-plugin/dolphinscheduler-task-sql/src/test/java/org/apache/dolphinscheduler/plugin/task/sql/HiveSqlLogThreadTest.java:
##########
@@ -0,0 +1,68 @@
+/*
+ * 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.sql;
+
+import org.apache.dolphinscheduler.plugin.task.api.TaskExecutionContext;
+
+import org.apache.hive.jdbc.HiveStatement;
+import java.sql.SQLException;
+import java.util.ArrayList;
+import java.util.List;
+import org.junit.Assert;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.powermock.api.mockito.PowerMockito;
+import org.powermock.core.classloader.annotations.PowerMockIgnore;
+import org.powermock.modules.junit4.PowerMockRunner;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * hive sql listener test

Review Comment:
   We do not need comment here.



##########
dolphinscheduler-task-plugin/dolphinscheduler-task-sql/src/main/java/org/apache/dolphinscheduler/plugin/task/sql/HiveSqlLogThread.java:
##########
@@ -0,0 +1,76 @@
+/*
+ * 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.sql;
+
+import org.apache.dolphinscheduler.common.utils.LoggerUtils;
+import org.apache.dolphinscheduler.plugin.task.api.TaskExecutionContext;
+
+import org.apache.hive.jdbc.HiveStatement;
+
+import java.sql.SQLException;
+import java.sql.Statement;
+import java.util.List;
+
+import org.slf4j.Logger;
+
+/**
+ * hive log listener thread
+ */
+public class HiveSqlLogThread extends Thread {
+    /**

Review Comment:
   We don't need this comment either.



##########
dolphinscheduler-task-plugin/dolphinscheduler-task-sql/src/test/java/org/apache/dolphinscheduler/plugin/task/sql/HiveSqlLogThreadTest.java:
##########
@@ -0,0 +1,68 @@
+/*
+ * 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.sql;
+
+import org.apache.dolphinscheduler.plugin.task.api.TaskExecutionContext;
+
+import org.apache.hive.jdbc.HiveStatement;
+import java.sql.SQLException;
+import java.util.ArrayList;
+import java.util.List;
+import org.junit.Assert;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.powermock.api.mockito.PowerMockito;
+import org.powermock.core.classloader.annotations.PowerMockIgnore;
+import org.powermock.modules.junit4.PowerMockRunner;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * hive sql listener test
+ */
+@RunWith(PowerMockRunner.class)
+@PowerMockIgnore({"javax.*"})
+public class HiveSqlLogThreadTest {
+    private static final Logger LOGGER = LoggerFactory.getLogger(HiveSqlLogThreadTest.class);
+
+    @Test
+    public void testHiveSql() throws SQLException {
+        TaskExecutionContext taskExecutionContext = PowerMockito.mock(TaskExecutionContext.class);
+        PowerMockito.when(taskExecutionContext.getTaskType()).thenReturn("hive");
+
+        String sql = "select count(*) from test.table";
+
+        List<String> mockLog = new ArrayList<>();
+        mockLog.add("start hive sql log\napplication_1231_2323");
+        HiveStatement statement = PowerMockito.mock(HiveStatement.class);
+        PowerMockito.when(statement.isClosed()).thenReturn(false);
+        PowerMockito.when(statement.hasMoreLogs()).thenReturn(true);
+        PowerMockito.when(statement.getQueryLog(true,500)).thenReturn(mockLog);

Review Comment:
   ```suggestion
           PowerMockito.when(statement.getQueryLog(true, 500)).thenReturn(mockLog);
   ```



##########
dolphinscheduler-task-plugin/dolphinscheduler-task-sql/src/main/java/org/apache/dolphinscheduler/plugin/task/sql/HiveSqlLogThread.java:
##########
@@ -0,0 +1,76 @@
+/*
+ * 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.sql;
+
+import org.apache.dolphinscheduler.common.utils.LoggerUtils;
+import org.apache.dolphinscheduler.plugin.task.api.TaskExecutionContext;
+
+import org.apache.hive.jdbc.HiveStatement;
+
+import java.sql.SQLException;
+import java.sql.Statement;
+import java.util.List;
+
+import org.slf4j.Logger;
+
+/**
+ * hive log listener thread
+ */
+public class HiveSqlLogThread extends Thread {
+    /**
+     * hive statement
+     */
+    private final HiveStatement statement;
+    /**
+     * logger
+     */
+    private final Logger threadLogger;
+
+    private final TaskExecutionContext taskExecutionContext;
+
+    public HiveSqlLogThread(Statement statement, Logger logger, TaskExecutionContext taskExecutionContext) {
+        this.statement = (HiveStatement) statement;
+        this.threadLogger = logger;
+        this.taskExecutionContext = taskExecutionContext;
+    }
+
+    @Override
+    public void run() {
+        if (statement == null) {
+            threadLogger.info("hive statement is null,end this log query!");
+            return;
+        }
+        try {
+            while (!statement.isClosed() && statement.hasMoreLogs()) {
+                for (String log : statement.getQueryLog(true, 500)) {
+                    //hive mapreduce log
+                    threadLogger.info(log);

Review Comment:
   What about change the name to `hiveMapReduceLogger` and remove the comment?



##########
dolphinscheduler-task-plugin/dolphinscheduler-task-sql/src/main/java/org/apache/dolphinscheduler/plugin/task/sql/SqlTask.java:
##########
@@ -294,6 +294,13 @@ private void sendAttachment(int groupId, String title, String content) {
     private String executeQuery(Connection connection, SqlBinds sqlBinds, String handlerType) throws Exception {
         try (PreparedStatement statement = prepareStatementAndBind(connection, sqlBinds)) {
             logger.info("{} statement execute query, for sql: {}", handlerType, sqlBinds.getSql());
+            //hive log listener

Review Comment:
   The same here, we could use a better name and remove the comment.



##########
dolphinscheduler-task-plugin/dolphinscheduler-task-sql/src/test/java/org/apache/dolphinscheduler/plugin/task/sql/HiveSqlLogThreadTest.java:
##########
@@ -0,0 +1,68 @@
+/*
+ * 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.sql;
+
+import org.apache.dolphinscheduler.plugin.task.api.TaskExecutionContext;
+
+import org.apache.hive.jdbc.HiveStatement;
+import java.sql.SQLException;
+import java.util.ArrayList;
+import java.util.List;
+import org.junit.Assert;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.powermock.api.mockito.PowerMockito;
+import org.powermock.core.classloader.annotations.PowerMockIgnore;
+import org.powermock.modules.junit4.PowerMockRunner;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * hive sql listener test
+ */
+@RunWith(PowerMockRunner.class)
+@PowerMockIgnore({"javax.*"})
+public class HiveSqlLogThreadTest {
+    private static final Logger LOGGER = LoggerFactory.getLogger(HiveSqlLogThreadTest.class);
+
+    @Test
+    public void testHiveSql() throws SQLException {
+        TaskExecutionContext taskExecutionContext = PowerMockito.mock(TaskExecutionContext.class);
+        PowerMockito.when(taskExecutionContext.getTaskType()).thenReturn("hive");
+
+        String sql = "select count(*) from test.table";
+
+        List<String> mockLog = new ArrayList<>();
+        mockLog.add("start hive sql log\napplication_1231_2323");
+        HiveStatement statement = PowerMockito.mock(HiveStatement.class);
+        PowerMockito.when(statement.isClosed()).thenReturn(false);
+        PowerMockito.when(statement.hasMoreLogs()).thenReturn(true);
+        PowerMockito.when(statement.getQueryLog(true,500)).thenReturn(mockLog);
+        try {
+            //print process log
+            HiveSqlLogThread queryThread = PowerMockito.spy(new HiveSqlLogThread(statement,LOGGER,taskExecutionContext));
+            queryThread.start();
+
+            Assert.assertEquals(taskExecutionContext.getTaskType(), "hive");

Review Comment:
   Verifying `task type` does not make too much sense here, you need to verify the method calls and your `mockLog`.



-- 
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] EricGao888 commented on pull request #11598: [Improvement-#9680][task-plugin] add hive sql log listener, print MapReduce progress log and obtain application_id allows developers to better track tasks

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

   This is a good feature. Thanks @fengjian1129 for submitting the PR! I add some comments. Could you please also take a look when available? @SbloodyS Thanks!


-- 
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


Re: [PR] [Improvement-#9680][task-plugin] #9680 add hive sql log listener, print MapReduce progress log and obtain application_id allows developers to better track tasks [dolphinscheduler]

Posted by "github-actions[bot] (via GitHub)" <gi...@apache.org>.
github-actions[bot] commented on PR #11598:
URL: https://github.com/apache/dolphinscheduler/pull/11598#issuecomment-1939859954

   This pull request has been closed because it has not had recent activity. You could reopen it if you try to continue your work, and anyone who are interested in it are encouraged to continue work on this pull request.


-- 
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