Unverified Commit 9b75bebf authored by qiaozhanwei's avatar qiaozhanwei Committed by GitHub
Browse files

1,add sendAsync method 2,refactor LoggerClient (#1965)



* 1,remove dolphinscheduler-rpc module  2,add dolphinscheduler-remote module 3,add dolphinscheduler-service module 4,refactor LoggerServer module (#1925)

* 1,remove dolphinscheduler-rpc module
2,add dolphinscheduler-remote module
3,add dolphinscheduler-service module
4,refactor LoggerServer module

* ProcessUtils modify

* Refactor architecture (#1926)

* move version to parent pom

* move version properties to parent pom for easy management

* remove freemarker dependency

* delete CombinedApplicationServer

* #1871 correct spelling

* #1873 some updates for TaskQueueZkImpl

* #1875 remove unused properties in pom

* #1878
1. remove tomcat dependency
2. remove combined_logback.xml in api module
3. format pom.xml for not aligning

* #1885 fix api server startup failure
1. add jsp-2.1 dependency
2. remove jasper-runtime dependency

* add stringutils ut (#1921)

* add stringutils ut

* Newfeature for #1675. (#1908)

Continue to finish the rest works, add the cache feature for dependence,mr,python,sub_process,procedure and shell.

* Add modify user name for process definition (#1919)

* class overrides equals() and should therefore also override hashCode()

* #1862 add modify user in process difinition list

* #1862 add pg-1.2.2 ddl.sql

* modify ScriptRunnerTest

* add updateProessDifinition UT

* modify updateProcessDifinition UT

* modify updateProcessDifinition UT

* modify mysql 1.2.2 ddl.sql&dml.sql

* add scope test to mysql in pom

* modify pg-1.2.2 ddl.sql

* refactor module

* updates

Co-authored-by: default avatarkhadgarmage <khadgar.mage@outlook.com>
Co-authored-by: default avatarzhukai <boness@qq.com>
Co-authored-by: default avatarYelli <amarantine@my.com>

* dolphinscheduler-common remove spring (#1931)

* dolphinscheduler-common remove spring

* dolphinscheduler-common remove spring

* dolphinscheduler-common remove spring

* dolphinscheduler-common remove spring

* dolphinscheduler-common remove spring

* SpringApplicationContext class title add license (#1932)

* dolphinscheduler-common remove spring

* dolphinscheduler-common remove spring

* dolphinscheduler-common remove spring

* dolphinscheduler-common remove spring

* dolphinscheduler-common remove spring

* dolphinscheduler-common remove spring

* add license (#1934)

* dolphinscheduler-common remove spring

* dolphinscheduler-common remove spring

* dolphinscheduler-common remove spring

* dolphinscheduler-common remove spring

* dolphinscheduler-common remove spring

* dolphinscheduler-common remove spring

* move datasource classes to dao module

* fix send4LetterWord bug

* Refactor architecture (#1936)

* move datasource classes to dao module

* fix send4LetterWord bug

* exclude jasper-compiler in case of runtime conflict

* LoggerServiceTest remove ProcessDao (#1944)

* dolphinscheduler-common remove spring

* dolphinscheduler-common remove spring

* dolphinscheduler-common remove spring

* dolphinscheduler-common remove spring

* dolphinscheduler-common remove spring

* dolphinscheduler-common remove spring

* LoggerServiceTest remove ProcessDao

* exclude jasper-compiler in case of runtime conflict (#1938)

* move datasource classes to dao module

* fix send4LetterWord bug

* exclude jasper-compiler in case of runtime conflict

* DataAnaylysisServiceTest and ProcessDefinitionService modify

* remote module add comment

* OSUtilsTest modify

* add finally block to close channel

* add finally block to close channel (#1951)

* move datasource classes to dao module

* fix send4LetterWord bug

* exclude jasper-compiler in case of runtime conflict

* add finally block to close channel

* refactor log client service

* add sendSync method

* 1,quartz.properties add conf category
2,dolphinscheduler-daemon.sh modify

* dolphinscheduler-binary.xml modify

* add sendAsync method (#1962)

* add sendAsync method

* add sendAsync method

* 1,add sendAsync method
2,refactor LoggerClient

* LogClientService modify

Co-authored-by: default avatarTboy <technoboy@yeah.net>
Co-authored-by: default avatarkhadgarmage <khadgar.mage@outlook.com>
Co-authored-by: default avatarzhukai <boness@qq.com>
Co-authored-by: default avatarYelli <amarantine@my.com>
parent d674eaba
Loading
Loading
Loading
Loading
+17 −22
Original line number Diff line number Diff line
@@ -28,6 +28,8 @@ import org.slf4j.LoggerFactory;
import org.springframework.beans.factory.annotation.Autowired;
import org.springframework.stereotype.Service;

import javax.annotation.PreDestroy;

/**
 * log service
 */
@@ -39,6 +41,17 @@ public class LoggerService {
  @Autowired
  private ProcessService processService;

  private final LogClientService logClient;

  public LoggerService(){
    logClient = new LogClientService();
  }

  @PreDestroy
  public void close(){
    logClient.close();
  }

  /**
   * view log
   *
@@ -64,18 +77,9 @@ public class LoggerService {
    Result result = new Result(Status.SUCCESS.getCode(), Status.SUCCESS.getMsg());

    logger.info("log host : {} , logPath : {} , logServer port : {}",host,taskInstance.getLogPath(),Constants.RPC_PORT);
    LogClientService logClient = null;
    try {
      logClient = new LogClientService(host, Constants.RPC_PORT);
      String log = logClient.rollViewLog(taskInstance.getLogPath(),skipLineNum,limit);
    String log = logClient.rollViewLog(host, Constants.RPC_PORT, taskInstance.getLogPath(),skipLineNum,limit);
    result.setData(log);
    logger.info(log);
    } finally {
      if(logClient != null){
        logClient.close();
      }
    }

    return result;
  }

@@ -90,16 +94,7 @@ public class LoggerService {
    if (taskInstance == null){
      throw new RuntimeException("task instance is null");
    }

    String host = taskInstance.getHost();
    LogClientService logClient = null;
    try {
      logClient = new LogClientService(host, Constants.RPC_PORT);
      return logClient.getLogBytes(taskInstance.getLogPath());
    } finally {
      if(logClient != null){
        logClient.close();
      }
    }
    return logClient.getLogBytes(host, Constants.RPC_PORT, taskInstance.getLogPath());
  }
}
+137 −73
Original line number Diff line number Diff line
@@ -25,21 +25,22 @@ import io.netty.channel.socket.nio.NioSocketChannel;
import org.apache.dolphinscheduler.remote.codec.NettyDecoder;
import org.apache.dolphinscheduler.remote.codec.NettyEncoder;
import org.apache.dolphinscheduler.remote.command.Command;
import org.apache.dolphinscheduler.remote.command.CommandType;
import org.apache.dolphinscheduler.remote.config.NettyClientConfig;
import org.apache.dolphinscheduler.remote.exceptions.RemotingException;
import org.apache.dolphinscheduler.remote.exceptions.RemotingTimeoutException;
import org.apache.dolphinscheduler.remote.exceptions.RemotingTooMuchRequestException;
import org.apache.dolphinscheduler.remote.future.InvokeCallback;
import org.apache.dolphinscheduler.remote.future.ReleaseSemaphore;
import org.apache.dolphinscheduler.remote.future.ResponseFuture;
import org.apache.dolphinscheduler.remote.handler.NettyClientHandler;
import org.apache.dolphinscheduler.remote.processor.NettyRequestProcessor;
import org.apache.dolphinscheduler.remote.utils.Address;
import org.apache.dolphinscheduler.remote.utils.Constants;
import org.apache.dolphinscheduler.remote.utils.CallerThreadExecutePolicy;
import org.apache.dolphinscheduler.remote.utils.NamedThreadFactory;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;

import java.net.InetSocketAddress;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.Executors;
import java.util.concurrent.ThreadFactory;
import java.util.concurrent.*;
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.atomic.AtomicInteger;

@@ -51,7 +52,7 @@ public class NettyRemotingClient {
    private final Logger logger = LoggerFactory.getLogger(NettyRemotingClient.class);

    /**
     *  bootstrap
     * client bootstrap
     */
    private final Bootstrap bootstrap = new Bootstrap();

@@ -63,12 +64,7 @@ public class NettyRemotingClient {
    /**
     * channels
     */
    private final ConcurrentHashMap<Address, Channel> channels = new ConcurrentHashMap();

    /**
     *  default executor
     */
    private final ExecutorService defaultExecutor = Executors.newFixedThreadPool(Constants.CPUS);
    private final ConcurrentHashMap<Address, Channel> channels = new ConcurrentHashMap(128);

    /**
     *  started flag
@@ -81,18 +77,27 @@ public class NettyRemotingClient {
    private final NioEventLoopGroup workerGroup;

    /**
     *  client handler
     *  client config
     */
    private final NettyClientHandler clientHandler = new NettyClientHandler(this);
    private final NettyClientConfig clientConfig;

    /**
     *  netty client config
     *  saync semaphore
     */
    private final NettyClientConfig clientConfig;
    private final Semaphore asyncSemaphore = new Semaphore(200, true);

    /**
     *  netty client init
     *
     *  callback thread executor
     */
    private final ExecutorService callbackExecutor;

    /**
     *  client handler
     */
    private final NettyClientHandler clientHandler;

    /**
     *  client init
     * @param clientConfig client config
     */
    public NettyRemotingClient(final NettyClientConfig clientConfig){
@@ -105,11 +110,16 @@ public class NettyRemotingClient {
                return new Thread(r, String.format("NettyClient_%d", this.threadIndex.incrementAndGet()));
            }
        });
        this.callbackExecutor = new ThreadPoolExecutor(5, 10, 1, TimeUnit.MINUTES,
                new LinkedBlockingQueue<>(1000), new NamedThreadFactory("CallbackExecutor", 10),
                new CallerThreadExecutePolicy());
        this.clientHandler = new NettyClientHandler(this, callbackExecutor);

        this.start();
    }

    /**
     *  netty server start
     *  start
     */
    private void start(){

@@ -129,63 +139,125 @@ public class NettyRemotingClient {
                                encoder);
                    }
                });
        //
        isStarted.compareAndSet(false, true);
    }

    /**
     *  register processor
     *
     * @param commandType command type
     * @param processor processor
     *  async send
     * @param address address
     * @param command command
     * @param timeoutMillis timeoutMillis
     * @param invokeCallback callback function
     * @throws InterruptedException
     * @throws RemotingException
     */
    public void registerProcessor(final CommandType commandType, final NettyRequestProcessor processor) {
        registerProcessor(commandType, processor, null);
    public void sendAsync(final Address address, final Command command,
                          final long timeoutMillis,
                          final InvokeCallback invokeCallback) throws InterruptedException, RemotingException {
        final Channel channel = getChannel(address);
        if (channel == null) {
            throw new RemotingException("network error");
        }
        /**
         * request unique identification
         */
        final long opaque = command.getOpaque();
        /**
         *  control concurrency number
         */
        boolean acquired = this.asyncSemaphore.tryAcquire(timeoutMillis, TimeUnit.MILLISECONDS);
        if(acquired){
            final ReleaseSemaphore releaseSemaphore = new ReleaseSemaphore(this.asyncSemaphore);

            /**
     * register processor
     *
     * @param commandType command type
     * @param processor processor
     * @param executor thread executor
             *  response future
             */
    public void registerProcessor(final CommandType commandType, final NettyRequestProcessor processor, final ExecutorService executor) {
        this.clientHandler.registerProcessor(commandType, processor, executor);
            final ResponseFuture responseFuture = new ResponseFuture(opaque,
                    timeoutMillis,
                    invokeCallback,
                    releaseSemaphore);
            try {
                channel.writeAndFlush(command).addListener(new ChannelFutureListener(){

                    @Override
                    public void operationComplete(ChannelFuture future) throws Exception {
                        if(future.isSuccess()){
                            responseFuture.setSendOk(true);
                            return;
                        } else {
                            responseFuture.setSendOk(false);
                        }
                        responseFuture.setCause(future.cause());
                        responseFuture.putResponse(null);
                        try {
                            responseFuture.executeInvokeCallback();
                        } catch (Throwable ex){
                            logger.error("execute callback error", ex);
                        } finally{
                            responseFuture.release();
                        }
                    }
                });
            } catch (Throwable ex){
                responseFuture.release();
                throw new RemotingException(String.format("send command to address: %s failed", address), ex);
            }
        } else{
            String message = String.format("try to acquire async semaphore timeout: %d, waiting thread num: %d, total permits: %d",
                    timeoutMillis, asyncSemaphore.getQueueLength(), asyncSemaphore.availablePermits());
            throw new RemotingTooMuchRequestException(message);
        }
    }

    /**
     *  send connect
     * sync send
     * @param address address
     * @param command command
     * @param timeoutMillis timeoutMillis
     * @return command
     * @throws InterruptedException
     * @throws RemotingException
     */
    public void send(final Address address, final Command command) throws RemotingException {
    public Command sendSync(final Address address, final Command command, final long timeoutMillis) throws InterruptedException, RemotingException {
        final Channel channel = getChannel(address);
        if (channel == null) {
            throw new RemotingException("network error");
            throw new RemotingException(String.format("connect to : %s fail", address));
        }
        try {
        final long opaque = command.getOpaque();
        final ResponseFuture responseFuture = new ResponseFuture(opaque, timeoutMillis, null, null);
        channel.writeAndFlush(command).addListener(new ChannelFutureListener() {

            @Override
            public void operationComplete(ChannelFuture future) throws Exception {
                if(future.isSuccess()){
                        logger.info("sent command {} to {}", command, address);
                    responseFuture.setSendOk(true);
                    return;
                } else {
                        logger.error("send command {} to {} failed, error {}", command, address, future.cause());
                    responseFuture.setSendOk(false);
                }
                responseFuture.setCause(future.cause());
                responseFuture.putResponse(null);
                logger.error("send command {} to address {} failed", command, address);
            }
        });
        } catch (Exception ex) {
            String msg = String.format("send command %s to address %s encounter error", command, address);
            throw new RemotingException(msg, ex);
        /**
         * sync wait for result
         */
        Command result = responseFuture.waitResponse();
        if(result == null){
            if(responseFuture.isSendOK()){
                throw new RemotingTimeoutException(address.toString(), timeoutMillis, responseFuture.getCause());
            } else{
                throw new RemotingException(address.toString(), responseFuture.getCause());
            }
        }
        return result;
    }

    /**
     *  get channel
     * @param address address
     * @return channel
     * @param address
     * @return
     */
    public Channel getChannel(Address address) {
        Channel channel = channels.get(address);
@@ -198,7 +270,7 @@ public class NettyRemotingClient {
    /**
     * create channel
     * @param address address
     * @param isSync is sync
     * @param isSync sync flag
     * @return channel
     */
    public Channel createChannel(Address address, boolean isSync) {
@@ -222,15 +294,7 @@ public class NettyRemotingClient {
    }

    /**
     *  get default thread executor
     * @return thread executor
     */
    public ExecutorService getDefaultExecutor() {
        return defaultExecutor;
    }

    /**
     *  close client
     * close
     */
    public void close() {
        if(isStarted.compareAndSet(true, false)){
@@ -239,8 +303,8 @@ public class NettyRemotingClient {
                if(workerGroup != null){
                    this.workerGroup.shutdownGracefully();
                }
                if(defaultExecutor != null){
                    defaultExecutor.shutdown();
                if(callbackExecutor != null){
                    this.callbackExecutor.shutdownNow();
                }
            } catch (Exception ex) {
                logger.error("netty client close exception", ex);
@@ -250,7 +314,7 @@ public class NettyRemotingClient {
    }

    /**
     *  close channel
     *  close channels
     */
    private void closeChannels(){
        for (Channel channel : this.channels.values()) {
@@ -260,10 +324,10 @@ public class NettyRemotingClient {
    }

    /**
     *  remove channel
     * close channel
     * @param address address
     */
    public void removeChannel(Address address){
    public void closeChannel(Address address){
        Channel channel = this.channels.remove(address);
        if(channel != null){
            channel.close();
+1 −1
Original line number Diff line number Diff line
@@ -49,7 +49,7 @@ public class NettyRemotingServer {
    private final Logger logger = LoggerFactory.getLogger(NettyRemotingServer.class);

    /**
     *  server bootstart
     *  server bootstrap
     */
    private final ServerBootstrap serverBootstrap = new ServerBootstrap();

+5 −1
Original line number Diff line number Diff line
@@ -16,7 +16,10 @@
 */
package org.apache.dolphinscheduler.remote.command;

import com.sun.org.apache.regexp.internal.RE;

import java.io.Serializable;
import java.util.concurrent.atomic.AtomicLong;

/**
 *  receive task log request command and content fill
@@ -24,11 +27,12 @@ import java.io.Serializable;
 */
public class Command implements Serializable {

    private static final long serialVersionUID = 1L;
    private static final AtomicLong REQUEST_ID = new AtomicLong(1);

    public static final byte MAGIC = (byte) 0xbabe;

    public Command(){
        this.opaque = REQUEST_ID.getAndIncrement();
    }

    public Command(long opaque){
+1 −1
Original line number Diff line number Diff line
/*
 * 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.remote.command;
import org.apache.dolphinscheduler.remote.utils.FastJsonSerializer;
import java.io.Serializable;import java.util.List;import java.util.concurrent.atomic.AtomicLong;
/**
 *  execute task request command
 */public class ExecuteTaskRequestCommand implements Serializable {

    private static final AtomicLong REQUEST = new AtomicLong(1);

    /**
     *  task id
     */
    private String taskId;

    /**
     *  attempt id
     */
    private String attemptId;

    /**
     *  application name
     */
    private String applicationName;

    /**
     *  group name
     */
    private String groupName;

    /**
     *  task name
     */
    private String taskName;

    /**
     *  connect port
     */
    private int connectorPort;

    /**
     *  description info
     */
    private String description;

    /**
     *  class name
     */
    private String className;

    /**
     *  method name
     */
    private String methodName;

    /**
     *  params
     */
    private String params;

    /**
     *  shard items
     */
    private List<Integer> shardItems;

    public List<Integer> getShardItems() {
        return shardItems;
    }

    public void setShardItems(List<Integer> shardItems) {
        this.shardItems = shardItems;
    }

    public String getParams() {
        return params;
    }

    public void setParams(String params) {
        this.params = params;
    }

    public String getTaskId() {
        return taskId;
    }

    public void setTaskId(String taskId) {
        this.taskId = taskId;
    }

    public String getApplicationName() {
        return applicationName;
    }

    public void setApplicationName(String applicationName) {
        this.applicationName = applicationName;
    }

    public String getGroupName() {
        return groupName;
    }

    public void setGroupName(String groupName) {
        this.groupName = groupName;
    }

    public String getTaskName() {
        return taskName;
    }

    public void setTaskName(String taskName) {
        this.taskName = taskName;
    }

    public int getConnectorPort() {
        return connectorPort;
    }

    public void setConnectorPort(int connectorPort) {
        this.connectorPort = connectorPort;
    }

    public String getDescription() {
        return description;
    }

    public void setDescription(String description) {
        this.description = description;
    }

    public String getClassName() {
        return className;
    }

    public void setClassName(String className) {
        this.className = className;
    }

    public String getMethodName() {
        return methodName;
    }

    public void setMethodName(String methodName) {
        this.methodName = methodName;
    }

    /**
     *  package request command
     *
     * @return command
     */
    public Command convert2Command(){
        Command command = new Command(REQUEST.getAndIncrement());
        command.setType(CommandType.EXECUTE_TASK_REQUEST);
        byte[] body = FastJsonSerializer.serialize(this);
        command.setBody(body);
        return command;
    }}
 No newline at end of file
/*
 * 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.remote.command;
import org.apache.dolphinscheduler.remote.utils.FastJsonSerializer;
import java.io.Serializable;import java.util.List;import java.util.concurrent.atomic.AtomicLong;
/**
 *  execute task request command
 */public class ExecuteTaskRequestCommand implements Serializable {

    /**
     *  task id
     */
    private String taskId;

    /**
     *  attempt id
     */
    private String attemptId;

    /**
     *  application name
     */
    private String applicationName;

    /**
     *  group name
     */
    private String groupName;

    /**
     *  task name
     */
    private String taskName;

    /**
     *  connector port
     */
    private int connectorPort;

    /**
     *  description info
     */
    private String description;

    /**
     *  class name
     */
    private String className;

    /**
     *  method name
     */
    private String methodName;

    /**
     *  parameters
     */
    private String params;

    /**
     *  shard itemds
     */
    private List<Integer> shardItems;

    public List<Integer> getShardItems() {
        return shardItems;
    }

    public void setShardItems(List<Integer> shardItems) {
        this.shardItems = shardItems;
    }

    public String getParams() {
        return params;
    }

    public void setParams(String params) {
        this.params = params;
    }

    public String getTaskId() {
        return taskId;
    }

    public void setTaskId(String taskId) {
        this.taskId = taskId;
    }

    public String getApplicationName() {
        return applicationName;
    }

    public void setApplicationName(String applicationName) {
        this.applicationName = applicationName;
    }

    public String getGroupName() {
        return groupName;
    }

    public void setGroupName(String groupName) {
        this.groupName = groupName;
    }

    public String getTaskName() {
        return taskName;
    }

    public void setTaskName(String taskName) {
        this.taskName = taskName;
    }

    public int getConnectorPort() {
        return connectorPort;
    }

    public void setConnectorPort(int connectorPort) {
        this.connectorPort = connectorPort;
    }

    public String getDescription() {
        return description;
    }

    public void setDescription(String description) {
        this.description = description;
    }

    public String getClassName() {
        return className;
    }

    public void setClassName(String className) {
        this.className = className;
    }

    public String getMethodName() {
        return methodName;
    }

    public void setMethodName(String methodName) {
        this.methodName = methodName;
    }

    /**
     *  package request command
     *
     * @return command
     */
    public Command convert2Command(){
        Command command = new Command();
        command.setType(CommandType.EXECUTE_TASK_REQUEST);
        byte[] body = FastJsonSerializer.serialize(this);
        command.setBody(body);
        return command;
    }}
 No newline at end of file
Loading