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/23 03:06:33 UTC

[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

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